From d7d15ba51f15f15ac438a05172c6e83e4702f166 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 23 Aug 2022 20:44:01 -0500 Subject: [PATCH] Add druid-multi-stage-query extension. (#12918) * Add druid-multi-stage-query extension. * Adjustments from CI. * Task ID validation. * Various changes from code review. * Remove unnecessary code. * LGTM-related. --- codestyle/spotbugs-exclude.xml | 7 + .../apache/druid/common/utils/IdUtils.java | 4 +- distribution/pom.xml | 6 + .../cluster/_common/common.runtime.properties | 2 +- .../coordinator-overlord/runtime.properties | 1 + .../large/_common/common.runtime.properties | 2 +- .../coordinator-overlord/runtime.properties | 1 + .../medium/_common/common.runtime.properties | 2 +- .../coordinator-overlord/runtime.properties | 1 + .../_common/common.runtime.properties | 2 +- .../coordinator-overlord/runtime.properties | 1 + .../_common/common.runtime.properties | 2 +- .../coordinator-overlord/runtime.properties | 1 + .../small/_common/common.runtime.properties | 2 +- .../coordinator-overlord/runtime.properties | 1 + .../xlarge/_common/common.runtime.properties | 2 +- .../coordinator-overlord/runtime.properties | 1 + extensions-core/multi-stage-query/README.md | 141 ++ extensions-core/multi-stage-query/pom.xml | 299 +++ .../druid/msq/counters/ChannelCounters.java | 274 +++ .../druid/msq/counters/CounterNames.java | 142 ++ .../druid/msq/counters/CounterSnapshots.java | 75 + .../counters/CounterSnapshotsSerializer.java | 55 + .../msq/counters/CounterSnapshotsTree.java | 104 + .../druid/msq/counters/CounterTracker.java | 74 + .../druid/msq/counters/QueryCounter.java | 32 + .../msq/counters/QueryCounterSnapshot.java | 31 + .../SuperSorterProgressTrackerCounter.java | 71 + .../druid/msq/counters/WarningCounters.java | 94 + .../org/apache/druid/msq/exec/Controller.java | 125 + .../druid/msq/exec/ControllerClient.java | 81 + .../druid/msq/exec/ControllerContext.java | 82 + .../apache/druid/msq/exec/ControllerImpl.java | 2171 +++++++++++++++++ .../exec/ExceptionWrappingWorkerClient.java | 145 ++ .../org/apache/druid/msq/exec/Limits.java | 61 + .../org/apache/druid/msq/exec/MSQTasks.java | 216 ++ .../apache/druid/msq/exec/QueryValidator.java | 77 + .../msq/exec/TaskDataSegmentProvider.java | 115 + .../org/apache/druid/msq/exec/Worker.java | 108 + .../apache/druid/msq/exec/WorkerClient.java | 84 + .../apache/druid/msq/exec/WorkerContext.java | 76 + .../org/apache/druid/msq/exec/WorkerImpl.java | 1249 ++++++++++ .../druid/msq/exec/WorkerManagerClient.java | 57 + .../msq/exec/WorkerMemoryParameters.java | 294 +++ .../msq/guice/MSQDurableStorageModule.java | 86 + .../guice/MSQExternalDataSourceModule.java | 52 + .../druid/msq/guice/MSQIndexingModule.java | 201 ++ .../msq/guice/MSQServiceClientModule.java | 86 + .../apache/druid/msq/guice/MSQSqlModule.java | 60 + .../druid/msq/guice/MultiStageQuery.java | 34 + .../apache/druid/msq/guice/SqlTaskModule.java | 53 + .../druid/msq/indexing/ColumnMapping.java | 82 + .../druid/msq/indexing/ColumnMappings.java | 144 ++ .../msq/indexing/ControllerChatHandler.java | 194 ++ .../indexing/CountableInputSourceReader.java | 58 + .../CountingOutputChannelFactory.java | 64 + .../CountingReadableFrameChannel.java | 75 + .../CountingWritableFrameChannel.java | 71 + .../indexing/DataSourceMSQDestination.java | 163 ++ .../msq/indexing/IndexerControllerClient.java | 182 ++ .../indexing/IndexerControllerContext.java | 123 + .../msq/indexing/IndexerFrameContext.java | 120 + .../msq/indexing/IndexerWorkerClient.java | 271 ++ .../msq/indexing/IndexerWorkerContext.java | 296 +++ .../indexing/IndexerWorkerManagerClient.java | 81 + .../msq/indexing/InputChannelFactory.java | 38 + .../druid/msq/indexing/InputChannelsImpl.java | 211 ++ .../KeyStatisticsCollectionProcessor.java | 183 ++ .../druid/msq/indexing/MSQControllerTask.java | 222 ++ .../druid/msq/indexing/MSQDestination.java | 33 + .../msq/indexing/MSQPartitionAssignment.java | 96 + .../apache/druid/msq/indexing/MSQSpec.java | 160 ++ .../druid/msq/indexing/MSQTaskList.java | 71 + .../druid/msq/indexing/MSQTuningConfig.java | 140 ++ .../druid/msq/indexing/MSQWorkerTask.java | 115 + .../msq/indexing/MSQWorkerTaskLauncher.java | 487 ++++ .../SegmentGeneratorFrameProcessor.java | 295 +++ ...SegmentGeneratorFrameProcessorFactory.java | 373 +++ .../indexing/TaskReportMSQDestination.java | 45 + .../druid/msq/indexing/WorkerChatHandler.java | 222 ++ .../msq/indexing/error/BaseMSQFault.java | 121 + .../error/BroadcastTablesTooLargeFault.java | 72 + .../msq/indexing/error/CanceledFault.java | 41 + .../error/CannotParseExternalDataFault.java | 35 + .../error/ColumnNameRestrictedFault.java | 75 + .../error/ColumnTypeNotSupportedFault.java | 89 + .../DurableStorageConfigurationFault.java | 80 + .../indexing/error/FaultsExceededChecker.java | 92 + .../InsertCannotAllocateSegmentFault.java | 82 + .../error/InsertCannotBeEmptyFault.java | 72 + .../InsertCannotOrderByDescendingFault.java | 72 + ...sertCannotReplaceExistingSegmentFault.java | 76 + .../error/InsertLockPreemptedFault.java | 45 + .../indexing/error/InsertTimeNullFault.java | 42 + .../error/InsertTimeOutOfBoundsFault.java | 66 + .../indexing/error/InvalidNullByteFault.java | 71 + .../msq/indexing/error/MSQErrorReport.java | 200 ++ .../msq/indexing/error/MSQException.java | 51 + .../druid/msq/indexing/error/MSQFault.java | 49 + .../MSQWarningReportLimiterPublisher.java | 96 + .../error/MSQWarningReportPublisher.java | 35 + .../MSQWarningReportSimplePublisher.java | 72 + .../druid/msq/indexing/error/MSQWarnings.java | 36 + .../indexing/error/NotEnoughMemoryFault.java | 108 + .../error/QueryNotSupportedFault.java | 41 + .../msq/indexing/error/RowTooLargeFault.java | 69 + .../indexing/error/TaskStartTimeoutFault.java | 76 + .../indexing/error/TooManyBucketsFault.java | 77 + .../indexing/error/TooManyColumnsFault.java | 80 + .../error/TooManyInputFilesFault.java | 102 + .../error/TooManyPartitionsFault.java | 74 + .../indexing/error/TooManyWarningsFault.java | 80 + .../indexing/error/TooManyWorkersFault.java | 89 + .../msq/indexing/error/UnknownFault.java | 84 + .../msq/indexing/error/WorkerFailedFault.java | 86 + .../indexing/error/WorkerRpcFailedFault.java | 69 + .../msq/indexing/report/MSQResultsReport.java | 84 + .../msq/indexing/report/MSQStagesReport.java | 248 ++ .../msq/indexing/report/MSQStatusReport.java | 135 + .../msq/indexing/report/MSQTaskReport.java | 64 + .../indexing/report/MSQTaskReportPayload.java | 85 + .../apache/druid/msq/input/InputSlice.java | 40 + .../druid/msq/input/InputSliceReader.java | 50 + .../apache/druid/msq/input/InputSlices.java | 65 + .../org/apache/druid/msq/input/InputSpec.java | 30 + .../druid/msq/input/InputSpecSlicer.java | 50 + .../msq/input/InputSpecSlicerFactory.java | 36 + .../apache/druid/msq/input/InputSpecs.java | 50 + .../druid/msq/input/MapInputSliceReader.java | 70 + .../druid/msq/input/MapInputSpecSlicer.java | 73 + .../apache/druid/msq/input/NilInputSlice.java | 60 + .../druid/msq/input/NilInputSliceReader.java | 55 + .../druid/msq/input/NilInputSource.java | 90 + .../apache/druid/msq/input/ReadableInput.java | 112 + .../druid/msq/input/ReadableInputs.java | 92 + .../apache/druid/msq/input/SlicerUtils.java | 129 + .../input/external/ExternalInputSlice.java | 111 + .../external/ExternalInputSliceReader.java | 243 ++ .../msq/input/external/ExternalInputSpec.java | 106 + .../external/ExternalInputSpecSlicer.java | 166 ++ .../stage/CollectedReadablePartitions.java | 126 + .../stage/CombinedReadablePartitions.java | 111 + .../druid/msq/input/stage/InputChannels.java | 42 + .../msq/input/stage/ReadablePartition.java | 118 + .../msq/input/stage/ReadablePartitions.java | 100 + .../msq/input/stage/StageInputSlice.java | 96 + .../input/stage/StageInputSliceReader.java | 96 + .../druid/msq/input/stage/StageInputSpec.java | 75 + .../msq/input/stage/StageInputSpecSlicer.java | 82 + .../stage/StripedReadablePartitions.java | 132 + .../input/table/RichSegmentDescriptor.java | 119 + .../input/table/SegmentWithDescriptor.java | 78 + .../msq/input/table/SegmentsInputSlice.java | 103 + .../input/table/SegmentsInputSliceReader.java | 100 + .../druid/msq/input/table/TableInputSpec.java | 130 + .../msq/input/table/TableInputSpecSlicer.java | 196 ++ .../druid/msq/kernel/ExtraInfoHolder.java | 80 + .../apache/druid/msq/kernel/FrameContext.java | 60 + .../msq/kernel/FrameProcessorFactory.java | 105 + .../druid/msq/kernel/MaxCountShuffleSpec.java | 137 ++ .../druid/msq/kernel/NilExtraInfoHolder.java | 40 + .../msq/kernel/ProcessorsAndChannels.java | 54 + .../druid/msq/kernel/QueryDefinition.java | 169 ++ .../msq/kernel/QueryDefinitionBuilder.java | 87 + .../apache/druid/msq/kernel/ShuffleSpec.java | 72 + .../druid/msq/kernel/StageDefinition.java | 353 +++ .../msq/kernel/StageDefinitionBuilder.java | 126 + .../org/apache/druid/msq/kernel/StageId.java | 115 + .../druid/msq/kernel/StagePartition.java | 86 + .../msq/kernel/TargetSizeShuffleSpec.java | 132 + .../apache/druid/msq/kernel/WorkOrder.java | 143 ++ .../msq/kernel/WorkerAssignmentStrategy.java | 119 + .../controller/ControllerQueryKernel.java | 546 +++++ .../controller/ControllerStagePhase.java | 118 + .../controller/ControllerStageTracker.java | 412 ++++ .../msq/kernel/controller/WorkerInputs.java | 179 ++ .../msq/kernel/worker/WorkerStageKernel.java | 217 ++ .../msq/kernel/worker/WorkerStagePhase.java | 87 + .../querykit/BaseFrameProcessorFactory.java | 72 + .../msq/querykit/BaseLeafFrameProcessor.java | 216 ++ .../BaseLeafFrameProcessorFactory.java | 307 +++ .../msq/querykit/BroadcastJoinHelper.java | 205 ++ .../msq/querykit/DataSegmentProvider.java | 37 + .../msq/querykit/DataSegmentTimelineView.java | 49 + .../druid/msq/querykit/DataSourcePlan.java | 384 +++ .../msq/querykit/InputNumberDataSource.java | 116 + .../msq/querykit/LazyResourceHolder.java | 73 + .../druid/msq/querykit/MultiQueryKit.java | 68 + .../apache/druid/msq/querykit/QueryKit.java | 52 + .../druid/msq/querykit/QueryKitUtils.java | 222 ++ .../msq/querykit/ShuffleSpecFactories.java | 51 + .../msq/querykit/ShuffleSpecFactory.java | 35 + .../common/OffsetLimitFrameProcessor.java | 168 ++ .../OffsetLimitFrameProcessorFactory.java | 162 ++ .../GroupByPostShuffleFrameProcessor.java | 324 +++ ...oupByPostShuffleFrameProcessorFactory.java | 131 + .../GroupByPreShuffleFrameProcessor.java | 245 ++ ...roupByPreShuffleFrameProcessorFactory.java | 80 + .../msq/querykit/groupby/GroupByQueryKit.java | 302 +++ .../scan/ScanQueryFrameProcessor.java | 316 +++ .../scan/ScanQueryFrameProcessorFactory.java | 96 + .../druid/msq/querykit/scan/ScanQueryKit.java | 173 ++ .../msq/rpc/CoordinatorServiceClient.java | 40 + .../msq/rpc/CoordinatorServiceClientImpl.java | 90 + .../DurableStorageInputChannelFactory.java | 122 + .../DurableStorageOutputChannelFactory.java | 142 ++ .../shuffle/WorkerInputChannelFactory.java | 101 + .../org/apache/druid/msq/sql/MSQMode.java | 81 + .../druid/msq/sql/MSQTaskQueryMaker.java | 298 +++ .../druid/msq/sql/MSQTaskSqlEngine.java | 299 +++ .../apache/druid/msq/sql/SqlTaskResource.java | 336 +++ .../apache/druid/msq/sql/SqlTaskStatus.java | 104 + .../ClusterByStatisticsCollector.java | 97 + .../ClusterByStatisticsCollectorImpl.java | 440 ++++ .../ClusterByStatisticsSnapshot.java | 133 + .../statistics/DelegateOrMinKeyCollector.java | 170 ++ .../DelegateOrMinKeyCollectorFactory.java | 122 + .../DelegateOrMinKeyCollectorSnapshot.java | 87 + .../msq/statistics/DistinctKeyCollector.java | 305 +++ .../DistinctKeyCollectorFactory.java | 87 + .../msq/statistics/DistinctKeySnapshot.java | 94 + .../druid/msq/statistics/KeyCollector.java | 77 + .../msq/statistics/KeyCollectorFactory.java | 46 + .../msq/statistics/KeyCollectorSnapshot.java | 27 + ...eyCollectorSnapshotDeserializerModule.java | 34 + .../druid/msq/statistics/KeyCollectors.java | 51 + .../QuantilesSketchKeyCollector.java | 168 ++ .../QuantilesSketchKeyCollectorFactory.java | 149 ++ .../QuantilesSketchKeyCollectorSnapshot.java | 61 + .../statistics/TooManyBucketsException.java | 38 + .../msq/util/DecoratedExecutorService.java | 162 ++ .../druid/msq/util/DimensionSchemaUtils.java | 70 + .../apache/druid/msq/util/IntervalUtils.java | 64 + .../apache/druid/msq/util/MSQFutureUtils.java | 70 + .../msq/util/MultiStageQueryContext.java | 237 ++ .../druid/msq/util/PassthroughAggregator.java | 78 + .../util/PassthroughAggregatorFactory.java | 208 ++ .../apache/druid/msq/util/SequenceUtils.java | 44 + .../druid/msq/util/SupplierIterator.java | 57 + ...rg.apache.druid.initialization.DruidModule | 21 + .../src/main/resources/log4j2.xml | 38 + .../druid/msq/TestArrayStorageAdapter.java | 56 + .../counters/CountersSnapshotTreeTest.java | 49 + .../apache/druid/msq/exec/MSQInsertTest.java | 661 +++++ .../apache/druid/msq/exec/MSQReplaceTest.java | 506 ++++ .../apache/druid/msq/exec/MSQSelectTest.java | 1052 ++++++++ .../apache/druid/msq/exec/MSQTasksTest.java | 247 ++ .../druid/msq/exec/QueryValidatorTest.java | 144 ++ .../msq/exec/WorkerMemoryParametersTest.java | 116 + .../druid/msq/indexing/ColumnMappingTest.java | 36 + .../DataSourceMSQDestinationTest.java | 37 + .../indexing/IndexerWorkerContextTest.java | 87 + .../indexing/MSQPartitionAssignmentTest.java | 57 + .../druid/msq/indexing/MSQSpecTest.java | 36 + .../msq/indexing/MSQTuningConfigTest.java | 64 + .../error/InsertLockPreemptedFaultTest.java | 74 + .../msq/indexing/error/MSQFaultSerdeTest.java | 97 + .../msq/indexing/error/MSQWarningsTest.java | 404 +++ .../indexing/report/MSQTaskReportTest.java | 214 ++ .../druid/msq/input/InputSpecsTest.java | 43 + .../druid/msq/input/NilInputSliceTest.java | 50 + .../external/ExternalInputSliceTest.java | 65 + .../external/ExternalInputSpecSlicerTest.java | 320 +++ .../input/external/ExternalInputSpecTest.java | 63 + .../CollectedReadablePartitionsTest.java | 83 + .../stage/CombinedReadablePartitionsTest.java | 95 + .../input/stage/ReadablePartitionTest.java | 32 + .../msq/input/stage/StageInputSliceTest.java | 54 + .../input/stage/StageInputSpecSlicerTest.java | 120 + .../msq/input/stage/StageInputSpecTest.java | 51 + .../stage/StripedReadablePartitionsTest.java | 91 + .../table/RichSegmentDescriptorTest.java | 88 + .../table/SegmentWithDescriptorTest.java | 32 + .../input/table/SegmentsInputSliceTest.java | 63 + .../input/table/TableInputSpecSlicerTest.java | 493 ++++ .../msq/input/table/TableInputSpecTest.java | 78 + .../druid/msq/kernel/QueryDefinitionTest.java | 76 + .../druid/msq/kernel/StageDefinitionTest.java | 35 + .../apache/druid/msq/kernel/StageIdTest.java | 76 + .../druid/msq/kernel/WorkOrderTest.java | 35 + .../BaseControllerQueryKernelTest.java | 320 +++ .../ControllerQueryKernelTests.java | 383 +++ .../controller/ControllerTestInputSlice.java | 31 + .../controller/ControllerTestInputSpec.java | 26 + .../ControllerTestInputSpecSlicer.java | 57 + .../MockQueryDefinitionBuilder.java | 221 ++ .../kernel/controller/WorkerInputsTest.java | 32 + .../msq/querykit/BroadcastJoinHelperTest.java | 278 +++ .../org/apache/druid/msq/sql/MSQModeTest.java | 57 + .../druid/msq/sql/MSQTaskQueryMakerTest.java | 53 + .../druid/msq/sql/SqlTaskStatusTest.java | 60 + .../ClusterByStatisticsCollectorImplTest.java | 951 ++++++++ .../DelegateOrMinKeyCollectorTest.java | 152 ++ .../statistics/DistinctKeyCollectorTest.java | 243 ++ .../msq/statistics/KeyCollectorTestUtils.java | 296 +++ .../QuantilesSketchKeyCollectorTest.java | 200 ++ .../apache/druid/msq/test/MSQTestBase.java | 1068 ++++++++ .../msq/test/MSQTestControllerClient.java | 94 + .../msq/test/MSQTestControllerContext.java | 261 ++ .../MSQTestDelegateDataSegmentPusher.java | 72 + .../test/MSQTestOverlordServiceClient.java | 119 + .../druid/msq/test/MSQTestSegmentManager.java | 84 + .../msq/test/MSQTestTaskActionClient.java | 98 + .../druid/msq/test/MSQTestWorkerClient.java | 128 + .../druid/msq/test/MSQTestWorkerContext.java | 183 ++ .../druid/msq/util/IntervalUtilsTest.java | 91 + .../msq/util/MultiStageQueryContextTest.java | 191 ++ .../PassthroughAggregatorFactoryTest.java | 77 + .../src/test/resources/unparseable.gz | Bin 0 -> 538 bytes .../wikipedia-sampled-unparsable.json | 3 + .../src/test/resources/wikipedia-sampled.json | 20 + .../indexing/common/config/TaskConfig.java | 3 +- .../indexing/common/task/AbstractTask.java | 2 +- .../druid/indexing/overlord/TaskQueue.java | 4 + pom.xml | 1 + 315 files changed, 42209 insertions(+), 10 deletions(-) create mode 100644 extensions-core/multi-stage-query/README.md create mode 100644 extensions-core/multi-stage-query/pom.xml create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/ChannelCounters.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterNames.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsSerializer.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterTracker.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounter.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounterSnapshot.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SuperSorterProgressTrackerCounter.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/WarningCounters.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryValidator.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQDurableStorageModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQExternalDataSourceModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQServiceClientModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MultiStageQuery.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMapping.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMappings.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingReadableFrameChannel.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQDestination.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQPartitionAssignment.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTaskList.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/TaskReportMSQDestination.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BroadcastTablesTooLargeFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CannotParseExternalDataFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnNameRestrictedFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnTypeNotSupportedFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/FaultsExceededChecker.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotReplaceExistingSegmentFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeNullFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarnings.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/QueryNotSupportedFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/RowTooLargeFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TaskStartTimeoutFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyColumnsFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyInputFilesFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyPartitionsFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWarningsFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkersFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/UnknownFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerFailedFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStagesReport.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReport.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReportPayload.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicer.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicerFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSpecSlicer.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSlice.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInputs.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/SlicerUtils.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSlice.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/InputChannels.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartition.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSlice.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpecSlicer.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ExtraInfoHolder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/MaxCountShuffleSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/NilExtraInfoHolder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinitionBuilder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ShuffleSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinitionBuilder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageId.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StagePartition.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/TargetSizeShuffleSpec.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkerAssignmentStrategy.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/WorkerInputs.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentTimelineView.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/LazyResourceHolder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactories.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClientImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQMode.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollector.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollector.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollector.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollector.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshotDeserializerModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/TooManyBucketsException.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DecoratedExecutorService.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQFutureUtils.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregator.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SequenceUtils.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SupplierIterator.java create mode 100644 extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule create mode 100644 extensions-core/multi-stage-query/src/main/resources/log4j2.xml create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/TestArrayStorageAdapter.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/counters/CountersSnapshotTreeTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/ColumnMappingTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DataSourceMSQDestinationTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQPartitionAssignmentTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFaultTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/NilInputSliceTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSliceTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicerTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CollectedReadablePartitionsTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CombinedReadablePartitionsTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/ReadablePartitionTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSliceTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecSlicerTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StripedReadablePartitionsTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/QueryDefinitionTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageIdTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/WorkOrderTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSlice.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpec.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpecSlicer.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/WorkerInputsTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQModeTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/KeyCollectorTestUtils.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/PassthroughAggregatorFactoryTest.java create mode 100644 extensions-core/multi-stage-query/src/test/resources/unparseable.gz create mode 100644 extensions-core/multi-stage-query/src/test/resources/wikipedia-sampled-unparsable.json create mode 100644 extensions-core/multi-stage-query/src/test/resources/wikipedia-sampled.json diff --git a/codestyle/spotbugs-exclude.xml b/codestyle/spotbugs-exclude.xml index f4efc658902..607d4da423a 100644 --- a/codestyle/spotbugs-exclude.xml +++ b/codestyle/spotbugs-exclude.xml @@ -71,6 +71,13 @@ + + + + + + + diff --git a/core/src/main/java/org/apache/druid/common/utils/IdUtils.java b/core/src/main/java/org/apache/druid/common/utils/IdUtils.java index 1dcac02b2e3..2d3f3301016 100644 --- a/core/src/main/java/org/apache/druid/common/utils/IdUtils.java +++ b/core/src/main/java/org/apache/druid/common/utils/IdUtils.java @@ -41,7 +41,7 @@ public class IdUtils private static final Joiner UNDERSCORE_JOINER = Joiner.on("_"); - public static void validateId(String thingToValidate, String stringToValidate) + public static String validateId(String thingToValidate, String stringToValidate) { Preconditions.checkArgument( !Strings.isNullOrEmpty(stringToValidate), @@ -71,6 +71,8 @@ public class IdUtils throw new IAE("%s cannot contain character #%d (at position %d).", thingToValidate, (int) c, i); } } + + return stringToValidate; } public static String getRandomId() diff --git a/distribution/pom.xml b/distribution/pom.xml index 84d3c218fb8..17e6b5381d0 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -216,6 +216,8 @@ -c org.apache.druid.extensions:druid-lookups-cached-single -c + org.apache.druid.extensions:druid-multi-stage-query + -c org.apache.druid.extensions:druid-protobuf-extensions -c org.apache.druid.extensions:mysql-metadata-storage @@ -396,6 +398,8 @@ -c org.apache.druid.extensions:druid-lookups-cached-single -c + org.apache.druid.extensions:druid-multi-stage-query + -c org.apache.druid.extensions:druid-protobuf-extensions -c org.apache.druid.extensions:mysql-metadata-storage @@ -679,6 +683,8 @@ -c org.apache.druid.extensions:druid-lookups-cached-global -c + org.apache.druid.extensions:druid-multi-stage-query + -c org.apache.druid.extensions:druid-protobuf-extensions -c org.apache.druid.extensions:mysql-metadata-storage diff --git a/examples/conf/druid/cluster/_common/common.runtime.properties b/examples/conf/druid/cluster/_common/common.runtime.properties index edcef960297..eafa11742d5 100644 --- a/examples/conf/druid/cluster/_common/common.runtime.properties +++ b/examples/conf/druid/cluster/_common/common.runtime.properties @@ -24,7 +24,7 @@ # If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. # If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. # More info: https://druid.apache.org/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"] +druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties b/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties index 8928cc9f8ed..00071a83d7d 100644 --- a/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties +++ b/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties @@ -22,6 +22,7 @@ druid.plaintextPort=8081 druid.coordinator.startDelay=PT10S druid.coordinator.period=PT5S +druid.manager.segments.pollDuration=PT5S # Run the overlord service in the coordinator process druid.coordinator.asOverlord.enabled=true diff --git a/examples/conf/druid/single-server/large/_common/common.runtime.properties b/examples/conf/druid/single-server/large/_common/common.runtime.properties index edcef960297..eafa11742d5 100644 --- a/examples/conf/druid/single-server/large/_common/common.runtime.properties +++ b/examples/conf/druid/single-server/large/_common/common.runtime.properties @@ -24,7 +24,7 @@ # If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. # If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. # More info: https://druid.apache.org/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"] +druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties index 8928cc9f8ed..00071a83d7d 100644 --- a/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties +++ b/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties @@ -22,6 +22,7 @@ druid.plaintextPort=8081 druid.coordinator.startDelay=PT10S druid.coordinator.period=PT5S +druid.manager.segments.pollDuration=PT5S # Run the overlord service in the coordinator process druid.coordinator.asOverlord.enabled=true diff --git a/examples/conf/druid/single-server/medium/_common/common.runtime.properties b/examples/conf/druid/single-server/medium/_common/common.runtime.properties index edcef960297..eafa11742d5 100644 --- a/examples/conf/druid/single-server/medium/_common/common.runtime.properties +++ b/examples/conf/druid/single-server/medium/_common/common.runtime.properties @@ -24,7 +24,7 @@ # If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. # If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. # More info: https://druid.apache.org/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"] +druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties index 8928cc9f8ed..00071a83d7d 100644 --- a/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties +++ b/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties @@ -22,6 +22,7 @@ druid.plaintextPort=8081 druid.coordinator.startDelay=PT10S druid.coordinator.period=PT5S +druid.manager.segments.pollDuration=PT5S # Run the overlord service in the coordinator process druid.coordinator.asOverlord.enabled=true diff --git a/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties b/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties index edcef960297..eafa11742d5 100644 --- a/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties +++ b/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties @@ -24,7 +24,7 @@ # If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. # If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. # More info: https://druid.apache.org/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"] +druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties index 8928cc9f8ed..00071a83d7d 100644 --- a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties +++ b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties @@ -22,6 +22,7 @@ druid.plaintextPort=8081 druid.coordinator.startDelay=PT10S druid.coordinator.period=PT5S +druid.manager.segments.pollDuration=PT5S # Run the overlord service in the coordinator process druid.coordinator.asOverlord.enabled=true diff --git a/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties b/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties index edcef960297..eafa11742d5 100644 --- a/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties +++ b/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties @@ -24,7 +24,7 @@ # If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. # If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. # More info: https://druid.apache.org/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"] +druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties index 8928cc9f8ed..00071a83d7d 100644 --- a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties +++ b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties @@ -22,6 +22,7 @@ druid.plaintextPort=8081 druid.coordinator.startDelay=PT10S druid.coordinator.period=PT5S +druid.manager.segments.pollDuration=PT5S # Run the overlord service in the coordinator process druid.coordinator.asOverlord.enabled=true diff --git a/examples/conf/druid/single-server/small/_common/common.runtime.properties b/examples/conf/druid/single-server/small/_common/common.runtime.properties index edcef960297..eafa11742d5 100644 --- a/examples/conf/druid/single-server/small/_common/common.runtime.properties +++ b/examples/conf/druid/single-server/small/_common/common.runtime.properties @@ -24,7 +24,7 @@ # If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. # If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. # More info: https://druid.apache.org/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"] +druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties index 8928cc9f8ed..00071a83d7d 100644 --- a/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties +++ b/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties @@ -22,6 +22,7 @@ druid.plaintextPort=8081 druid.coordinator.startDelay=PT10S druid.coordinator.period=PT5S +druid.manager.segments.pollDuration=PT5S # Run the overlord service in the coordinator process druid.coordinator.asOverlord.enabled=true diff --git a/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties b/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties index edcef960297..eafa11742d5 100644 --- a/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties +++ b/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties @@ -24,7 +24,7 @@ # If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. # If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. # More info: https://druid.apache.org/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"] +druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties index 8928cc9f8ed..00071a83d7d 100644 --- a/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties +++ b/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties @@ -22,6 +22,7 @@ druid.plaintextPort=8081 druid.coordinator.startDelay=PT10S druid.coordinator.period=PT5S +druid.manager.segments.pollDuration=PT5S # Run the overlord service in the coordinator process druid.coordinator.asOverlord.enabled=true diff --git a/extensions-core/multi-stage-query/README.md b/extensions-core/multi-stage-query/README.md new file mode 100644 index 00000000000..491be586d8b --- /dev/null +++ b/extensions-core/multi-stage-query/README.md @@ -0,0 +1,141 @@ + + +# `druid-multi-stage-query` developer notes + +This document provides developer notes for the major packages of the `druid-multi-stage-query` extension. It does not +discuss future plans; these are discussed on the list or in GitHub issues. + +## Model + +Multi-stage queries are modeled as a directed acyclic graph (DAG) of stages. Each stage has some inputs (or, possibly, +zero inputs, if the stage generates data out of thin air). Those inputs can be Druid tables, external data, or +the outputs of other stages of the same query. There is one final stage that produces the query result. Stage outputs, +whether they are inputs to other stages or results of the query itself, are optionally shuffled. + +SQL-based ingestion jobs run as multi-stage query tasks. In this case, the result of the query is inserted into the +target table. + +Package `org.apache.druid.msq.kernel` and `org.apache.druid.msq.input` contain the model classes for multi-stage +queries. + +Main classes: + +- [QueryDefinition](src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java) represents a multi-stage query. +- [StageDefinition](src/main/java/org/apache/druid/msq/kernel/StageDefinition.java) represents an individual stage of + a multi-stage query. +- [InputSpec](src/main/java/org/apache/druid/msq/input/InputSpec.java) represents an input to a stage. Links between + stages are represented by [StageInputSpec](src/main/java/org/apache/druid/msq/input/stage/StageInputSpec.java). +- [ShuffleSpec](src/main/java/org/apache/druid/msq/input/ShuffleSpec.java) represents the shuffle that happens as part + of stage output. + +## Indexing service + +Package `org.apache.druid.msq.indexing` contains code related to integrating with the indexing service. This allows +multi-stage queries to run as indexing service tasks. + +Main classes: + +- [MSQControllerTask](src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java) is a `query_controller` task. + Each query has one controller task. The controller task launches worker tasks to do the actual work. +- [MSQWorkerTask](src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java) is a `query_worker` task. These stick + around for the lifetime of the query. Each task may do work for multiple stages. It has a specific worker number + that is retained across all stages that the task may work on. + +## Planning + +Multi-stage queries, when run as SQL via query tasks, are planned in three phases: + +1. The SQL planner generates a native query corresponding to the user's SQL query. +2. The `query_controller` task generates a multi-stage QueryDefinition corresponding to the native query, using + QueryKit. +3. The `query_controller` task determines how many workers will run and generates WorkOrders for each worker. + +Once all three of these phases are complete, `query_worker` tasks are launched, sent their WorkOrders, and query +execution begins. + +Packages `org.apache.druid.msq.querykit`, `org.apache.druid.msq.input`, and `org.apache.druid.msq.kernel` contain code +related to query planning. + +Main classes: + +- [QueryKit](src/main/java/org/apache/druid/msq/querykit/QueryKit.java) implementations produce QueryDefinition + instances from native Druid queries. +- [InputSlice](src/main/java/org/apache/druid/msq/input/InputSlice.java) represents a slice of stage input assigned to + a particular worker. +- [WorkerAssignmentStrategy](src/main/java/org/apache/druid/msq/kernel/WorkerAssignmentStrategy.java) drives the splitting + of input specs into input slices, and is therefore responsible for assigning work to workers. +- [WorkOrder](src/main/java/org/apache/druid/msq/kernel/WorkOrder.java) represents the work assigned to a particular + worker in a particular stage. + +## Execution + +Package `org.apache.druid.msq.exec` and `org.apache.druid.msq.kernel` contain code related to driving query execution. + +Main classes: + +- [ControllerQueryKernel](src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java) is the state + machine that drives execution on the controller. +- [WorkerStageKernel](src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java) is the state machine + that drives execution on workers. +- [ControllerImpl](src/main/java/org/apache/druid/msq/kernel/exec/ControllerImpl.java) embeds a ControllerQueryKernel + and handles controller-side execution beyond the state machine, including query planning, RPC, counters, and so on. +- [WorkerImpl](src/main/java/org/apache/druid/msq/kernel/exec/WorkerImpl.java) embeds a WorkerStageKernel and handles + worker-side execution beyond the state machine, including setup of processors, channels, counters, and so on. + +## Statistics + +Package `org.apache.druid.msq.statistics` contains code related to determining partition boundaries as part of +doing a range-based shuffle. During a stage that intends to do range-based shuffle, workers gather statistics +using a ClusterByStatisticsCollector, which are then merged on the controller and used to generate partition +boundaries. + +- [ClusterByStatisticsCollector](src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollector.java) + is the interface to statistics collection. +- [ClusterByStatisticsCollectorImpl](src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java) + is the main implementation of the statistics collection interface. + +## Counters + +Package `org.apache.druid.msq.counters` contains code related to tracking and reporting query execution metrics. + +Main classes: + +- [CounterTracker](src/main/java/org/apache/druid/msq/counters/CounterTracker.java) is used by workers to keep track of + named counters of various types. +- [CounterSnapshots](src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java) are periodically reported from + workers to the controller. +- [CounterSnapshotsTree](src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java) is used by the + controller to store worker snapshots. It is also included in task reports, which enables live metrics, and also + allows query counters to be reviewed after the query has been completed. + +## SQL + +Package `org.apache.druid.msq.sql` contains code related to integration with Druid SQL APIs. + +Main classes: + +- [SqlTaskResource](src/main/java/org/apache/druid/msq/counters/CounterTracker.java) offers the endpoint + `/druid/v2/sql/task`, where SQL queries are executed as multi-stage query tasks. +- [MSQTaskSqlEngine](src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java) is a SqlEngine implementation that + executes SQL queries as multi-stage query tasks. It is injected into the SqlTaskResource. + +## References + +- Multi-stage distributed query proposal: https://github.com/apache/druid/issues/12262 diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml new file mode 100644 index 00000000000..a8a2513bab5 --- /dev/null +++ b/extensions-core/multi-stage-query/pom.xml @@ -0,0 +1,299 @@ + + + + + + 4.0.0 + + org.apache.druid.extensions + druid-multi-stage-query + druid-multi-stage-query + druid-multi-stage-query + + + org.apache.druid + druid + 0.24.0-SNAPSHOT + ../../pom.xml + + + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + provided + + + org.apache.druid + druid-sql + ${project.parent.version} + provided + + + org.apache.druid + druid-services + ${project.parent.version} + provided + + + com.google.inject + guice + provided + + + com.google.inject.extensions + guice-multibindings + provided + + + com.google.guava + guava + provided + + + com.opencsv + opencsv + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + javax.validation + validation-api + provided + + + com.google.code.findbugs + jsr305 + provided + + + jakarta.inject + jakarta.inject-api + provided + + + joda-time + joda-time + provided + + + io.netty + netty + provided + + + org.apache.calcite + calcite-core + provided + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + provided + + + org.jdbi + jdbi + provided + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + provided + + + javax.ws.rs + jsr311-api + provided + + + org.apache.commons + commons-lang3 + provided + + + javax.servlet + javax.servlet-api + provided + + + com.sun.jersey + jersey-server + provided + + + com.google.errorprone + error_prone_annotations + provided + + + org.lz4 + lz4-java + provided + + + org.apache.datasketches + datasketches-java + provided + + + org.apache.datasketches + datasketches-memory + provided + + + it.unimi.dsi + fastutil-core + provided + + + commons-lang + commons-lang + provided + + + commons-io + commons-io + provided + + + + + org.easymock + easymock + test + + + org.hamcrest + hamcrest-all + test + + + org.hamcrest + hamcrest-core + test + + + junit + junit + test + + + org.mockito + mockito-core + test + + + nl.jqno.equalsverifier + equalsverifier + test + + + org.apache.druid + druid-core + ${project.parent.version} + test + test-jar + + + org.apache.druid + druid-processing + ${project.parent.version} + test + test-jar + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-sql + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-hll + ${project.parent.version} + + provided + + + + + + + org.jacoco + jacoco-maven-plugin + + + + org/apache/druid/msq/guice/* + org/apache/druid/msq/indexing/IndexerControllerClient* + org/apache/druid/msq/indexing/IndexerControllerContext* + org/apache/druid/msq/indexing/IndexerWorkerClient* + org/apache/druid/msq/indexing/IndexerWorkerContext* + org/apache/druid/msq/indexing/IndexerWorkerManagerClient* + + + + + + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/ChannelCounters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/ChannelCounters.java new file mode 100644 index 00000000000..461b066ce9d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/ChannelCounters.java @@ -0,0 +1,274 @@ +/* + * 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.druid.msq.counters; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import it.unimi.dsi.fastutil.longs.LongArrayList; +import it.unimi.dsi.fastutil.longs.LongList; +import org.apache.druid.frame.Frame; + +import javax.annotation.Nullable; +import java.util.Arrays; + +/** + * Counters for inputs and outputs. Created by {@link CounterTracker#channel}. + */ +public class ChannelCounters implements QueryCounter +{ + private static final int NO_PARTITION = 0; + + @GuardedBy("this") + private final LongList rows = new LongArrayList(); + + @GuardedBy("this") + private final LongList bytes = new LongArrayList(); + + @GuardedBy("this") + private final LongList frames = new LongArrayList(); + + @GuardedBy("this") + private final LongList files = new LongArrayList(); + + @GuardedBy("this") + private final LongList totalFiles = new LongArrayList(); + + public void incrementRowCount() + { + add(NO_PARTITION, 1, 0, 0, 0); + } + + public void incrementFileCount() + { + add(NO_PARTITION, 0, 0, 0, 1); + } + + public void addFile(final long nRows, final long nBytes) + { + add(NO_PARTITION, nRows, nBytes, 0, 1); + } + + public void addFrame(final int partitionNumber, final Frame frame) + { + add(partitionNumber, frame.numRows(), frame.numBytes(), 1, 0); + } + + public ChannelCounters setTotalFiles(final long nFiles) + { + synchronized (this) { + ensureCapacityForPartition(NO_PARTITION); + totalFiles.set(NO_PARTITION, nFiles); + return this; + } + } + + private void add( + final int partitionNumber, + final long nRows, + final long nBytes, + final long nFrames, + final long nFiles + ) + { + synchronized (this) { + ensureCapacityForPartition(partitionNumber); + rows.set(partitionNumber, rows.getLong(partitionNumber) + nRows); + bytes.set(partitionNumber, bytes.getLong(partitionNumber) + nBytes); + frames.set(partitionNumber, frames.getLong(partitionNumber) + nFrames); + files.set(partitionNumber, files.getLong(partitionNumber) + nFiles); + } + } + + @GuardedBy("this") + private void ensureCapacityForPartition(final int partitionNumber) + { + while (partitionNumber >= rows.size()) { + rows.add(0); + } + + while (partitionNumber >= bytes.size()) { + bytes.add(0); + } + + while (partitionNumber >= frames.size()) { + frames.add(0); + } + + while (partitionNumber >= files.size()) { + files.add(0); + } + + while (partitionNumber >= totalFiles.size()) { + totalFiles.add(0); + } + } + + @Override + @Nullable + public Snapshot snapshot() + { + final long[] rowsArray; + final long[] bytesArray; + final long[] framesArray; + final long[] filesArray; + final long[] totalFilesArray; + + synchronized (this) { + rowsArray = listToArray(rows); + bytesArray = listToArray(bytes); + framesArray = listToArray(frames); + filesArray = listToArray(files); + totalFilesArray = listToArray(totalFiles); + } + + if (rowsArray == null + && bytesArray == null + && framesArray == null + && filesArray == null + && totalFilesArray == null) { + return null; + } else { + return new Snapshot(rowsArray, bytesArray, framesArray, filesArray, totalFilesArray); + } + } + + @Nullable + private static long[] listToArray(final LongList longList) + { + boolean allZero = true; + + for (int i = 0; i < longList.size(); i++) { + if (longList.getLong(i) != 0) { + allZero = false; + break; + } + } + + if (allZero) { + return null; + } else { + return longList.toArray(new long[0]); + } + } + + @JsonTypeName("channel") + public static class Snapshot implements QueryCounterSnapshot + { + private final long[] rows; + private final long[] bytes; + private final long[] frames; + private final long[] files; + private final long[] totalFiles; + + @JsonCreator + public Snapshot( + @Nullable @JsonProperty("rows") final long[] rows, + @Nullable @JsonProperty("bytes") final long[] bytes, + @Nullable @JsonProperty("frames") final long[] frames, + @Nullable @JsonProperty("files") final long[] files, + @Nullable @JsonProperty("totalFiles") final long[] totalFiles + ) + { + this.rows = rows; + this.bytes = bytes; + this.frames = frames; + this.files = files; + this.totalFiles = totalFiles; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public long[] getRows() + { + return rows; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public long[] getBytes() + { + return bytes; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public long[] getFrames() + { + return frames; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public long[] getFiles() + { + return files; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public long[] getTotalFiles() + { + return totalFiles; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Snapshot snapshot = (Snapshot) o; + return Arrays.equals(rows, snapshot.rows) + && Arrays.equals(bytes, snapshot.bytes) + && Arrays.equals(frames, snapshot.frames) + && Arrays.equals(files, snapshot.files) + && Arrays.equals(totalFiles, snapshot.totalFiles); + } + + @Override + public int hashCode() + { + int result = Arrays.hashCode(rows); + result = 31 * result + Arrays.hashCode(bytes); + result = 31 * result + Arrays.hashCode(frames); + result = 31 * result + Arrays.hashCode(files); + result = 31 * result + Arrays.hashCode(totalFiles); + return result; + } + + @Override + public String toString() + { + return "ChannelCounters.Snapshot{" + + "rows=" + Arrays.toString(rows) + + ", bytes=" + Arrays.toString(bytes) + + ", frames=" + Arrays.toString(frames) + + ", files=" + Arrays.toString(files) + + ", totalFiles=" + Arrays.toString(totalFiles) + + '}'; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterNames.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterNames.java new file mode 100644 index 00000000000..3c1d58ec59b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterNames.java @@ -0,0 +1,142 @@ +/* + * 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.druid.msq.counters; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.ordering.StringComparators; + +import java.util.Comparator; +import java.util.Map; + +/** + * Standard names for counters. + */ +public class CounterNames +{ + private static final String INPUT = "input"; + private static final String OUTPUT = "output"; + private static final String SHUFFLE = "shuffle"; + private static final String SORT_PROGRESS = "sortProgress"; + private static final String WARNINGS = "warnings"; + private static final Comparator COMPARATOR = new NameComparator(); + + private CounterNames() + { + // No construction: statics only. + } + + /** + * Standard name for an input channel counter created by {@link CounterTracker#channel}. + */ + public static String inputChannel(final int inputNumber) + { + return StringUtils.format("%s%d", INPUT, inputNumber); + } + + /** + * Standard name for an output channel counter created by {@link CounterTracker#channel}. + */ + public static String outputChannel() + { + return OUTPUT; + } + + /** + * Standard name for a shuffle channel counter created by {@link CounterTracker#channel}. + */ + public static String shuffleChannel() + { + return SHUFFLE; + } + + /** + * Standard name for a sort progress counter created by {@link CounterTracker#sortProgress()}. + */ + public static String sortProgress() + { + return SORT_PROGRESS; + } + + /** + * Standard name for a warnings counter created by {@link CounterTracker#warnings()}. + */ + public static String warnings() + { + return WARNINGS; + } + + /** + * Standard comparator for counter names. Not necessary for functionality, but helps with human-readability. + */ + public static Comparator comparator() + { + return COMPARATOR; + } + + /** + * Comparator that ensures counters are sorted in a nice order when serialized to JSON. Not necessary for + * functionality, but helps with human-readability. + */ + private static class NameComparator implements Comparator + { + private static final Map ORDER = + ImmutableMap.builder() + .put(OUTPUT, 0) + .put(SHUFFLE, 1) + .put(SORT_PROGRESS, 2) + .put(WARNINGS, 3) + .build(); + + @Override + public int compare(final String name1, final String name2) + { + final boolean isInput1 = name1.startsWith(INPUT); + final boolean isInput2 = name2.startsWith(INPUT); + + if (isInput1 && isInput2) { + // Compare INPUT alphanumerically, so e.g. "input2" is before "input10" + return StringComparators.ALPHANUMERIC.compare(name1, name2); + } else if (isInput1 != isInput2) { + // INPUT goes first + return isInput1 ? -1 : 1; + } + + assert !isInput1 && !isInput2; + + final Integer order1 = ORDER.get(name1); + final Integer order2 = ORDER.get(name2); + + if (order1 != null && order2 != null) { + // Respect ordering from ORDER + return Integer.compare(order1, order2); + } else if (order1 != null) { + // Names from ORDER go before names that are not in ORDER + return -1; + } else if (order2 != null) { + // Names from ORDER go before names that are not in ORDER + return 1; + } else { + assert order1 == null && order2 == null; + return name1.compareTo(name2); + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java new file mode 100644 index 00000000000..d75eb1ce8cb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java @@ -0,0 +1,75 @@ +/* + * 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.druid.msq.counters; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.collect.ImmutableSortedMap; + +import java.util.Map; +import java.util.Objects; + +/** + * Named counter snapshots. Immutable. Often part of a {@link CounterSnapshotsTree}. + */ +public class CounterSnapshots +{ + private final Map snapshotMap; + + @JsonCreator + public CounterSnapshots(final Map snapshotMap) + { + this.snapshotMap = ImmutableSortedMap.copyOf(snapshotMap, CounterNames.comparator()); + } + + public Map getMap() + { + return snapshotMap; + } + + public boolean isEmpty() + { + return snapshotMap.isEmpty(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CounterSnapshots that = (CounterSnapshots) o; + return Objects.equals(snapshotMap, that.snapshotMap); + } + + @Override + public int hashCode() + { + return Objects.hash(snapshotMap); + } + + @Override + public String toString() + { + return snapshotMap.toString(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsSerializer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsSerializer.java new file mode 100644 index 00000000000..7cbb6bc6c42 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsSerializer.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.druid.msq.counters; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; +import java.util.Map; + +/** + * Serializer for {@link CounterSnapshots}. Necessary because otherwise the map structure causes Jackson + * to miss including type codes. + */ +public class CounterSnapshotsSerializer extends StdSerializer +{ + public CounterSnapshotsSerializer() + { + super(CounterSnapshots.class); + } + + @Override + public void serialize( + final CounterSnapshots value, + final JsonGenerator jg, + final SerializerProvider serializers + ) throws IOException + { + jg.writeStartObject(); + + for (final Map.Entry entry : value.getMap().entrySet()) { + jg.writeObjectField(entry.getKey(), entry.getValue()); + } + + jg.writeEndObject(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java new file mode 100644 index 00000000000..953f8b718de --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java @@ -0,0 +1,104 @@ +/* + * 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.druid.msq.counters; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; + +import java.util.Map; + +/** + * Tree of {@link CounterSnapshots} (named counter snapshots) organized by stage and worker. + * + * These are used for worker-to-controller counters propagation with + * {@link ControllerClient#postCounters} and reporting to end users with + * {@link MSQTaskReportPayload#getCounters}). + * + * The map is mutable, but thread-safe. The individual snapshot objects are immutable. + */ +public class CounterSnapshotsTree +{ + // stage -> worker -> counters + @GuardedBy("snapshotsMap") + private final Int2ObjectMap> snapshotsMap; + + public CounterSnapshotsTree() + { + this.snapshotsMap = new Int2ObjectAVLTreeMap<>(); + } + + @JsonCreator + public static CounterSnapshotsTree fromMap(final Map> map) + { + final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); + retVal.putAll(map); + return retVal; + } + + public void put(final int stageNumber, final int workerNumber, final CounterSnapshots snapshots) + { + synchronized (snapshotsMap) { + snapshotsMap.computeIfAbsent(stageNumber, ignored -> new Int2ObjectAVLTreeMap<>()) + .put(workerNumber, snapshots); + } + } + + public void putAll(final CounterSnapshotsTree other) + { + putAll(other.copyMap()); + } + + public boolean isEmpty() + { + synchronized (snapshotsMap) { + return snapshotsMap.isEmpty(); + } + } + + @JsonValue + public Map> copyMap() + { + final Map> retVal = new Int2ObjectAVLTreeMap<>(); + + synchronized (snapshotsMap) { + for (Int2ObjectMap.Entry> entry : snapshotsMap.int2ObjectEntrySet()) { + retVal.put(entry.getIntKey(), new Int2ObjectAVLTreeMap<>(entry.getValue())); + } + } + + return retVal; + } + + private void putAll(final Map> otherMap) + { + synchronized (snapshotsMap) { + for (Map.Entry> stageEntry : otherMap.entrySet()) { + for (Map.Entry workerEntry : stageEntry.getValue().entrySet()) { + put(stageEntry.getKey(), workerEntry.getKey(), workerEntry.getValue()); + } + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterTracker.java new file mode 100644 index 00000000000..36af372a56c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterTracker.java @@ -0,0 +1,74 @@ +/* + * 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.druid.msq.counters; + +import org.apache.druid.frame.processor.SuperSorterProgressTracker; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + +/** + * Class that tracks query counters for a particular worker in a particular stage. + * + * Counters are all tracked on a (stage, worker, counter) basis by the {@link #countersMap} object. + * + * Immutable {@link CounterSnapshots} snapshots can be created by {@link #snapshot()}. + */ +public class CounterTracker +{ + private final ConcurrentHashMap countersMap = new ConcurrentHashMap<>(); + + public ChannelCounters channel(final String name) + { + return counter(name, ChannelCounters::new); + } + + public SuperSorterProgressTracker sortProgress() + { + return counter(CounterNames.sortProgress(), SuperSorterProgressTrackerCounter::new).tracker(); + } + + public WarningCounters warnings() + { + return counter(CounterNames.warnings(), WarningCounters::new); + } + + @SuppressWarnings("unchecked") + public T counter(final String counterName, final Supplier newCounterFn) + { + return (T) countersMap.computeIfAbsent(counterName, ignored -> newCounterFn.get()); + } + + public CounterSnapshots snapshot() + { + final Map m = new HashMap<>(); + + for (final Map.Entry entry : countersMap.entrySet()) { + final QueryCounterSnapshot counterSnapshot = entry.getValue().snapshot(); + if (counterSnapshot != null) { + m.put(entry.getKey(), counterSnapshot); + } + } + + return new CounterSnapshots(m); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounter.java new file mode 100644 index 00000000000..c23022c30f4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounter.java @@ -0,0 +1,32 @@ +/* + * 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.druid.msq.counters; + +import javax.annotation.Nullable; + +/** + * A kind of query counter. Counters are flexible in what they can track and how they work, so this interface + * does not specify anything beyond requiring the ability to take a snapshot. + */ +public interface QueryCounter +{ + @Nullable + QueryCounterSnapshot snapshot(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounterSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounterSnapshot.java new file mode 100644 index 00000000000..c065f7f8252 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounterSnapshot.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.druid.msq.counters; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * Marker interface for the results of {@link QueryCounter#snapshot()}. No methods, because the only purpose of these + * snapshots is to pass things along from worker -> controller -> report. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface QueryCounterSnapshot +{ +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SuperSorterProgressTrackerCounter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SuperSorterProgressTrackerCounter.java new file mode 100644 index 00000000000..b10a2b4ddbc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SuperSorterProgressTrackerCounter.java @@ -0,0 +1,71 @@ +/* + * 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.druid.msq.counters; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.druid.frame.processor.SuperSorterProgressSnapshot; +import org.apache.druid.frame.processor.SuperSorterProgressTracker; + +import javax.annotation.Nullable; + +public class SuperSorterProgressTrackerCounter implements QueryCounter +{ + private final SuperSorterProgressTracker tracker; + + public SuperSorterProgressTrackerCounter() + { + this.tracker = new SuperSorterProgressTracker(); + } + + public SuperSorterProgressTracker tracker() + { + return tracker; + } + + @Nullable + @Override + public QueryCounterSnapshot snapshot() + { + return new Snapshot(tracker.snapshot()); + } + + /** + * Wrapper class that exists for JSON serde. + */ + @JsonTypeName("sortProgress") + public static class Snapshot implements QueryCounterSnapshot + { + private final SuperSorterProgressSnapshot snapshot; + + @JsonCreator + public Snapshot(SuperSorterProgressSnapshot snapshot) + { + this.snapshot = snapshot; + } + + @JsonValue + public SuperSorterProgressSnapshot getSnapshot() + { + return snapshot; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/WarningCounters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/WarningCounters.java new file mode 100644 index 00000000000..7c56dfe02c6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/WarningCounters.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.druid.msq.counters; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Counters for warnings. Created by {@link CounterTracker#warnings()}. + */ +@JsonTypeName("warnings") +public class WarningCounters implements QueryCounter +{ + private final ConcurrentHashMap warningCodeCounter = new ConcurrentHashMap<>(); + + public void incrementWarningCount(String errorCode) + { + warningCodeCounter.compute(errorCode, (ignored, oldCount) -> oldCount == null ? 1 : oldCount + 1); + } + + @Override + @Nullable + public Snapshot snapshot() + { + if (warningCodeCounter.isEmpty()) { + return null; + } + + final Map countCopy = ImmutableMap.copyOf(warningCodeCounter); + return new Snapshot(countCopy); + } + + @JsonTypeName("warnings") + public static class Snapshot implements QueryCounterSnapshot + { + private final Map warningCountMap; + + @JsonCreator + public Snapshot(Map warningCountMap) + { + this.warningCountMap = Preconditions.checkNotNull(warningCountMap, "warningCountMap"); + } + + @JsonValue + public Map getWarningCountMap() + { + return warningCountMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Snapshot snapshot = (Snapshot) o; + return Objects.equals(warningCountMap, snapshot.warningCountMap); + } + + @Override + public int hashCode() + { + return Objects.hash(warningCountMap); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java new file mode 100644 index 00000000000..07730de45e7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java @@ -0,0 +1,125 @@ +/* + * 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.druid.msq.exec; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.msq.counters.CounterSnapshots; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + +/** + * Interface for the controller of a multi-stage query. + */ +public interface Controller +{ + /** + * POJO for capturing the status of a controller task that is currently running. + */ + class RunningControllerStatus + { + private final String id; + + @JsonCreator + public RunningControllerStatus(String id) + { + this.id = id; + } + + @JsonProperty("id") + public String getId() + { + return id; + } + } + + /** + * Unique task/query ID for the batch query run by this controller. + */ + String id(); + + /** + * The task which this controller runs. + */ + MSQControllerTask task(); + + /** + * Runs the controller logic in the current thread. Surrounding classes provide the execution thread. + */ + TaskStatus run() throws Exception; + + /** + * Terminate the query DAG upon a cancellation request. + */ + void stopGracefully(); + + // Worker-to-controller messages + + /** + * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages. + */ + void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject); + + /** + * System error reported by a subtask. Note that the errors are organized by + * taskId, not by query/stage/worker, because system errors are associated + * with a task rather than a specific query/stage/worker execution context. + */ + void workerError(MSQErrorReport errorReport); + + /** + * System warning reported by a subtask. Indicates that the worker has encountered a non-lethal error. Worker should + * continue its execution in such a case. If the worker wants to report an error and stop its execution, + * please use {@link Controller#workerError} + */ + void workerWarning(List errorReports); + + /** + * Periodic update of {@link CounterSnapshots} from subtasks. + */ + void updateCounters(CounterSnapshotsTree snapshotsTree); + + /** + * Reports that results are ready for a subtask. + */ + void resultsComplete( + String queryId, + int stageNumber, + int workerNumber, + Object resultObject + ); + + /** + * Returns the current list of task ids, ordered by worker number. The Nth task has worker number N. + */ + List getTaskIds(); + + @Nullable + Map liveReports(); + +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java new file mode 100644 index 00000000000..f621133586c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java @@ -0,0 +1,81 @@ +/* + * 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.druid.msq.exec; + +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; + +/** + * Client for the multi-stage query controller. Used by a Worker task. + */ +public interface ControllerClient extends AutoCloseable +{ + /** + * Client side method to update the controller with key statistics for a particular stage and worker. + * Controller's implementation collates all the key statistics for a stage to generate the partition boundaries. + */ + void postKeyStatistics( + StageId stageId, + int workerNumber, + ClusterByStatisticsSnapshot keyStatistics + ) throws IOException; + + /** + * Client-side method to update the controller with counters for a particular stage and worker. The controller uses + * this to compile live reports, track warnings generated etc. + */ + void postCounters(CounterSnapshotsTree snapshotsTree) throws IOException; + + /** + * Client side method to update the controller with the result object for a particular stage and worker. This also + * informs the controller that the computation for that particular stage has been done by the worker. + */ + void postResultsComplete( + StageId stageId, + int workerNumber, + @Nullable Object resultObject + ) throws IOException; + + /** + * Client side method to inform the controller that the error has occured in the given worker. + */ + void postWorkerError( + String workerId, + MSQErrorReport errorWrapper + ) throws IOException; + + /** + * Client side method to inform the controller about the warnings generated by the given worker. + */ + void postWorkerWarning( + String workerId, + List MSQErrorReports + ) throws IOException; + List getTaskList() throws IOException; + + @Override + void close(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java new file mode 100644 index 00000000000..f77290c0921 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java @@ -0,0 +1,82 @@ +/* + * 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.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Injector; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.server.DruidNode; + +import java.util.Map; + +/** + * Context used by multi-stage query controllers. + * + * Useful because it allows test fixtures to provide their own implementations. + */ +public interface ControllerContext +{ + ObjectMapper jsonMapper(); + + /** + * Provides a way for tasks to request injectable objects. Useful because tasks are not able to request injection + * at the time of server startup, because the server doesn't know what tasks it will be running. + */ + Injector injector(); + + /** + * Fetch node info about self. + */ + DruidNode selfNode(); + + /** + * Provide access to the Coordinator service. + */ + CoordinatorClient coordinatorClient(); + + /** + * Provide access to segment actions in the Overlord. + */ + TaskActionClient taskActionClient(); + + /** + * Provides services about workers: starting, canceling, obtaining status. + */ + WorkerManagerClient workerManager(); + + /** + * Callback from the controller implementation to "register" the controller. Used in the indexing task implementation + * to set up the task chat web service. + */ + void registerController(Controller controller, Closer closer); + + /** + * Client for communicating with workers. + */ + WorkerClient taskClientFor(Controller controller); + + /** + * Writes controller task report. + */ + void writeReports(String controllerTaskId, Map reports); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java new file mode 100644 index 00000000000..3aa346c3846 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -0,0 +1,2171 @@ +/* + * 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.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.data.input.StringTuple; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.FrameChannelSequence; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.key.RowKeyReader; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.frame.processor.FrameProcessorExecutor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.MarkSegmentsAsUnusedAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.counters.CounterSnapshots; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.indexing.ColumnMapping; +import org.apache.druid.msq.indexing.ColumnMappings; +import org.apache.druid.msq.indexing.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.indexing.InputChannelsImpl; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; +import org.apache.druid.msq.indexing.SegmentGeneratorFrameProcessorFactory; +import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.error.CanceledFault; +import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; +import org.apache.druid.msq.indexing.error.FaultsExceededChecker; +import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault; +import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; +import org.apache.druid.msq.indexing.error.InsertCannotOrderByDescendingFault; +import org.apache.druid.msq.indexing.error.InsertCannotReplaceExistingSegmentFault; +import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault; +import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; +import org.apache.druid.msq.indexing.error.MSQWarnings; +import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; +import org.apache.druid.msq.indexing.error.TooManyWarningsFault; +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.report.MSQResultsReport; +import org.apache.druid.msq.indexing.report.MSQStagesReport; +import org.apache.druid.msq.indexing.report.MSQStatusReport; +import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.input.InputSpecSlicerFactory; +import org.apache.druid.msq.input.InputSpecs; +import org.apache.druid.msq.input.MapInputSpecSlicer; +import org.apache.druid.msq.input.external.ExternalInputSpec; +import org.apache.druid.msq.input.external.ExternalInputSpecSlicer; +import org.apache.druid.msq.input.stage.InputChannels; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.input.stage.StageInputSpecSlicer; +import org.apache.druid.msq.input.table.TableInputSpec; +import org.apache.druid.msq.input.table.TableInputSpecSlicer; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.QueryDefinitionBuilder; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.msq.kernel.TargetSizeShuffleSpec; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernel; +import org.apache.druid.msq.kernel.controller.ControllerStagePhase; +import org.apache.druid.msq.kernel.controller.WorkerInputs; +import org.apache.druid.msq.querykit.DataSegmentTimelineView; +import org.apache.druid.msq.querykit.MultiQueryKit; +import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ShuffleSpecFactories; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.msq.querykit.groupby.GroupByQueryKit; +import org.apache.druid.msq.querykit.scan.ScanQueryKit; +import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory; +import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory; +import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.util.DimensionSchemaUtils; +import org.apache.druid.msq.util.IntervalUtils; +import org.apache.druid.msq.util.MSQFutureUtils; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.msq.util.PassthroughAggregatorFactory; +import org.apache.druid.query.Query; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.server.DruidNode; +import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.DimensionRangeShardSpec; +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.StreamSupport; + +public class ControllerImpl implements Controller +{ + private static final Logger log = new Logger(ControllerImpl.class); + + private final MSQControllerTask task; + private final ControllerContext context; + + /** + * Queue of "commands" to run on the {@link ControllerQueryKernel}. Various threads insert into the queue + * using {@link #addToKernelManipulationQueue}. The main thread running {@link RunQueryUntilDone#run()} reads + * from the queue and executes the commands. + * + * This ensures that all manipulations on {@link ControllerQueryKernel}, and all core logic, are run in + * a single-threaded manner. + */ + private final BlockingQueue> kernelManipulationQueue = + new ArrayBlockingQueue<>(Limits.MAX_KERNEL_MANIPULATION_QUEUE_SIZE); + + // For system error reporting. This is the very first error we got from a worker. (We only report that one.) + private final AtomicReference workerErrorRef = new AtomicReference<>(); + + // For system warning reporting + private final ConcurrentLinkedQueue workerWarnings = new ConcurrentLinkedQueue<>(); + + // Query definition. + // For live reports. Written by the main controller thread, read by HTTP threads. + private final AtomicReference queryDefRef = new AtomicReference<>(); + + // Last reported CounterSnapshots per stage per worker + // For live reports. Written by the main controller thread, read by HTTP threads. + private final CounterSnapshotsTree taskCountersForLiveReports = new CounterSnapshotsTree(); + + // Stage number -> stage phase + // For live reports. Written by the main controller thread, read by HTTP threads. + private final ConcurrentHashMap stagePhasesForLiveReports = new ConcurrentHashMap<>(); + + // Stage number -> runtime interval. Endpoint is eternity's end if the stage is still running. + // For live reports. Written by the main controller thread, read by HTTP threads. + private final ConcurrentHashMap stageRuntimesForLiveReports = new ConcurrentHashMap<>(); + + // Stage number -> worker count. Only set for stages that have started. + // For live reports. Written by the main controller thread, read by HTTP threads. + private final ConcurrentHashMap stageWorkerCountsForLiveReports = new ConcurrentHashMap<>(); + + // Stage number -> partition count. Only set for stages that have started. + // For live reports. Written by the main controller thread, read by HTTP threads. + private final ConcurrentHashMap stagePartitionCountsForLiveReports = new ConcurrentHashMap<>(); + + // Time at which the query started. + // For live reports. Written by the main controller thread, read by HTTP threads. + private volatile DateTime queryStartTime = null; + + private volatile DruidNode selfDruidNode; + private volatile MSQWorkerTaskLauncher workerTaskLauncher; + private volatile WorkerClient netClient; + + private volatile FaultsExceededChecker faultsExceededChecker = null; + + public ControllerImpl( + final MSQControllerTask task, + final ControllerContext context + ) + { + this.task = task; + this.context = context; + } + + @Override + public String id() + { + return task.getId(); + } + + @Override + public MSQControllerTask task() + { + return task; + } + + @Override + public TaskStatus run() throws Exception + { + final Closer closer = Closer.create(); + + try { + return runTask(closer); + } + catch (Throwable e) { + try { + closer.close(); + } + catch (Throwable e2) { + e.addSuppressed(e2); + } + + // We really don't expect this to error out. runTask should handle everything nicely. If it doesn't, something + // strange happened, so log it. + log.warn(e, "Encountered unhandled controller exception."); + return TaskStatus.failure(id(), e.toString()); + } + finally { + closer.close(); + } + } + + @Override + public void stopGracefully() + { + final QueryDefinition queryDef = queryDefRef.get(); + + // stopGracefully() is called when the containing process is terminated, or when the task is canceled. + log.info("Query [%s] canceled.", queryDef != null ? queryDef.getQueryId() : ""); + + addToKernelManipulationQueue( + kernel -> { + throw new MSQException(CanceledFault.INSTANCE); + } + ); + } + + public TaskStatus runTask(final Closer closer) + { + QueryDefinition queryDef = null; + ControllerQueryKernel queryKernel = null; + ListenableFuture workerTaskRunnerFuture = null; + CounterSnapshotsTree countersSnapshot = null; + Yielder resultsYielder = null; + Throwable exceptionEncountered = null; + + final TaskState taskStateForReport; + final MSQErrorReport errorForReport; + + try { + // Planning-related: convert the native query from MSQSpec into a multi-stage QueryDefinition. + this.queryStartTime = DateTimes.nowUtc(); + queryDef = initializeQueryDefAndState(closer); + + final InputSpecSlicerFactory inputSpecSlicerFactory = makeInputSpecSlicerFactory(makeDataSegmentTimelineView()); + + // Execution-related: run the multi-stage QueryDefinition. + final Pair> queryRunResult = + new RunQueryUntilDone(queryDef, inputSpecSlicerFactory, closer).run(); + + queryKernel = Preconditions.checkNotNull(queryRunResult.lhs); + workerTaskRunnerFuture = Preconditions.checkNotNull(queryRunResult.rhs); + resultsYielder = getFinalResultsYielder(queryDef, queryKernel); + publishSegmentsIfNeeded(queryDef, queryKernel); + } + catch (Throwable e) { + exceptionEncountered = e; + } + + // Fetch final counters in separate try, in case runQueryUntilDone threw an exception. + try { + countersSnapshot = getFinalCountersSnapshot(queryKernel); + } + catch (Throwable e) { + if (exceptionEncountered != null) { + exceptionEncountered.addSuppressed(e); + } else { + exceptionEncountered = e; + } + } + + if (queryKernel != null && queryKernel.isSuccess() && exceptionEncountered == null) { + taskStateForReport = TaskState.SUCCESS; + errorForReport = null; + } else { + // Query failure. Generate an error report and log the error(s) we encountered. + final String selfHost = MSQTasks.getHostFromSelfNode(selfDruidNode); + final MSQErrorReport controllerError = + exceptionEncountered != null + ? MSQErrorReport.fromException(id(), selfHost, null, exceptionEncountered) + : null; + final MSQErrorReport workerError = workerErrorRef.get(); + + taskStateForReport = TaskState.FAILED; + errorForReport = MSQTasks.makeErrorReport(id(), selfHost, controllerError, workerError); + + // Log the errors we encountered. + if (controllerError != null) { + log.warn("Controller: %s", MSQTasks.errorReportToLogMessage(controllerError)); + } + + if (workerError != null) { + log.warn("Worker: %s", MSQTasks.errorReportToLogMessage(workerError)); + } + } + + try { + // Write report even if something went wrong. + final MSQStagesReport stagesReport; + final MSQResultsReport resultsReport; + + if (queryDef != null) { + final Map stagePhaseMap; + + if (queryKernel != null) { + // Once the query finishes, cleanup would have happened for all the stages that were successful + // Therefore we mark it as done to make the reports prettier and more accurate + queryKernel.markSuccessfulTerminalStagesAsFinished(); + stagePhaseMap = queryKernel.getActiveStages() + .stream() + .collect( + Collectors.toMap(StageId::getStageNumber, queryKernel::getStagePhase) + ); + } else { + stagePhaseMap = Collections.emptyMap(); + } + + stagesReport = makeStageReport( + queryDef, + stagePhaseMap, + stageRuntimesForLiveReports, + stageWorkerCountsForLiveReports, + stagePartitionCountsForLiveReports + ); + } else { + stagesReport = null; + } + + if (resultsYielder != null) { + resultsReport = makeResultsTaskReport( + queryDef, + resultsYielder, + task.getQuerySpec().getColumnMappings(), + task.getSqlTypeNames() + ); + } else { + resultsReport = null; + } + + final MSQTaskReportPayload taskReportPayload = new MSQTaskReportPayload( + makeStatusReport( + taskStateForReport, + errorForReport, + workerWarnings, + queryStartTime, + new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis() + ), + stagesReport, + countersSnapshot, + resultsReport + ); + + context.writeReports( + id(), + TaskReport.buildTaskReports(new MSQTaskReport(id(), taskReportPayload)) + ); + } + catch (Throwable e) { + log.warn(e, "Error encountered while writing task report. Skipping."); + } + + if (queryKernel != null && queryKernel.isSuccess()) { + // If successful, encourage the tasks to exit successfully. + postFinishToAllTasks(); + workerTaskLauncher.stop(false); + } else { + // If not successful, cancel running tasks. + if (workerTaskLauncher != null) { + workerTaskLauncher.stop(true); + } + } + + // Wait for worker tasks to exit. Ignore their return status. At this point, we've done everything we need to do, + // so we don't care about the task exit status. + if (workerTaskRunnerFuture != null) { + try { + workerTaskRunnerFuture.get(); + } + catch (Exception ignored) { + // Suppress. + } + } + + cleanUpDurableStorageIfNeeded(); + + if (taskStateForReport == TaskState.SUCCESS) { + return TaskStatus.success(id()); + } else { + // errorForReport is nonnull when taskStateForReport != SUCCESS. Use that message. + return TaskStatus.failure(id(), errorForReport.getFault().getCodeWithMessage()); + } + } + + /** + * Adds some logic to {@link #kernelManipulationQueue}, where it will, in due time, be executed by the main + * controller loop in {@link RunQueryUntilDone#run()}. + * + * If the consumer throws an exception, the query fails. + */ + private void addToKernelManipulationQueue(Consumer kernelConsumer) + { + if (!kernelManipulationQueue.offer(kernelConsumer)) { + final String message = "Controller kernel queue is full. Main controller loop may be delayed or stuck."; + log.warn(message); + throw new IllegalStateException(message); + } + } + + private QueryDefinition initializeQueryDefAndState(final Closer closer) + { + this.selfDruidNode = context.selfNode(); + context.registerController(this, closer); + + this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); + closer.register(netClient::close); + + final boolean isDurableStorageEnabled = + MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext()); + + final QueryDefinition queryDef = makeQueryDefinition( + id(), + makeQueryControllerToolKit(), + task.getQuerySpec() + ); + + QueryValidator.validateQueryDef(queryDef); + queryDefRef.set(queryDef); + + log.debug("Query [%s] durable storage mode is set to %s.", queryDef.getQueryId(), isDurableStorageEnabled); + + this.workerTaskLauncher = new MSQWorkerTaskLauncher( + id(), + task.getDataSource(), + context, + isDurableStorageEnabled, + + // 10 minutes +- 2 minutes jitter + TimeUnit.SECONDS.toMillis(600 + ThreadLocalRandom.current().nextInt(-4, 5) * 30L) + ); + + long maxParseExceptions = -1; + + if (task.getSqlQueryContext() != null) { + maxParseExceptions = Optional.ofNullable( + task.getSqlQueryContext().get(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED)) + .map(DimensionHandlerUtils::convertObjectToLong) + .orElse(MSQWarnings.DEFAULT_MAX_PARSE_EXCEPTIONS_ALLOWED); + } + + this.faultsExceededChecker = new FaultsExceededChecker( + ImmutableMap.of(CannotParseExternalDataFault.CODE, maxParseExceptions) + ); + + return queryDef; + } + + /** + * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages. + */ + @Override + public void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject) + { + addToKernelManipulationQueue( + queryKernel -> { + final StageId stageId = queryKernel.getStageId(stageNumber); + + // We need a specially-decorated ObjectMapper to deserialize key statistics. + final StageDefinition stageDef = queryKernel.getStageDefinition(stageId); + final ObjectMapper mapper = MSQTasks.decorateObjectMapperForKeyCollectorSnapshot( + context.jsonMapper(), + stageDef.getShuffleSpec().get().getClusterBy(), + stageDef.getShuffleSpec().get().doesAggregateByClusterKey() + ); + + final ClusterByStatisticsSnapshot keyStatistics; + try { + keyStatistics = mapper.convertValue(keyStatisticsObject, ClusterByStatisticsSnapshot.class); + } + catch (IllegalArgumentException e) { + throw new IAE( + e, + "Unable to deserialize the key statistic for stage [%s] received from the worker [%d]", + stageId, + workerNumber + ); + } + + queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics); + } + ); + } + + @Override + public void workerError(MSQErrorReport errorReport) + { + if (!workerTaskLauncher.isTaskCanceledByController(errorReport.getTaskId())) { + workerErrorRef.compareAndSet(null, errorReport); + } + } + + /** + * This method intakes all the warnings that are generated by the worker. It is the responsibility of the + * worker node to ensure that it doesn't spam the controller with unneseccary warning stack traces. Currently, that + * limiting is implemented in {@link MSQWarningReportLimiterPublisher} + */ + @Override + public void workerWarning(List errorReports) + { + // This check safeguards that the controller doesn't run out of memory. Workers apply their own limiting to + // protect their own memory, and to conserve worker -> controller bandwidth. + long numReportsToAddCheck = Math.min( + errorReports.size(), + Limits.MAX_WORKERS * Limits.MAX_VERBOSE_WARNINGS - workerWarnings.size() + ); + if (numReportsToAddCheck > 0) { + synchronized (workerWarnings) { + long numReportsToAdd = Math.min( + errorReports.size(), + Limits.MAX_WORKERS * Limits.MAX_VERBOSE_WARNINGS - workerWarnings.size() + ); + for (int i = 0; i < numReportsToAdd; ++i) { + workerWarnings.add(errorReports.get(i)); + } + } + } + } + + /** + * Periodic update of {@link CounterSnapshots} from subtasks. + */ + @Override + public void updateCounters(CounterSnapshotsTree snapshotsTree) + { + taskCountersForLiveReports.putAll(snapshotsTree); + Optional> warningsExceeded = + faultsExceededChecker.addFaultsAndCheckIfExceeded(taskCountersForLiveReports); + + if (warningsExceeded.isPresent()) { + // Present means the warning limit was exceeded, and warnings have therefore turned into an error. + String errorCode = warningsExceeded.get().lhs; + Long limit = warningsExceeded.get().rhs; + workerError(MSQErrorReport.fromFault( + id(), + selfDruidNode.getHost(), + null, + new TooManyWarningsFault(limit.intValue(), errorCode) + )); + addToKernelManipulationQueue( + queryKernel -> + queryKernel.getActiveStages().forEach(queryKernel::failStage) + ); + } + } + + /** + * Reports that results are ready for a subtask. + */ + @SuppressWarnings("unchecked") + @Override + public void resultsComplete( + final String queryId, + final int stageNumber, + final int workerNumber, + Object resultObject + ) + { + addToKernelManipulationQueue( + queryKernel -> { + final StageId stageId = new StageId(queryId, stageNumber); + final Object convertedResultObject; + try { + convertedResultObject = context.jsonMapper().convertValue( + resultObject, + queryKernel.getStageDefinition(stageId).getProcessorFactory().getAccumulatedResultTypeReference() + ); + } + catch (IllegalArgumentException e) { + throw new IAE( + e, + "Unable to deserialize the result object for stage [%s] received from the worker [%d]", + stageId, + workerNumber + ); + } + + + queryKernel.setResultsCompleteForStageAndWorker(stageId, workerNumber, convertedResultObject); + } + ); + } + + @Override + @Nullable + public Map liveReports() + { + final QueryDefinition queryDef = queryDefRef.get(); + + if (queryDef == null) { + return null; + } + + return TaskReport.buildTaskReports( + new MSQTaskReport( + id(), + new MSQTaskReportPayload( + makeStatusReport( + TaskState.RUNNING, + null, + workerWarnings, + queryStartTime, + queryStartTime == null ? -1L : new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis() + ), + makeStageReport( + queryDef, + stagePhasesForLiveReports, + stageRuntimesForLiveReports, + stageWorkerCountsForLiveReports, + stagePartitionCountsForLiveReports + ), + makeCountersSnapshotForLiveReports(), + null + ) + ) + ); + } + + /** + * Returns the segments that will be generated by this job. Delegates to + * {@link #generateSegmentIdsWithShardSpecsForAppend} or {@link #generateSegmentIdsWithShardSpecsForReplace} as + * appropriate. This is a potentially expensive call, since it requires calling Overlord APIs. + * + * @throws MSQException with {@link InsertCannotAllocateSegmentFault} if an allocation cannot be made + */ + private List generateSegmentIdsWithShardSpecs( + final DataSourceMSQDestination destination, + final RowSignature signature, + final ClusterBy clusterBy, + final ClusterByPartitions partitionBoundaries, + final boolean mayHaveMultiValuedClusterByFields + ) throws IOException + { + if (destination.isReplaceTimeChunks()) { + return generateSegmentIdsWithShardSpecsForReplace( + destination, + signature, + clusterBy, + partitionBoundaries, + mayHaveMultiValuedClusterByFields + ); + } else { + final RowKeyReader keyReader = clusterBy.keyReader(signature); + return generateSegmentIdsWithShardSpecsForAppend(destination, partitionBoundaries, keyReader); + } + } + + /** + * Used by {@link #generateSegmentIdsWithShardSpecs}. + */ + private List generateSegmentIdsWithShardSpecsForAppend( + final DataSourceMSQDestination destination, + final ClusterByPartitions partitionBoundaries, + final RowKeyReader keyReader + ) throws IOException + { + final Granularity segmentGranularity = destination.getSegmentGranularity(); + + String previousSegmentId = null; + + final List retVal = new ArrayList<>(partitionBoundaries.size()); + + for (ClusterByPartition partitionBoundary : partitionBoundaries) { + final DateTime timestamp = getBucketDateTime(partitionBoundary, segmentGranularity, keyReader); + final SegmentIdWithShardSpec allocation; + try { + allocation = context.taskActionClient().submit( + new SegmentAllocateAction( + task.getDataSource(), + timestamp, + // Same granularity for queryGranularity, segmentGranularity because we don't have insight here + // into what queryGranularity "actually" is. (It depends on what time floor function was used.) + segmentGranularity, + segmentGranularity, + id(), + previousSegmentId, + false, + NumberedPartialShardSpec.instance(), + LockGranularity.TIME_CHUNK, + TaskLockType.SHARED + ) + ); + } + catch (ISE e) { + if (isTaskLockPreemptedException(e)) { + throw new MSQException(e, InsertLockPreemptedFault.instance()); + } else { + throw e; + } + } + + if (allocation == null) { + throw new MSQException( + new InsertCannotAllocateSegmentFault( + task.getDataSource(), + segmentGranularity.bucket(timestamp) + ) + ); + } + + retVal.add(allocation); + previousSegmentId = allocation.asSegmentId().toString(); + } + + return retVal; + } + + /** + * Used by {@link #generateSegmentIdsWithShardSpecs}. + */ + private List generateSegmentIdsWithShardSpecsForReplace( + final DataSourceMSQDestination destination, + final RowSignature signature, + final ClusterBy clusterBy, + final ClusterByPartitions partitionBoundaries, + final boolean mayHaveMultiValuedClusterByFields + ) throws IOException + { + final RowKeyReader keyReader = clusterBy.keyReader(signature); + final SegmentIdWithShardSpec[] retVal = new SegmentIdWithShardSpec[partitionBoundaries.size()]; + final Granularity segmentGranularity = destination.getSegmentGranularity(); + final List shardColumns; + + if (mayHaveMultiValuedClusterByFields) { + // DimensionRangeShardSpec cannot handle multi-valued fields. + shardColumns = Collections.emptyList(); + } else { + shardColumns = computeShardColumns(signature, clusterBy, task.getQuerySpec().getColumnMappings()); + } + + // Group partition ranges by bucket (time chunk), so we can generate shardSpecs for each bucket independently. + final Map>> partitionsByBucket = new HashMap<>(); + for (int i = 0; i < partitionBoundaries.ranges().size(); i++) { + ClusterByPartition partitionBoundary = partitionBoundaries.ranges().get(i); + final DateTime bucketDateTime = getBucketDateTime(partitionBoundary, segmentGranularity, keyReader); + partitionsByBucket.computeIfAbsent(bucketDateTime, ignored -> new ArrayList<>()) + .add(Pair.of(i, partitionBoundary)); + } + + // Process buckets (time chunks) one at a time. + for (final Map.Entry>> bucketEntry : partitionsByBucket.entrySet()) { + final Interval interval = segmentGranularity.bucket(bucketEntry.getKey()); + + // Validate interval against the replaceTimeChunks set of intervals. + if (destination.getReplaceTimeChunks().stream().noneMatch(chunk -> chunk.contains(interval))) { + throw new MSQException(new InsertTimeOutOfBoundsFault(interval)); + } + + final List> ranges = bucketEntry.getValue(); + String version = null; + + final List locks = context.taskActionClient().submit(new LockListAction()); + for (final TaskLock lock : locks) { + if (lock.getInterval().contains(interval)) { + version = lock.getVersion(); + } + } + + if (version == null) { + // Lock was revoked, probably, because we should have originally acquired it in isReady. + throw new MSQException(InsertLockPreemptedFault.INSTANCE); + } + + for (int segmentNumber = 0; segmentNumber < ranges.size(); segmentNumber++) { + final int partitionNumber = ranges.get(segmentNumber).lhs; + final ShardSpec shardSpec; + + if (shardColumns.isEmpty()) { + shardSpec = new NumberedShardSpec(segmentNumber, ranges.size()); + } else { + final ClusterByPartition range = ranges.get(segmentNumber).rhs; + final StringTuple start = + segmentNumber == 0 ? null : makeStringTuple(clusterBy, keyReader, range.getStart()); + final StringTuple end = + segmentNumber == ranges.size() - 1 ? null : makeStringTuple(clusterBy, keyReader, range.getEnd()); + + shardSpec = new DimensionRangeShardSpec(shardColumns, start, end, segmentNumber, ranges.size()); + } + + retVal[partitionNumber] = new SegmentIdWithShardSpec(task.getDataSource(), interval, version, shardSpec); + } + } + + return Arrays.asList(retVal); + } + + /** + * Returns a complete list of task ids, ordered by worker number. The Nth task has worker number N. + * + * If the currently-running set of tasks is incomplete, returns an absent Optional. + */ + @Override + public List getTaskIds() + { + if (workerTaskLauncher == null) { + return Collections.emptyList(); + } + + return workerTaskLauncher.getTaskList(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Nullable + private Int2ObjectMap makeWorkerFactoryInfosForStage( + final QueryDefinition queryDef, + final int stageNumber, + final WorkerInputs workerInputs, + @Nullable final List segmentsToGenerate + ) + { + if (MSQControllerTask.isIngestion(task.getQuerySpec()) && + stageNumber == queryDef.getFinalStageDefinition().getStageNumber()) { + // noinspection unchecked,rawtypes + return (Int2ObjectMap) makeSegmentGeneratorWorkerFactoryInfos(workerInputs, segmentsToGenerate); + } else { + return null; + } + } + + @SuppressWarnings("rawtypes") + private QueryKit makeQueryControllerToolKit() + { + final Map, QueryKit> kitMap = + ImmutableMap., QueryKit>builder() + .put(ScanQuery.class, new ScanQueryKit(context.jsonMapper())) + .put(GroupByQuery.class, new GroupByQueryKit()) + .build(); + + return new MultiQueryKit(kitMap); + } + + private DataSegmentTimelineView makeDataSegmentTimelineView() + { + return (dataSource, intervals) -> { + final Collection dataSegments = + context.coordinatorClient().fetchUsedSegmentsInDataSourceForIntervals(dataSource, intervals); + + if (dataSegments.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(VersionedIntervalTimeline.forSegments(dataSegments)); + } + }; + } + + private Int2ObjectMap> makeSegmentGeneratorWorkerFactoryInfos( + final WorkerInputs workerInputs, + final List segmentsToGenerate + ) + { + final Int2ObjectMap> retVal = new Int2ObjectAVLTreeMap<>(); + + for (final int workerNumber : workerInputs.workers()) { + // SegmentGenerator stage has a single input from another stage. + final StageInputSlice stageInputSlice = + (StageInputSlice) Iterables.getOnlyElement(workerInputs.inputsForWorker(workerNumber)); + + final List workerSegments = new ArrayList<>(); + retVal.put(workerNumber, workerSegments); + + for (final ReadablePartition partition : stageInputSlice.getPartitions()) { + workerSegments.add(segmentsToGenerate.get(partition.getPartitionNumber())); + } + } + + return retVal; + } + + private void contactWorkersForStage(final TaskContactFn contactFn, final IntSet workers) + { + final List taskIds = getTaskIds(); + final List> taskFutures = new ArrayList<>(workers.size()); + + for (int workerNumber : workers) { + final String taskId = taskIds.get(workerNumber); + taskFutures.add(contactFn.contactTask(netClient, taskId, workerNumber)); + } + + FutureUtils.getUnchecked(MSQFutureUtils.allAsList(taskFutures, true), true); + } + + private void startWorkForStage( + final QueryDefinition queryDef, + final ControllerQueryKernel queryKernel, + final int stageNumber, + @Nullable final List segmentsToGenerate + ) + { + final Int2ObjectMap extraInfos = makeWorkerFactoryInfosForStage( + queryDef, + stageNumber, + queryKernel.getWorkerInputsForStage(queryKernel.getStageId(stageNumber)), + segmentsToGenerate + ); + + final Int2ObjectMap workOrders = queryKernel.createWorkOrders(stageNumber, extraInfos); + + contactWorkersForStage( + (netClient, taskId, workerNumber) -> netClient.postWorkOrder(taskId, workOrders.get(workerNumber)), + workOrders.keySet() + ); + } + + private void postResultPartitionBoundariesForStage( + final QueryDefinition queryDef, + final int stageNumber, + final ClusterByPartitions resultPartitionBoundaries, + final IntSet workers + ) + { + contactWorkersForStage( + (netClient, taskId, workerNumber) -> + netClient.postResultPartitionBoundaries( + taskId, + new StageId(queryDef.getQueryId(), stageNumber), + resultPartitionBoundaries + ), + workers + ); + } + + /** + * Publish the list of segments. Additionally, if {@link DataSourceMSQDestination#isReplaceTimeChunks()}, + * also drop all other segments within the replacement intervals. + * + * If any existing segments cannot be dropped because their intervals are not wholly contained within the + * replacement parameter, throws a {@link MSQException} with {@link InsertCannotReplaceExistingSegmentFault}. + */ + private void publishAllSegments(final Set segments) throws IOException + { + final DataSourceMSQDestination destination = + (DataSourceMSQDestination) task.getQuerySpec().getDestination(); + final Set segmentsToDrop; + + if (destination.isReplaceTimeChunks()) { + final List intervalsToDrop = findIntervalsToDrop(Preconditions.checkNotNull(segments, "segments")); + + if (intervalsToDrop.isEmpty()) { + segmentsToDrop = null; + } else { + // Determine which segments to drop as part of the replace operation. This is safe because, in the case where we + // are doing a replace, the isReady method (which runs prior to the task starting) acquires an exclusive lock. + segmentsToDrop = + ImmutableSet.copyOf( + context.taskActionClient().submit( + new RetrieveUsedSegmentsAction( + task.getDataSource(), + null, + intervalsToDrop, + Segments.ONLY_VISIBLE + ) + ) + ); + + // Validate that there are no segments that partially overlap the intervals-to-drop. Otherwise, the replace + // may be incomplete. + for (final DataSegment segmentToDrop : segmentsToDrop) { + if (destination.getReplaceTimeChunks() + .stream() + .noneMatch(interval -> interval.contains(segmentToDrop.getInterval()))) { + throw new MSQException(new InsertCannotReplaceExistingSegmentFault(segmentToDrop.getId())); + } + } + } + + if (segments.isEmpty()) { + // Nothing to publish, only drop. We already validated that the intervalsToDrop do not have any + // partially-overlapping segments, so it's safe to drop them as intervals instead of as specific segments. + for (final Interval interval : intervalsToDrop) { + context.taskActionClient() + .submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval)); + } + } else { + try { + context.taskActionClient() + .submit(SegmentTransactionalInsertAction.overwriteAction(null, segmentsToDrop, segments)); + } + catch (Exception e) { + if (isTaskLockPreemptedException(e)) { + throw new MSQException(e, InsertLockPreemptedFault.instance()); + } else { + throw e; + } + } + } + } else if (!segments.isEmpty()) { + // Append mode. + try { + context.taskActionClient().submit(new SegmentInsertAction(segments)); + } + catch (Exception e) { + if (isTaskLockPreemptedException(e)) { + throw new MSQException(e, InsertLockPreemptedFault.instance()); + } else { + throw e; + } + } + } + } + + /** + * When doing an ingestion with {@link DataSourceMSQDestination#isReplaceTimeChunks()}, finds intervals + * containing data that should be dropped. + */ + private List findIntervalsToDrop(final Set publishedSegments) + { + // Safe to cast because publishAllSegments is only called for dataSource destinations. + final DataSourceMSQDestination destination = + (DataSourceMSQDestination) task.getQuerySpec().getDestination(); + final List replaceIntervals = + new ArrayList<>(JodaUtils.condenseIntervals(destination.getReplaceTimeChunks())); + final List publishIntervals = + JodaUtils.condenseIntervals(Iterables.transform(publishedSegments, DataSegment::getInterval)); + return IntervalUtils.difference(replaceIntervals, publishIntervals); + } + + private CounterSnapshotsTree getCountersFromAllTasks() + { + final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); + final List taskList = workerTaskLauncher.getTaskList(); + + final List> futures = new ArrayList<>(); + + for (String taskId : taskList) { + futures.add(netClient.getCounters(taskId)); + } + + final List snapshotsTrees = + FutureUtils.getUnchecked(MSQFutureUtils.allAsList(futures, true), true); + + for (CounterSnapshotsTree snapshotsTree : snapshotsTrees) { + retVal.putAll(snapshotsTree); + } + + return retVal; + } + + private void postFinishToAllTasks() + { + final List taskList = workerTaskLauncher.getTaskList(); + + final List> futures = new ArrayList<>(); + + for (String taskId : taskList) { + futures.add(netClient.postFinish(taskId)); + } + + FutureUtils.getUnchecked(MSQFutureUtils.allAsList(futures, true), true); + } + + private CounterSnapshotsTree makeCountersSnapshotForLiveReports() + { + // taskCountersForLiveReports is mutable: Copy so we get a point-in-time snapshot. + return CounterSnapshotsTree.fromMap(taskCountersForLiveReports.copyMap()); + } + + private CounterSnapshotsTree getFinalCountersSnapshot(@Nullable final ControllerQueryKernel queryKernel) + { + if (queryKernel != null && queryKernel.isSuccess()) { + return getCountersFromAllTasks(); + } else { + return makeCountersSnapshotForLiveReports(); + } + } + + @Nullable + private Yielder getFinalResultsYielder( + final QueryDefinition queryDef, + final ControllerQueryKernel queryKernel + ) + { + if (queryKernel.isSuccess() && isInlineResults(task.getQuerySpec())) { + final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber()); + final List taskIds = getTaskIds(); + final Closer closer = Closer.create(); + + final ListeningExecutorService resultReaderExec = + MoreExecutors.listeningDecorator(Execs.singleThreaded("result-reader-%d")); + closer.register(resultReaderExec::shutdownNow); + + final InputChannelFactory inputChannelFactory; + + if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext())) { + inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation( + id(), + () -> taskIds, + MSQTasks.makeStorageConnector(context.injector()), + closer + ); + } else { + inputChannelFactory = new WorkerInputChannelFactory(netClient, () -> taskIds); + } + + final InputChannels inputChannels = new InputChannelsImpl( + queryDef, + queryKernel.getResultPartitionsForStage(finalStageId), + inputChannelFactory, + () -> ArenaMemoryAllocator.createOnHeap(5_000_000), + new FrameProcessorExecutor(resultReaderExec), + null + ); + + return Yielders.each( + Sequences.concat( + StreamSupport.stream(queryKernel.getResultPartitionsForStage(finalStageId).spliterator(), false) + .map( + readablePartition -> { + try { + return new FrameChannelSequence( + inputChannels.openChannel( + new StagePartition( + queryKernel.getStageDefinition(finalStageId).getId(), + readablePartition.getPartitionNumber() + ) + ) + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ).collect(Collectors.toList()) + ).flatMap( + frame -> { + final Cursor cursor = FrameProcessors.makeCursor( + frame, + queryKernel.getStageDefinition(finalStageId).getFrameReader() + ); + + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final ColumnMappings columnMappings = task.getQuerySpec().getColumnMappings(); + @SuppressWarnings("rawtypes") + final List selectors = + columnMappings.getMappings() + .stream() + .map( + mapping -> + columnSelectorFactory.makeColumnValueSelector( + mapping.getQueryColumn()) + ).collect(Collectors.toList()); + + final List retVal = new ArrayList<>(); + while (!cursor.isDone()) { + final Object[] row = new Object[columnMappings.getMappings().size()]; + for (int i = 0; i < row.length; i++) { + row[i] = selectors.get(i).getObject(); + } + retVal.add(row); + cursor.advance(); + } + + return Sequences.simple(retVal); + } + ).withBaggage(resultReaderExec::shutdownNow) + ); + } else { + return null; + } + } + + private void publishSegmentsIfNeeded( + final QueryDefinition queryDef, + final ControllerQueryKernel queryKernel + ) throws IOException + { + if (queryKernel.isSuccess() && MSQControllerTask.isIngestion(task.getQuerySpec())) { + final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber()); + + //noinspection unchecked + @SuppressWarnings("unchecked") + final Set segments = (Set) queryKernel.getResultObjectForStage(finalStageId); + log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size()); + publishAllSegments(segments); + } + } + + /** + * Clean up durable storage, if used for stage output. + * + * Note that this is only called by the controller task itself. It isn't called automatically by anything in + * particular if the controller fails early without being able to run its cleanup routines. This can cause files + * to be left in durable storage beyond their useful life. + */ + private void cleanUpDurableStorageIfNeeded() + { + if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext())) { + final String controllerDirName = DurableStorageOutputChannelFactory.getControllerDirectory(task.getId()); + try { + // Delete all temporary files as a failsafe + MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(controllerDirName); + } + catch (Exception e) { + // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup + log.warn(e, "Error while cleaning up temporary files at path %s", controllerDirName); + } + } + } + + @SuppressWarnings("unchecked") + private static QueryDefinition makeQueryDefinition( + final String queryId, + @SuppressWarnings("rawtypes") final QueryKit toolKit, + final MSQSpec querySpec + ) + { + final MSQTuningConfig tuningConfig = querySpec.getTuningConfig(); + final ShuffleSpecFactory shuffleSpecFactory; + + if (MSQControllerTask.isIngestion(querySpec)) { + shuffleSpecFactory = (clusterBy, aggregate) -> + new TargetSizeShuffleSpec( + clusterBy, + tuningConfig.getRowsPerSegment(), + aggregate + ); + } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { + shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); + } else { + throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); + } + + final Query queryToPlan; + + if (querySpec.getColumnMappings().hasOutputColumn(ColumnHolder.TIME_COLUMN_NAME)) { + queryToPlan = querySpec.getQuery().withOverriddenContext( + ImmutableMap.of( + QueryKitUtils.CTX_TIME_COLUMN_NAME, + querySpec.getColumnMappings().getQueryColumnForOutputColumn(ColumnHolder.TIME_COLUMN_NAME) + ) + ); + } else { + queryToPlan = querySpec.getQuery(); + } + + final QueryDefinition queryDef; + + try { + queryDef = toolKit.makeQueryDefinition( + queryId, + queryToPlan, + toolKit, + shuffleSpecFactory, + tuningConfig.getMaxNumWorkers(), + 0 + ); + } + catch (MSQException e) { + // If the toolkit throws a MSQFault, don't wrap it in a more generic QueryNotSupportedFault + throw e; + } + catch (Exception e) { + throw new MSQException(e, QueryNotSupportedFault.INSTANCE); + } + + if (MSQControllerTask.isIngestion(querySpec)) { + final RowSignature querySignature = queryDef.getFinalStageDefinition().getSignature(); + final ClusterBy queryClusterBy = queryDef.getFinalStageDefinition().getClusterBy(); + final ColumnMappings columnMappings = querySpec.getColumnMappings(); + + // Find the stage that provides shuffled input to the final segment-generation stage. + StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition(); + + while (!finalShuffleStageDef.doesShuffle() + && InputSpecs.getStageNumbers(finalShuffleStageDef.getInputSpecs()).size() == 1) { + finalShuffleStageDef = queryDef.getStageDefinition( + Iterables.getOnlyElement(InputSpecs.getStageNumbers(finalShuffleStageDef.getInputSpecs())) + ); + } + + if (!finalShuffleStageDef.doesShuffle()) { + finalShuffleStageDef = null; + } + + // Add all query stages. + // Set shuffleCheckHasMultipleValues on the stage that serves as input to the final segment-generation stage. + final QueryDefinitionBuilder builder = QueryDefinition.builder(); + + for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { + if (stageDef.equals(finalShuffleStageDef)) { + builder.add(StageDefinition.builder(stageDef).shuffleCheckHasMultipleValues(true)); + } else { + builder.add(StageDefinition.builder(stageDef)); + } + } + + // Then, add a segment-generation stage. + final DataSchema dataSchema = generateDataSchema(querySpec, querySignature, queryClusterBy, columnMappings); + builder.add( + StageDefinition.builder(queryDef.getNextStageNumber()) + .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) + .maxWorkerCount(tuningConfig.getMaxNumWorkers()) + .processorFactory( + new SegmentGeneratorFrameProcessorFactory( + dataSchema, + columnMappings, + tuningConfig + ) + ) + ); + + return builder.build(); + } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { + return queryDef; + } else { + throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); + } + } + + private static DataSchema generateDataSchema( + MSQSpec querySpec, + RowSignature querySignature, + ClusterBy queryClusterBy, + ColumnMappings columnMappings + ) + { + final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); + final boolean isRollupQuery = isRollupQuery(querySpec.getQuery()); + + final Pair, List> dimensionsAndAggregators = + makeDimensionsAndAggregatorsForIngestion( + querySignature, + queryClusterBy, + destination.getSegmentSortOrder(), + columnMappings, + isRollupQuery, + querySpec.getQuery() + ); + + return new DataSchema( + destination.getDataSource(), + new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), + new DimensionsSpec(dimensionsAndAggregators.lhs), + dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]), + makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery), + new TransformSpec(null, Collections.emptyList()) + ); + } + + private static GranularitySpec makeGranularitySpecForIngestion( + final Query query, + final ColumnMappings columnMappings, + final boolean isRollupQuery + ) + { + if (isRollupQuery) { + final String queryGranularity = query.getContextValue(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_GRANULARITY, ""); + + if (timeIsGroupByDimension((GroupByQuery) query, columnMappings) && !queryGranularity.isEmpty()) { + return new ArbitraryGranularitySpec( + Granularity.fromString(queryGranularity), + true, + Intervals.ONLY_ETERNITY + ); + } + return new ArbitraryGranularitySpec(Granularities.NONE, true, Intervals.ONLY_ETERNITY); + } else { + return new ArbitraryGranularitySpec(Granularities.NONE, false, Intervals.ONLY_ETERNITY); + } + } + + /** + * Checks that a {@link GroupByQuery} is grouping on the primary time column. + * + * The logic here is roundabout. First, we check which column in the {@link GroupByQuery} corresponds to the + * output column {@link ColumnHolder#TIME_COLUMN_NAME}, using our {@link ColumnMappings}. Then, we check for the + * presence of an optimization done in {@link DruidQuery#toGroupByQuery()}, where the context parameter + * {@link GroupByQuery#CTX_TIMESTAMP_RESULT_FIELD} and various related parameters are set when one of the dimensions + * is detected to be a time-floor. Finally, we check that the name of that dimension, and the name of our time field + * from {@link ColumnMappings}, are the same. + */ + private static boolean timeIsGroupByDimension(GroupByQuery groupByQuery, ColumnMappings columnMappings) + { + if (columnMappings.hasOutputColumn(ColumnHolder.TIME_COLUMN_NAME)) { + final String queryTimeColumn = columnMappings.getQueryColumnForOutputColumn(ColumnHolder.TIME_COLUMN_NAME); + return queryTimeColumn.equals(groupByQuery.getContextValue(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD)); + } else { + return false; + } + } + + /** + * Whether a native query represents an ingestion with rollup. + * + * Checks for three things: + * + * - The query must be a {@link GroupByQuery}, because rollup requires columns to be split into dimensions and + * aggregations. + * - The query must not finalize aggregations, because rollup requires inserting the intermediate type of + * complex aggregations, not the finalized type. (So further rollup is possible.) + * - The query must explicitly disable {@link GroupByQueryConfig#CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING}, because + * groupBy on multi-value dimensions implicitly unnests, which is not desired behavior for rollup at ingestion time + * (rollup expects multi-value dimensions to be treated as arrays). + */ + private static boolean isRollupQuery(Query query) + { + return query instanceof GroupByQuery + && !MultiStageQueryContext.isFinalizeAggregations(query.getQueryContext()) + && !query.getContextBoolean(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, true); + } + + private static boolean isInlineResults(final MSQSpec querySpec) + { + return querySpec.getDestination() instanceof TaskReportMSQDestination; + } + + private static boolean isTimeBucketedIngestion(final MSQSpec querySpec) + { + return MSQControllerTask.isIngestion(querySpec) + && !((DataSourceMSQDestination) querySpec.getDestination()).getSegmentGranularity() + .equals(Granularities.ALL); + } + + /** + * Compute shard columns for {@link DimensionRangeShardSpec}. Returns an empty list if range-based sharding + * is not applicable. + */ + private static List computeShardColumns( + final RowSignature signature, + final ClusterBy clusterBy, + final ColumnMappings columnMappings + ) + { + final List clusterByColumns = clusterBy.getColumns(); + final List shardColumns = new ArrayList<>(); + final boolean boosted = isClusterByBoosted(clusterBy); + final int numShardColumns = clusterByColumns.size() - clusterBy.getBucketByCount() - (boosted ? 1 : 0); + + if (numShardColumns == 0) { + return Collections.emptyList(); + } + + for (int i = clusterBy.getBucketByCount(); i < clusterBy.getBucketByCount() + numShardColumns; i++) { + final SortColumn column = clusterByColumns.get(i); + final List outputColumns = columnMappings.getOutputColumnsForQueryColumn(column.columnName()); + + // DimensionRangeShardSpec only handles ascending order. + if (column.descending()) { + return Collections.emptyList(); + } + + ColumnType columnType = signature.getColumnType(column.columnName()).orElse(null); + + // DimensionRangeShardSpec only handles strings. + if (!(ColumnType.STRING.equals(columnType))) { + return Collections.emptyList(); + } + + // DimensionRangeShardSpec only handles columns that appear as-is in the output. + if (outputColumns.isEmpty()) { + return Collections.emptyList(); + } + + shardColumns.add(outputColumns.get(0)); + } + + return shardColumns; + } + + /** + * Checks if the {@link ClusterBy} has a {@link QueryKitUtils#PARTITION_BOOST_COLUMN}. See javadocs for that + * constant for more details about what it does. + */ + private static boolean isClusterByBoosted(final ClusterBy clusterBy) + { + return !clusterBy.getColumns().isEmpty() + && clusterBy.getColumns() + .get(clusterBy.getColumns().size() - 1) + .columnName() + .equals(QueryKitUtils.PARTITION_BOOST_COLUMN); + } + + private static StringTuple makeStringTuple( + final ClusterBy clusterBy, + final RowKeyReader keyReader, + final RowKey key + ) + { + final String[] array = new String[clusterBy.getColumns().size() - clusterBy.getBucketByCount()]; + final boolean boosted = isClusterByBoosted(clusterBy); + + for (int i = 0; i < array.length; i++) { + final Object val = keyReader.read(key, clusterBy.getBucketByCount() + i); + + if (i == array.length - 1 && boosted) { + // Boost column + //noinspection RedundantCast: false alarm; the cast is necessary + array[i] = StringUtils.format("%016d", (long) val); + } else { + array[i] = (String) val; + } + } + + return new StringTuple(array); + } + + private static Pair, List> makeDimensionsAndAggregatorsForIngestion( + final RowSignature querySignature, + final ClusterBy queryClusterBy, + final List segmentSortOrder, + final ColumnMappings columnMappings, + final boolean isRollupQuery, + final Query query + ) + { + final List dimensions = new ArrayList<>(); + final List aggregators = new ArrayList<>(); + + // During ingestion, segment sort order is determined by the order of fields in the DimensionsSchema. We want + // this to match user intent as dictated by the declared segment sort order and CLUSTERED BY, so add things in + // that order. + + // Start with segmentSortOrder. + final Set outputColumnsInOrder = new LinkedHashSet<>(segmentSortOrder); + + // Then the query-level CLUSTERED BY. + // Note: this doesn't work when CLUSTERED BY specifies an expression that is not being selected. + // Such fields in CLUSTERED BY still control partitioning as expected, but do not affect sort order of rows + // within an individual segment. + for (final SortColumn clusterByColumn : queryClusterBy.getColumns()) { + if (clusterByColumn.descending()) { + throw new MSQException(new InsertCannotOrderByDescendingFault(clusterByColumn.columnName())); + } + + outputColumnsInOrder.addAll(columnMappings.getOutputColumnsForQueryColumn(clusterByColumn.columnName())); + } + + // Then all other columns. + outputColumnsInOrder.addAll(columnMappings.getOutputColumnNames()); + + Map outputColumnAggregatorFactories = new HashMap<>(); + + if (isRollupQuery) { + // Populate aggregators from the native query when doing an ingest in rollup mode. + for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) { + String outputColumn = Iterables.getOnlyElement(columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName())); + if (outputColumnAggregatorFactories.containsKey(outputColumn)) { + throw new ISE("There can only be one aggregator factory for column [%s].", outputColumn); + } else { + outputColumnAggregatorFactories.put( + outputColumn, + aggregatorFactory.withName(outputColumn).getCombiningFactory() + ); + } + } + } + + // Each column can be of either time, dimension, aggregator. For this method. we can ignore the time column. + // For non-complex columns, If the aggregator factory of the column is not available, we treat the column as + // a dimension. For complex columns, certains hacks are in place. + for (final String outputColumn : outputColumnsInOrder) { + final String queryColumn = columnMappings.getQueryColumnForOutputColumn(outputColumn); + final ColumnType type = + querySignature.getColumnType(queryColumn) + .orElseThrow(() -> new ISE("No type for column [%s]", outputColumn)); + + if (!outputColumn.equals(ColumnHolder.TIME_COLUMN_NAME)) { + + if (!type.is(ValueType.COMPLEX)) { + // non complex columns + populateDimensionsAndAggregators( + dimensions, + aggregators, + outputColumnAggregatorFactories, + outputColumn, + type + ); + } else { + // complex columns only + if (DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.containsKey(type.getComplexTypeName())) { + dimensions.add(DimensionSchemaUtils.createDimensionSchema(outputColumn, type)); + } else if (!isRollupQuery) { + aggregators.add(new PassthroughAggregatorFactory(outputColumn, type.getComplexTypeName())); + } else { + populateDimensionsAndAggregators( + dimensions, + aggregators, + outputColumnAggregatorFactories, + outputColumn, + type + ); + } + } + } + } + + return Pair.of(dimensions, aggregators); + } + + + /** + * If the output column is present in the outputColumnAggregatorFactories that means we already have the aggregator information for this column. + * else treat this column as a dimension. + * + * @param dimensions list is poulated if the output col is deemed to be a dimension + * @param aggregators list is populated with the aggregator if the output col is deemed to be a aggregation column. + * @param outputColumnAggregatorFactories output col -> AggregatorFactory map + * @param outputColumn column name + * @param type columnType + */ + private static void populateDimensionsAndAggregators( + List dimensions, + List aggregators, + Map outputColumnAggregatorFactories, + String outputColumn, + ColumnType type + ) + { + if (outputColumnAggregatorFactories.containsKey(outputColumn)) { + aggregators.add(outputColumnAggregatorFactories.get(outputColumn)); + } else { + dimensions.add(DimensionSchemaUtils.createDimensionSchema(outputColumn, type)); + } + } + + private static DateTime getBucketDateTime( + final ClusterByPartition partitionBoundary, + final Granularity segmentGranularity, + final RowKeyReader keyReader + ) + { + if (Granularities.ALL.equals(segmentGranularity)) { + return DateTimes.utc(0); + } else { + final RowKey startKey = partitionBoundary.getStart(); + final DateTime timestamp = + DateTimes.utc(MSQTasks.primaryTimestampFromObjectForInsert(keyReader.read(startKey, 0))); + + if (segmentGranularity.bucketStart(timestamp.getMillis()) != timestamp.getMillis()) { + // It's a bug in... something? if this happens. + throw new ISE( + "Received boundary value [%s] misaligned with segmentGranularity [%s]", + timestamp, + segmentGranularity + ); + } + + return timestamp; + } + } + + private static MSQStagesReport makeStageReport( + final QueryDefinition queryDef, + final Map stagePhaseMap, + final Map stageRuntimeMap, + final Map stageWorkerCountMap, + final Map stagePartitionCountMap + ) + { + return MSQStagesReport.create( + queryDef, + ImmutableMap.copyOf(stagePhaseMap), + copyOfStageRuntimesEndingAtCurrentTime(stageRuntimeMap), + stageWorkerCountMap, + stagePartitionCountMap + ); + } + + private static MSQResultsReport makeResultsTaskReport( + final QueryDefinition queryDef, + final Yielder resultsYielder, + final ColumnMappings columnMappings, + @Nullable final List sqlTypeNames + ) + { + final RowSignature querySignature = queryDef.getFinalStageDefinition().getSignature(); + final RowSignature.Builder mappedSignature = RowSignature.builder(); + + for (final ColumnMapping mapping : columnMappings.getMappings()) { + mappedSignature.add( + mapping.getOutputColumn(), + querySignature.getColumnType(mapping.getQueryColumn()).orElse(null) + ); + } + + return new MSQResultsReport(mappedSignature.build(), sqlTypeNames, resultsYielder); + } + + private static MSQStatusReport makeStatusReport( + final TaskState taskState, + @Nullable final MSQErrorReport errorReport, + final Queue errorReports, + @Nullable final DateTime queryStartTime, + final long queryDuration + ) + { + return new MSQStatusReport(taskState, errorReport, errorReports, queryStartTime, queryDuration); + } + + private static InputSpecSlicerFactory makeInputSpecSlicerFactory(final DataSegmentTimelineView timelineView) + { + return stagePartitionsMap -> new MapInputSpecSlicer( + ImmutableMap., InputSpecSlicer>builder() + .put(StageInputSpec.class, new StageInputSpecSlicer(stagePartitionsMap)) + .put(ExternalInputSpec.class, new ExternalInputSpecSlicer()) + .put(TableInputSpec.class, new TableInputSpecSlicer(timelineView)) + .build() + ); + } + + private static Map copyOfStageRuntimesEndingAtCurrentTime( + final Map stageRuntimesMap + ) + { + final Int2ObjectMap retVal = new Int2ObjectOpenHashMap<>(stageRuntimesMap.size()); + final DateTime now = DateTimes.nowUtc(); + + for (Map.Entry entry : stageRuntimesMap.entrySet()) { + final int stageNumber = entry.getKey(); + final Interval interval = entry.getValue(); + + retVal.put( + stageNumber, + interval.getEnd().equals(DateTimes.MAX) ? new Interval(interval.getStart(), now) : interval + ); + } + + return retVal; + } + + /** + * Method that determines whether an exception was raised due to the task lock for the controller task being + * preempted. Uses string comparison, because the relevant Overlord APIs do not have a more reliable way of + * discerning the cause of errors. + * + * Error strings are taken from {@link org.apache.druid.indexing.common.actions.TaskLocks} + * and {@link SegmentAllocateAction}. + */ + private static boolean isTaskLockPreemptedException(Exception e) + { + final String exceptionMsg = e.getMessage(); + final List validExceptionExcerpts = ImmutableList.of( + "are not covered by locks" /* From TaskLocks */, + "is preempted and no longer valid" /* From SegmentAllocateAction */ + ); + return validExceptionExcerpts.stream().anyMatch(exceptionMsg::contains); + } + + private static void logKernelStatus(final String queryId, final ControllerQueryKernel queryKernel) + { + if (log.isDebugEnabled()) { + log.debug( + "Query [%s] kernel state: %s", + queryId, + queryKernel.getActiveStages() + .stream() + .sorted(Comparator.comparing(id -> queryKernel.getStageDefinition(id).getStageNumber())) + .map(id -> StringUtils.format( + "%d:%d[%s:%s]>%s", + queryKernel.getStageDefinition(id).getStageNumber(), + queryKernel.getWorkerInputsForStage(id).workerCount(), + queryKernel.getStageDefinition(id).doesShuffle() ? "SHUFFLE" : "RETAIN", + queryKernel.getStagePhase(id), + queryKernel.doesStageHaveResultPartitions(id) + ? Iterators.size(queryKernel.getResultPartitionsForStage(id).iterator()) + : "?" + ) + ) + .collect(Collectors.joining("; ")) + ); + } + } + + /** + * Main controller logic for running a multi-stage query. + */ + private class RunQueryUntilDone + { + private final QueryDefinition queryDef; + private final InputSpecSlicerFactory inputSpecSlicerFactory; + private final Closer closer; + private final ControllerQueryKernel queryKernel; + + /** + * Set of stages that have got their partition boundaries sent out. + */ + private final Set stageResultPartitionBoundariesSent = new HashSet<>(); + + /** + * Return value of {@link MSQWorkerTaskLauncher#start()}. Set by {@link #startTaskLauncher()}. + */ + private ListenableFuture workerTaskLauncherFuture; + + /** + * Segments to generate. Populated prior to launching the final stage of a query with destination + * {@link DataSourceMSQDestination} (which originate from SQL INSERT or REPLACE). The final stage of such a query + * uses {@link SegmentGeneratorFrameProcessorFactory}, which requires a list of segment IDs to generate. + */ + private List segmentsToGenerate; + + public RunQueryUntilDone( + final QueryDefinition queryDef, + final InputSpecSlicerFactory inputSpecSlicerFactory, + final Closer closer + ) + { + this.queryDef = queryDef; + this.inputSpecSlicerFactory = inputSpecSlicerFactory; + this.closer = closer; + this.queryKernel = new ControllerQueryKernel(queryDef); + } + + /** + * Primary 'run' method. + */ + private Pair> run() throws IOException, InterruptedException + { + startTaskLauncher(); + + while (!queryKernel.isDone()) { + startStages(); + sendPartitionBoundaries(); + updateLiveReportMaps(); + cleanUpEffectivelyFinishedStages(); + runKernelCommands(); + } + + if (!queryKernel.isSuccess()) { + throwKernelExceptionIfNotUnknown(); + } + + cleanUpEffectivelyFinishedStages(); + return Pair.of(queryKernel, workerTaskLauncherFuture); + } + + /** + * Run at least one command from {@link #kernelManipulationQueue}, waiting for it if necessary. + */ + private void runKernelCommands() throws InterruptedException + { + if (!queryKernel.isDone()) { + // Run the next command, waiting for it if necessary. + Consumer command = kernelManipulationQueue.take(); + command.accept(queryKernel); + + // Run all pending commands after that one. Helps avoid deep queues. + // After draining the command queue, move on to the next iteration of the controller loop. + while ((command = kernelManipulationQueue.poll()) != null) { + command.accept(queryKernel); + } + } + } + + /** + * Start up the {@link MSQWorkerTaskLauncher}, such that later on it can be used to launch new tasks + * via {@link MSQWorkerTaskLauncher#launchTasksIfNeeded}. + */ + private void startTaskLauncher() + { + // Start tasks. + log.debug("Query [%s] starting task launcher.", queryDef.getQueryId()); + + workerTaskLauncherFuture = workerTaskLauncher.start(); + closer.register(() -> workerTaskLauncher.stop(true)); + + workerTaskLauncherFuture.addListener( + () -> + addToKernelManipulationQueue(queryKernel -> { + // Throw an exception in the main loop, if anything went wrong. + FutureUtils.getUncheckedImmediately(workerTaskLauncherFuture); + }), + Execs.directExecutor() + ); + } + + /** + * Start up any stages that are ready to start. + */ + private void startStages() throws IOException, InterruptedException + { + logKernelStatus(queryDef.getQueryId(), queryKernel); + final List newStageIds = queryKernel.createAndGetNewStageIds( + inputSpecSlicerFactory, + task.getQuerySpec().getAssignmentStrategy() + ); + + for (final StageId stageId : newStageIds) { + queryKernel.startStage(stageId); + + // Allocate segments, if this is the final stage of an ingestion. + if (MSQControllerTask.isIngestion(task.getQuerySpec()) + && stageId.getStageNumber() == queryDef.getFinalStageDefinition().getStageNumber()) { + // We need to find the shuffle details (like partition ranges) to generate segments. Generally this is + // going to correspond to the stage immediately prior to the final segment-generator stage. + int shuffleStageNumber = Iterables.getOnlyElement(queryDef.getFinalStageDefinition().getInputStageNumbers()); + + // The following logic assumes that output of all the stages without a shuffle retain the partition boundaries + // of the input to that stage. This may not always be the case. For example: GROUP BY queries without an + // ORDER BY clause. This works for QueryKit generated queries up until now, but it should be reworked as it + // might not always be the case. + while (!queryDef.getStageDefinition(shuffleStageNumber).doesShuffle()) { + shuffleStageNumber = + Iterables.getOnlyElement(queryDef.getStageDefinition(shuffleStageNumber).getInputStageNumbers()); + } + + final StageId shuffleStageId = new StageId(queryDef.getQueryId(), shuffleStageNumber); + final boolean isTimeBucketed = isTimeBucketedIngestion(task.getQuerySpec()); + final ClusterByPartitions partitionBoundaries = + queryKernel.getResultPartitionBoundariesForStage(shuffleStageId); + + // We require some data to be inserted in case it is partitioned by anything other than all and we are + // inserting everything into a single bucket. This can be handled more gracefully instead of throwing an exception + // Note: This can also be the case when we have limit queries but validation in Broker SQL layer prevents such + // queries + if (isTimeBucketed && partitionBoundaries.equals(ClusterByPartitions.oneUniversalPartition())) { + throw new MSQException(new InsertCannotBeEmptyFault(task.getDataSource())); + } else { + log.info("Query [%s] generating %d segments.", queryDef.getQueryId(), partitionBoundaries.size()); + } + + final boolean mayHaveMultiValuedClusterByFields = + !queryKernel.getStageDefinition(shuffleStageId).mustGatherResultKeyStatistics() + || queryKernel.hasStageCollectorEncounteredAnyMultiValueField(shuffleStageId); + + segmentsToGenerate = generateSegmentIdsWithShardSpecs( + (DataSourceMSQDestination) task.getQuerySpec().getDestination(), + queryKernel.getStageDefinition(shuffleStageId).getSignature(), + queryKernel.getStageDefinition(shuffleStageId).getShuffleSpec().get().getClusterBy(), + partitionBoundaries, + mayHaveMultiValuedClusterByFields + ); + } + + final int workerCount = queryKernel.getWorkerInputsForStage(stageId).workerCount(); + log.info( + "Query [%s] starting %d workers for stage %d.", + stageId.getQueryId(), + workerCount, + stageId.getStageNumber() + ); + + workerTaskLauncher.launchTasksIfNeeded(workerCount); + stageRuntimesForLiveReports.put(stageId.getStageNumber(), new Interval(DateTimes.nowUtc(), DateTimes.MAX)); + startWorkForStage(queryDef, queryKernel, stageId.getStageNumber(), segmentsToGenerate); + } + } + + /** + * Send partition boundaries to any stages that are ready to receive partition boundaries. + */ + private void sendPartitionBoundaries() + { + logKernelStatus(queryDef.getQueryId(), queryKernel); + for (final StageId stageId : queryKernel.getActiveStages()) { + + if (queryKernel.getStageDefinition(stageId).mustGatherResultKeyStatistics() + && queryKernel.doesStageHaveResultPartitions(stageId) + && stageResultPartitionBoundariesSent.add(stageId)) { + if (log.isDebugEnabled()) { + final ClusterByPartitions partitions = queryKernel.getResultPartitionBoundariesForStage(stageId); + log.debug( + "Query [%s] sending out partition boundaries for stage %d: %s", + stageId.getQueryId(), + stageId.getStageNumber(), + IntStream.range(0, partitions.size()) + .mapToObj(i -> StringUtils.format("%s:%s", i, partitions.get(i))) + .collect(Collectors.joining(", ")) + ); + } else { + log.info( + "Query [%s] sending out partition boundaries for stage %d.", + stageId.getQueryId(), + stageId.getStageNumber() + ); + } + + postResultPartitionBoundariesForStage( + queryDef, + stageId.getStageNumber(), + queryKernel.getResultPartitionBoundariesForStage(stageId), + queryKernel.getWorkerInputsForStage(stageId).workers() + ); + } + } + } + + /** + * Update the various maps used for live reports. + */ + private void updateLiveReportMaps() + { + logKernelStatus(queryDef.getQueryId(), queryKernel); + + // Live reports: update stage phases, worker counts, partition counts. + for (StageId stageId : queryKernel.getActiveStages()) { + final int stageNumber = stageId.getStageNumber(); + stagePhasesForLiveReports.put(stageNumber, queryKernel.getStagePhase(stageId)); + + if (queryKernel.doesStageHaveResultPartitions(stageId)) { + stagePartitionCountsForLiveReports.computeIfAbsent( + stageNumber, + k -> Iterators.size(queryKernel.getResultPartitionsForStage(stageId).iterator()) + ); + } + + stageWorkerCountsForLiveReports.putIfAbsent( + stageNumber, + queryKernel.getWorkerInputsForStage(stageId).workerCount() + ); + } + + // Live reports: update stage end times for any stages that just ended. + for (StageId stageId : queryKernel.getActiveStages()) { + if (ControllerStagePhase.isSuccessfulTerminalPhase(queryKernel.getStagePhase(stageId))) { + stageRuntimesForLiveReports.compute( + queryKernel.getStageDefinition(stageId).getStageNumber(), + (k, currentValue) -> { + if (currentValue.getEnd().equals(DateTimes.MAX)) { + return new Interval(currentValue.getStart(), DateTimes.nowUtc()); + } else { + return currentValue; + } + } + ); + } + } + } + + /** + * Issue cleanup commands to any stages that are effectivley finished, allowing them to delete their outputs. + */ + private void cleanUpEffectivelyFinishedStages() + { + for (final StageId stageId : queryKernel.getEffectivelyFinishedStageIds()) { + log.info("Query [%s] issuing cleanup order for stage %d.", queryDef.getQueryId(), stageId.getStageNumber()); + contactWorkersForStage( + (netClient, taskId, workerNumber) -> netClient.postCleanupStage(taskId, stageId), + queryKernel.getWorkerInputsForStage(stageId).workers() + ); + queryKernel.finishStage(stageId, true); + } + } + + /** + * Throw {@link MSQException} if the kernel method {@link ControllerQueryKernel#getFailureReasonForStage} + * has any failure reason other than {@link UnknownFault}. + */ + private void throwKernelExceptionIfNotUnknown() + { + for (final StageId stageId : queryKernel.getActiveStages()) { + if (queryKernel.getStagePhase(stageId) == ControllerStagePhase.FAILED) { + final MSQFault fault = queryKernel.getFailureReasonForStage(stageId); + + // Fall through (without throwing an exception) in case of UnknownFault; we may be able to generate + // a better exception later in query teardown. + if (!UnknownFault.CODE.equals(fault.getErrorCode())) { + throw new MSQException(fault); + } + } + } + } + } + + /** + * Interface used by {@link #contactWorkersForStage}. + */ + private interface TaskContactFn + { + ListenableFuture contactTask(WorkerClient client, String taskId, int workerNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java new file mode 100644 index 00000000000..1c3cc39987c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java @@ -0,0 +1,145 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; + +import javax.annotation.Nullable; +import java.io.IOException; + +/** + * Wrapper around any {@link WorkerClient} that converts exceptions into {@link MSQException} + * with {@link WorkerRpcFailedFault}. Useful so each implementation of WorkerClient does not need to do this on + * its own. + */ +public class ExceptionWrappingWorkerClient implements WorkerClient +{ + private final WorkerClient client; + + public ExceptionWrappingWorkerClient(final WorkerClient client) + { + this.client = Preconditions.checkNotNull(client, "client"); + } + + @Override + public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workOrder) + { + return wrap(workerTaskId, client, c -> c.postWorkOrder(workerTaskId, workOrder)); + } + + @Override + public ListenableFuture postResultPartitionBoundaries( + final String workerTaskId, + final StageId stageId, + final ClusterByPartitions partitionBoundaries + ) + { + return wrap(workerTaskId, client, c -> c.postResultPartitionBoundaries(workerTaskId, stageId, partitionBoundaries)); + } + + @Override + public ListenableFuture postCleanupStage(String workerTaskId, StageId stageId) + { + return wrap(workerTaskId, client, c -> c.postCleanupStage(workerTaskId, stageId)); + } + + @Override + public ListenableFuture postFinish(String workerTaskId) + { + return wrap(workerTaskId, client, c -> c.postFinish(workerTaskId)); + } + + @Override + public ListenableFuture getCounters(String workerTaskId) + { + return wrap(workerTaskId, client, c -> c.getCounters(workerTaskId)); + } + + @Override + public ListenableFuture fetchChannelData( + String workerTaskId, + StageId stageId, + int partitionNumber, + long offset, + ReadableByteChunksFrameChannel channel + ) + { + return wrap(workerTaskId, client, c -> c.fetchChannelData(workerTaskId, stageId, partitionNumber, offset, channel)); + } + + @Override + public void close() throws IOException + { + client.close(); + } + + private static ListenableFuture wrap( + final String workerTaskId, + final WorkerClient client, + final ClientFn clientFn + ) + { + final SettableFuture retVal = SettableFuture.create(); + final ListenableFuture clientFuture; + + try { + clientFuture = clientFn.apply(client); + } + catch (Exception e) { + throw new MSQException(e, new WorkerRpcFailedFault(workerTaskId)); + } + + Futures.addCallback( + clientFuture, + new FutureCallback() + { + @Override + public void onSuccess(@Nullable T result) + { + retVal.set(result); + } + + @Override + public void onFailure(Throwable t) + { + retVal.setException(new MSQException(t, new WorkerRpcFailedFault(workerTaskId))); + } + } + ); + + return retVal; + } + + private interface ClientFn + { + ListenableFuture apply(WorkerClient client); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java new file mode 100644 index 00000000000..ac5cf95d1dd --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -0,0 +1,61 @@ +/* + * 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.druid.msq.exec; + +public class Limits +{ + /** + * Maximum number of columns that can appear in a frame signature. + * + * Somewhat less than {@link WorkerMemoryParameters#STANDARD_FRAME_SIZE} divided by typical minimum column size: + * {@link org.apache.druid.frame.allocation.AppendableMemory#DEFAULT_INITIAL_ALLOCATION_SIZE}. + */ + public static final int MAX_FRAME_COLUMNS = 2000; + + /** + * Maximum number of workers that can be used in a stage, regardless of available memory. + */ + public static final int MAX_WORKERS = 1000; + + /** + * Maximum number of input files per worker + */ + public static final int MAX_INPUT_FILES_PER_WORKER = 10_000; + + /** + * Maximum number of parse exceptions with their stack traces a worker can send to the controller. + */ + public static final long MAX_VERBOSE_PARSE_EXCEPTIONS = 5; + + /** + * Maximum number of warnings with their stack traces a worker can send to the controller. + */ + public static final long MAX_VERBOSE_WARNINGS = 10; + + /** + * Maximum number of input bytes per worker in case number of tasks is determined automatically. + */ + public static final long MAX_INPUT_BYTES_PER_WORKER = 10 * 1024 * 1024 * 1024L; + + /** + * Maximum size of the kernel manipulation queue in {@link org.apache.druid.msq.indexing.MSQControllerTask}. + */ + public static final int MAX_KERNEL_MANIPULATION_QUEUE_SIZE = 100_000; +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java new file mode 100644 index 00000000000..fa1371f132e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java @@ -0,0 +1,216 @@ +/* + * 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.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Injector; +import com.google.inject.Key; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.guice.MultiStageQuery; +import org.apache.druid.msq.indexing.error.CanceledFault; +import org.apache.druid.msq.indexing.error.DurableStorageConfigurationFault; +import org.apache.druid.msq.indexing.error.InsertTimeNullFault; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.error.WorkerFailedFault; +import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; +import org.apache.druid.msq.statistics.KeyCollectorFactory; +import org.apache.druid.msq.statistics.KeyCollectorSnapshot; +import org.apache.druid.msq.statistics.KeyCollectorSnapshotDeserializerModule; +import org.apache.druid.msq.statistics.KeyCollectors; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.server.DruidNode; +import org.apache.druid.storage.StorageConnector; + +import javax.annotation.Nullable; +import java.util.UUID; + +public class MSQTasks +{ + /** + * Message used by {@link #makeErrorReport} when no other message is known. + */ + static final String GENERIC_QUERY_FAILED_MESSAGE = "Query failed"; + + private static final String TASK_ID_PREFIX = "query-"; + + /** + * Returns a controller task ID given a SQL query id. + */ + public static String controllerTaskId(@Nullable final String queryId) + { + return TASK_ID_PREFIX + (queryId == null ? UUID.randomUUID().toString() : queryId); + } + + /** + * Returns a worker task ID given a SQL query id. + */ + public static String workerTaskId(final String controllerTaskId, final int workerNumber) + { + return StringUtils.format("%s-worker%d", controllerTaskId, workerNumber); + } + + /** + * If "Object" is a Long, returns it. Otherwise, throws an appropriate exception assuming this operation is + * being done to read the primary timestamp (__time) as part of an INSERT. + */ + public static long primaryTimestampFromObjectForInsert(final Object timestamp) + { + if (timestamp instanceof Long) { + return (long) timestamp; + } else if (timestamp == null) { + throw new MSQException(InsertTimeNullFault.INSTANCE); + } else { + // Normally we expect the SQL layer to validate that __time for INSERT is a TIMESTAMP type, which would + // be a long at execution time. So a nice user-friendly message isn't needed here: it would only happen + // if the SQL layer is bypassed. Nice, friendly users wouldn't do that :) + final UnknownFault fault = + UnknownFault.forMessage(StringUtils.format("Incorrect type for [%s]", ColumnHolder.TIME_COLUMN_NAME)); + throw new MSQException(fault); + } + } + + /** + * Returns a decorated copy of an ObjectMapper that knows how to deserialize the appropriate kind of + * {@link KeyCollectorSnapshot}. + */ + static ObjectMapper decorateObjectMapperForKeyCollectorSnapshot( + final ObjectMapper mapper, + final ClusterBy clusterBy, + final boolean aggregate + ) + { + final KeyCollectorFactory keyCollectorFactory = + KeyCollectors.makeStandardFactory(clusterBy, aggregate); + + final ObjectMapper mapperCopy = mapper.copy(); + mapperCopy.registerModule(new KeyCollectorSnapshotDeserializerModule(keyCollectorFactory)); + return mapperCopy; + } + + /** + * Returns the host:port from a {@link DruidNode}. Convenience method to make it easier to construct + * {@link MSQErrorReport} instances. + */ + @Nullable + static String getHostFromSelfNode(@Nullable final DruidNode selfNode) + { + return selfNode != null ? selfNode.getHostAndPortToUse() : null; + } + + static StorageConnector makeStorageConnector(final Injector injector) + { + try { + return injector.getInstance(Key.get(StorageConnector.class, MultiStageQuery.class)); + } + catch (Exception e) { + throw new MSQException(new DurableStorageConfigurationFault(e.toString())); + } + } + + /** + * Builds an error report from a possible controller error report and a possible worker error report. Both may be + * null, in which case this function will return a report with {@link UnknownFault}. + * + * We only include a single {@link MSQErrorReport} in the task report, because it's important that a query have + * a single {@link MSQFault} explaining why it failed. To aid debugging + * in cases where we choose the controller error over the worker error, we'll log the worker error too, even though + * it doesn't appear in the report. + * + * Logic: we prefer the controller exception unless it's {@link WorkerFailedFault}, {@link WorkerRpcFailedFault}, + * or {@link CanceledFault}. In these cases we prefer the worker error report. This ensures we get the best, most + * useful exception even when the controller cancels worker tasks after a failure. (As tasks are canceled one by + * one, worker -> worker and controller -> worker RPCs to the canceled tasks will fail. We want to ignore these + * failed RPCs and get to the "true" error that started it all.) + */ + static MSQErrorReport makeErrorReport( + final String controllerTaskId, + final String controllerHost, + @Nullable MSQErrorReport controllerErrorReport, + @Nullable MSQErrorReport workerErrorReport + ) + { + if (controllerErrorReport == null && workerErrorReport == null) { + // Something went wrong, but we have no idea what. + return MSQErrorReport.fromFault( + controllerTaskId, + controllerHost, + null, + UnknownFault.forMessage(GENERIC_QUERY_FAILED_MESSAGE) + ); + } else if (controllerErrorReport == null) { + // workerErrorReport is nonnull. + return workerErrorReport; + } else { + // controllerErrorReport is nonnull. + + // Pick the "best" error if both are set. See the javadoc for the logic we use. In these situations, we + // expect the caller to also log the other one. (There is no logging in _this_ method, because it's a helper + // function, and it's best if helper functions run quietly.) + if (workerErrorReport != null && (controllerErrorReport.getFault() instanceof WorkerFailedFault + || controllerErrorReport.getFault() instanceof WorkerRpcFailedFault + || controllerErrorReport.getFault() instanceof CanceledFault)) { + return workerErrorReport; + } else { + return controllerErrorReport; + } + } + } + + /** + * Returns a string form of a {@link MSQErrorReport} suitable for logging. + */ + static String errorReportToLogMessage(final MSQErrorReport errorReport) + { + final StringBuilder logMessage = new StringBuilder("Work failed"); + + if (errorReport.getStageNumber() != null) { + logMessage.append("; stage ").append(errorReport.getStageNumber()); + } + + logMessage.append("; task ").append(errorReport.getTaskId()); + + if (errorReport.getHost() != null) { + logMessage.append("; host ").append(errorReport.getHost()); + } + + logMessage.append(": ").append(errorReport.getFault().getCodeWithMessage()); + + if (errorReport.getExceptionStackTrace() != null) { + if (errorReport.getFault() instanceof UnknownFault) { + // Log full stack trace for unknown faults. + logMessage.append('\n').append(errorReport.getExceptionStackTrace()); + } else { + // Log first line only (error class, message) for known faults, to avoid polluting logs. + final String stackTrace = errorReport.getExceptionStackTrace(); + final int firstNewLine = stackTrace.indexOf('\n'); + + logMessage.append(" (") + .append(firstNewLine > 0 ? stackTrace.substring(0, firstNewLine) : stackTrace) + .append(")"); + } + } + + return logMessage.toString(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryValidator.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryValidator.java new file mode 100644 index 00000000000..87980130d23 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryValidator.java @@ -0,0 +1,77 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.math.IntMath; +import com.google.common.primitives.Ints; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.TooManyColumnsFault; +import org.apache.druid.msq.indexing.error.TooManyInputFilesFault; +import org.apache.druid.msq.indexing.error.TooManyWorkersFault; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.WorkOrder; + +import java.math.RoundingMode; + +public class QueryValidator +{ + /** + * Validate that a {@link QueryDefinition} falls within the {@link Limits#MAX_FRAME_COLUMNS} and + * {@link Limits#MAX_WORKERS} limits. + */ + public static void validateQueryDef(final QueryDefinition queryDef) + { + for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { + final int numColumns = stageDef.getSignature().size(); + + if (numColumns > Limits.MAX_FRAME_COLUMNS) { + throw new MSQException(new TooManyColumnsFault(numColumns, Limits.MAX_FRAME_COLUMNS)); + } + + final int numWorkers = stageDef.getMaxWorkerCount(); + if (numWorkers > Limits.MAX_WORKERS) { + throw new MSQException(new TooManyWorkersFault(numWorkers, Limits.MAX_WORKERS)); + } else if (numWorkers <= 0) { + throw new ISE("Number of workers must be greater than 0"); + } + } + } + + /** + * Validate that a {@link WorkOrder} falls within the {@link Limits#MAX_INPUT_FILES_PER_WORKER} limit. + */ + public static void validateWorkOrder(final WorkOrder order) + { + final int numInputFiles = Ints.checkedCast(order.getInputs().stream().mapToLong(InputSlice::fileCount).sum()); + + if (numInputFiles > Limits.MAX_INPUT_FILES_PER_WORKER) { + throw new MSQException( + new TooManyInputFilesFault( + numInputFiles, + Limits.MAX_INPUT_FILES_PER_WORKER, + IntMath.divide(numInputFiles, Limits.MAX_INPUT_FILES_PER_WORKER, RoundingMode.CEILING) + ) + ); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java new file mode 100644 index 00000000000..7c4dc2bb09a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java @@ -0,0 +1,115 @@ +/* + * 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.druid.msq.exec; + +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.msq.querykit.LazyResourceHolder; +import org.apache.druid.msq.rpc.CoordinatorServiceClient; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.utils.CloseableUtils; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.ExecutionException; + +/** + * Production implementation of {@link DataSegmentProvider} using Coordinator APIs. + */ +public class TaskDataSegmentProvider implements DataSegmentProvider +{ + private final CoordinatorServiceClient coordinatorClient; + private final SegmentCacheManager segmentCacheManager; + private final IndexIO indexIO; + + public TaskDataSegmentProvider( + CoordinatorServiceClient coordinatorClient, + SegmentCacheManager segmentCacheManager, + IndexIO indexIO + ) + { + this.coordinatorClient = coordinatorClient; + this.segmentCacheManager = segmentCacheManager; + this.indexIO = indexIO; + } + + @Override + public LazyResourceHolder fetchSegment( + final SegmentId segmentId, + final ChannelCounters channelCounters + ) + { + try { + // Use LazyResourceHolder so Coordinator call and segment downloads happen in processing threads, + // rather than the main thread. + return new LazyResourceHolder<>( + () -> { + final DataSegment dataSegment; + try { + dataSegment = FutureUtils.get( + coordinatorClient.fetchUsedSegment( + segmentId.getDataSource(), + segmentId.toString() + ), + true + ); + } + catch (InterruptedException | ExecutionException e) { + throw new RE(e, "Failed to fetch segment details from Coordinator for [%s]", segmentId); + } + + final Closer closer = Closer.create(); + try { + final File segmentDir = segmentCacheManager.getSegmentFiles(dataSegment); + closer.register(() -> FileUtils.deleteDirectory(segmentDir)); + + final QueryableIndex index = indexIO.loadIndex(segmentDir); + final int numRows = index.getNumRows(); + final long size = dataSegment.getSize(); + closer.register(() -> channelCounters.addFile(numRows, size)); + closer.register(index); + return Pair.of(new QueryableIndexSegment(index, dataSegment.getId()), closer); + } + catch (IOException | SegmentLoadingException e) { + throw CloseableUtils.closeInCatch( + new RE(e, "Failed to download segment [%s]", segmentId), + closer + ); + } + } + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java new file mode 100644 index 00000000000..f069c91e145 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -0,0 +1,108 @@ +/* + * 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.druid.msq.exec; + +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.indexing.MSQWorkerTask; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.InputStream; + +public interface Worker +{ + /** + * Unique ID for this worker. + */ + String id(); + + /** + * The task which this worker runs. + */ + MSQWorkerTask task(); + + /** + * Runs the worker in the current thread. Surrounding classes provide + * the execution thread. + */ + TaskStatus run() throws Exception; + + /** + * Terminate the worker upon a cancellation request. + */ + void stopGracefully(); + + /** + * Report that the controller has failed. The worker must cease work immediately. Cleanup then exit. + * Do not send final messages to the controller: there will be no one home at the other end. + */ + void controllerFailed(); + + // Controller-to-worker, and worker-to-worker messages + + /** + * Called when the worker chat handler receives a request for a work order. Accepts the work order and schedules it for + * execution + */ + void postWorkOrder(WorkOrder workOrder); + + /** + * Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber + * and queryId + */ + boolean postResultPartitionBoundaries( + ClusterByPartitions stagePartitionBoundaries, + String queryId, + int stageNumber + ); + + /** + * Returns an InputStream of the worker output for a particular queryId, stageNumber and partitionNumber. + * Offset indicates the number of bytes to skip the channel data, and is used to prevent re-reading the same data + * during retry in case of a connection error + * + * Returns a null if the workerOutput for a particular queryId, stageNumber, and partitionNumber is not found. + * + * @throws IOException when the worker output is found but there is an error while reading it. + */ + @Nullable + InputStream readChannel(String queryId, int stageNumber, int partitionNumber, long offset) throws IOException; + + /** + * Returns the snapshot of the worker counters + */ + CounterSnapshotsTree getCounters(); + + /** + * Called when the worker receives a POST request to clean up the stage with stageId, and is no longer required. + * This marks the stage as FINISHED in its stage kernel, cleans up the worker output for the stage and optionally + * frees any resources held on by the worker for the particular stage + */ + void postCleanupStage(StageId stageId); + + /** + * Called when the work required for the query has been finished + */ + void postFinish(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java new file mode 100644 index 00000000000..72a0a81604d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java @@ -0,0 +1,84 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; + +import java.io.IOException; + +/** + * Client for multi-stage query workers. Used by the controller task. + */ +public interface WorkerClient extends AutoCloseable +{ + /** + * Worker's client method to add a {@link WorkOrder} to the worker to work on + */ + ListenableFuture postWorkOrder(String workerId, WorkOrder workOrder); + + /** + * Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the + * controller after collating the result statistics from all the workers processing the query + */ + ListenableFuture postResultPartitionBoundaries( + String workerTaskId, + StageId stageId, + ClusterByPartitions partitionBoundaries + ); + + /** + * Worker's client method to inform that the work has been done, and it can initiate cleanup and shutdown + */ + ListenableFuture postFinish(String workerId); + + /** + * Fetches all the counters gathered by that worker + */ + ListenableFuture getCounters(String workerId); + + /** + * Worker's client method that informs it that the results and resources for the given stage are no longer required + * and that they can be cleaned up + */ + ListenableFuture postCleanupStage(String workerTaskId, StageId stageId); + + /** + * Fetch some data from a worker and add it to the provided channel. The exact amount of data is determined + * by the server. + * + * Returns a future that resolves to true (no more data left), false (there is more data left), or exception (some + * kind of unrecoverable exception). + */ + ListenableFuture fetchChannelData( + String workerTaskId, + StageId stageId, + int partitionNumber, + long offset, + ReadableByteChunksFrameChannel channel + ); + + @Override + void close() throws IOException; +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java new file mode 100644 index 00000000000..6b4a387b8da --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java @@ -0,0 +1,76 @@ +/* + * 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.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Injector; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.server.DruidNode; + +import java.io.File; + +/** + * Context used by multi-stage query workers. + * + * Useful because it allows test fixtures to provide their own implementations. + */ +public interface WorkerContext +{ + ObjectMapper jsonMapper(); + + // Using an Injector directly because tasks do not have a way to provide their own Guice modules. + Injector injector(); + + /** + * Callback from the worker implementation to "register" the worker. Used in + * the indexer to set up the task chat services. + */ + void registerWorker(Worker worker, Closer closer); + + /** + * Creates and fetches the controller client for the provided controller ID. + */ + ControllerClient makeControllerClient(String controllerId); + + /** + * Creates and fetches a {@link WorkerClient}. It is independent of the workerId because the workerId is passed + * in to every method of the client. + */ + WorkerClient makeWorkerClient(); + + /** + * Fetch a directory for temporary outputs + */ + File tempDir(); + + FrameContext frameContext(QueryDefinition queryDef, int stageNumber); + + int threadCount(); + + /** + * Fetch node info about self + */ + DruidNode selfNode(); + + Bouncer processorBouncer(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java new file mode 100644 index 00000000000..275965abbe4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -0,0 +1,1249 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.SettableFuture; +import it.unimi.dsi.fastutil.bytes.ByteArrays; +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableNilFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.frame.file.FrameFileWriter; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.frame.processor.FileOutputChannelFactory; +import org.apache.druid.frame.processor.FrameChannelMuxer; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessorExecutor; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.SuperSorter; +import org.apache.druid.frame.processor.SuperSorterProgressTracker; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.indexing.CountingOutputChannelFactory; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.indexing.InputChannelsImpl; +import org.apache.druid.msq.indexing.KeyStatisticsCollectionProcessor; +import org.apache.druid.msq.indexing.MSQWorkerTask; +import org.apache.druid.msq.indexing.error.CanceledFault; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; +import org.apache.druid.msq.indexing.error.MSQWarningReportPublisher; +import org.apache.druid.msq.indexing.error.MSQWarningReportSimplePublisher; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.InputSlices; +import org.apache.druid.msq.input.MapInputSliceReader; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.input.NilInputSliceReader; +import org.apache.druid.msq.input.external.ExternalInputSlice; +import org.apache.druid.msq.input.external.ExternalInputSliceReader; +import org.apache.druid.msq.input.stage.InputChannels; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.input.stage.StageInputSliceReader; +import org.apache.druid.msq.input.table.SegmentsInputSlice; +import org.apache.druid.msq.input.table.SegmentsInputSliceReader; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.FrameProcessorFactory; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.kernel.worker.WorkerStageKernel; +import org.apache.druid.msq.kernel.worker.WorkerStagePhase; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory; +import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory; +import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.util.DecoratedExecutorService; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.PrioritizedCallable; +import org.apache.druid.query.PrioritizedRunnable; +import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.server.DruidNode; + +import javax.annotation.Nullable; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +/** + * Interface for a worker of a multi-stage query. + */ +public class WorkerImpl implements Worker +{ + private static final Logger log = new Logger(WorkerImpl.class); + + private final MSQWorkerTask task; + private final WorkerContext context; + private final DruidNode selfDruidNode; + private final Bouncer processorBouncer; + + private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>(); + private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); + private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); + private final boolean durableStageStorageEnabled; + + /** + * Set once in {@link #runTask} and never reassigned. + */ + private volatile ControllerClient controllerClient; + + /** + * Set once in {@link #runTask} and never reassigned. Used by processing threads so we can contact other workers + * during a shuffle. + */ + private volatile WorkerClient workerClient; + + /** + * Set to false by {@link #controllerFailed()} as a way of enticing the {@link #runTask} method to exit promptly. + */ + private volatile boolean controllerAlive = true; + + public WorkerImpl(MSQWorkerTask task, WorkerContext context) + { + this.task = task; + this.context = context; + this.selfDruidNode = context.selfNode(); + this.processorBouncer = context.processorBouncer(); + this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(task.getContext()); + } + + @Override + public String id() + { + return task.getId(); + } + + @Override + public MSQWorkerTask task() + { + return task; + } + + @Override + public TaskStatus run() throws Exception + { + try (final Closer closer = Closer.create()) { + Optional maybeErrorReport; + + try { + maybeErrorReport = runTask(closer); + } + catch (Throwable e) { + maybeErrorReport = Optional.of( + MSQErrorReport.fromException(id(), MSQTasks.getHostFromSelfNode(selfDruidNode), null, e) + ); + } + + if (maybeErrorReport.isPresent()) { + final MSQErrorReport errorReport = maybeErrorReport.get(); + final String errorLogMessage = MSQTasks.errorReportToLogMessage(errorReport); + log.warn(errorLogMessage); + + closer.register(() -> { + if (controllerAlive && controllerClient != null && selfDruidNode != null) { + controllerClient.postWorkerError(id(), errorReport); + } + }); + + return TaskStatus.failure(id(), errorReport.getFault().getCodeWithMessage()); + } else { + return TaskStatus.success(id()); + } + } + } + + /** + * Runs worker logic. Returns an empty Optional on success. On failure, returns an error report for errors that + * happened in other threads; throws exceptions for errors that happened in the main worker loop. + */ + public Optional runTask(final Closer closer) throws Exception + { + this.controllerClient = context.makeControllerClient(task.getControllerTaskId()); + closer.register(controllerClient::close); + context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized + this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient()); + closer.register(workerClient::close); + + final KernelHolder kernelHolder = new KernelHolder(); + final String cancellationId = id(); + + final FrameProcessorExecutor workerExec = new FrameProcessorExecutor(makeProcessingPool()); + + // Delete all the stage outputs + closer.register(() -> { + for (final StageId stageId : stageOutputs.keySet()) { + cleanStageOutput(stageId); + } + }); + + // Close stage output processors and running futures (if present) + closer.register(() -> { + try { + workerExec.cancel(cancellationId); + } + catch (InterruptedException e) { + // Strange that cancellation would itself be interrupted. Throw an exception, since this is unexpected. + throw new RuntimeException(e); + } + }); + + final MSQWarningReportPublisher msqWarningReportPublisher = new MSQWarningReportLimiterPublisher( + new MSQWarningReportSimplePublisher( + id(), + controllerClient, + id(), + MSQTasks.getHostFromSelfNode(selfDruidNode) + ) + ); + + closer.register(msqWarningReportPublisher); + + final Map> partitionBoundariesFutureMap = new HashMap<>(); + + final Map stageFrameContexts = new HashMap<>(); + + while (!kernelHolder.isDone()) { + boolean didSomething = false; + + for (final WorkerStageKernel kernel : kernelHolder.getStageKernelMap().values()) { + final StageDefinition stageDefinition = kernel.getStageDefinition(); + + if (kernel.getPhase() == WorkerStagePhase.NEW) { + log.debug("New work order: %s", context.jsonMapper().writeValueAsString(kernel.getWorkOrder())); + + // Create separate inputChannelFactory per stage, because the list of tasks can grow between stages, and + // so we need to avoid the memoization in baseInputChannelFactory. + final InputChannelFactory inputChannelFactory = makeBaseInputChannelFactory(closer); + + // Compute memory parameters for all stages, even ones that haven't been assigned yet, so we can fail-fast + // if some won't work. (We expect that all stages will get assigned to the same pool of workers.) + for (final StageDefinition stageDef : kernel.getWorkOrder().getQueryDefinition().getStageDefinitions()) { + stageFrameContexts.computeIfAbsent( + stageDef.getId(), + stageId -> context.frameContext( + kernel.getWorkOrder().getQueryDefinition(), + stageId.getStageNumber() + ) + ); + } + + // Start working on this stage immediately. + kernel.startReading(); + final SettableFuture partitionBoundariesFuture = + startWorkOrder( + kernel, + inputChannelFactory, + stageCounters.computeIfAbsent(stageDefinition.getId(), ignored -> new CounterTracker()), + workerExec, + cancellationId, + context.threadCount(), + stageFrameContexts.get(stageDefinition.getId()), + msqWarningReportPublisher + ); + + if (partitionBoundariesFuture != null) { + if (partitionBoundariesFutureMap.put(stageDefinition.getId(), partitionBoundariesFuture) != null) { + throw new ISE("Work order collision for stage [%s]", stageDefinition.getId()); + } + } + + didSomething = true; + logKernelStatus(kernelHolder.getStageKernelMap().values()); + } + + if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { + if (controllerAlive) { + controllerClient.postKeyStatistics( + stageDefinition.getId(), + kernel.getWorkOrder().getWorkerNumber(), + kernel.getResultKeyStatisticsSnapshot() + ); + } + kernel.startPreshuffleWaitingForResultPartitionBoundaries(); + + didSomething = true; + logKernelStatus(kernelHolder.getStageKernelMap().values()); + } + + logKernelStatus(kernelHolder.getStageKernelMap().values()); + if (kernel.getPhase() == WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES + && kernel.hasResultPartitionBoundaries()) { + partitionBoundariesFutureMap.get(stageDefinition.getId()).set(kernel.getResultPartitionBoundaries()); + kernel.startPreshuffleWritingOutput(); + + didSomething = true; + logKernelStatus(kernelHolder.getStageKernelMap().values()); + } + + if (kernel.getPhase() == WorkerStagePhase.RESULTS_READY + && kernel.addPostedResultsComplete(Pair.of(stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber()))) { + if (controllerAlive) { + controllerClient.postResultsComplete( + stageDefinition.getId(), + kernel.getWorkOrder().getWorkerNumber(), + kernel.getResultObject() + ); + } + } + + if (kernel.getPhase() == WorkerStagePhase.FAILED) { + // Better than throwing an exception, because we can include the stage number. + return Optional.of( + MSQErrorReport.fromException( + id(), + MSQTasks.getHostFromSelfNode(selfDruidNode), + stageDefinition.getId().getStageNumber(), + kernel.getException() + ) + ); + } + } + + if (!didSomething && !kernelHolder.isDone()) { + Consumer nextCommand; + + do { + postCountersToController(); + } while ((nextCommand = kernelManipulationQueue.poll(5, TimeUnit.SECONDS)) == null); + + nextCommand.accept(kernelHolder); + logKernelStatus(kernelHolder.getStageKernelMap().values()); + } + } + + // Empty means success. + return Optional.empty(); + } + + @Override + public void stopGracefully() + { + kernelManipulationQueue.add( + kernel -> { + // stopGracefully() is called when the containing process is terminated, or when the task is canceled. + throw new MSQException(CanceledFault.INSTANCE); + } + ); + } + + @Override + public void controllerFailed() + { + controllerAlive = false; + stopGracefully(); + } + + @Override + public InputStream readChannel( + final String queryId, + final int stageNumber, + final int partitionNumber, + final long offset + ) throws IOException + { + final StageId stageId = new StageId(queryId, stageNumber); + final StagePartition stagePartition = new StagePartition(stageId, partitionNumber); + final ConcurrentHashMap partitionOutputsForStage = stageOutputs.get(stageId); + + if (partitionOutputsForStage == null) { + return null; + } + final ReadableFrameChannel channel = partitionOutputsForStage.get(partitionNumber); + + if (channel == null) { + return null; + } + + if (channel instanceof ReadableNilFrameChannel) { + // Build an empty frame file. + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + FrameFileWriter.open(Channels.newChannel(baos), null).close(); + + final ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray()); + + //noinspection ResultOfMethodCallIgnored: OK to ignore since "skip" always works for ByteArrayInputStream. + in.skip(offset); + + return in; + } else if (channel instanceof ReadableFileFrameChannel) { + // Close frameFile once we've returned an input stream: no need to retain a reference to the mmap after that, + // since we aren't using it. + try (final FrameFile frameFile = ((ReadableFileFrameChannel) channel).newFrameFileReference()) { + final RandomAccessFile randomAccessFile = new RandomAccessFile(frameFile.file(), "r"); + + if (offset >= randomAccessFile.length()) { + randomAccessFile.close(); + return new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY); + } else { + randomAccessFile.seek(offset); + return Channels.newInputStream(randomAccessFile.getChannel()); + } + } + } else { + String errorMsg = StringUtils.format( + "Returned server error to client because channel for [%s] is not nil or file-based (class = %s)", + stagePartition, + channel.getClass().getName() + ); + log.error(StringUtils.encodeForFormat(errorMsg)); + + throw new IOException(errorMsg); + } + } + + @Override + public void postWorkOrder(final WorkOrder workOrder) + { + if (task.getWorkerNumber() != workOrder.getWorkerNumber()) { + throw new ISE("Worker number mismatch: expected [%d]", task.getWorkerNumber()); + } + + kernelManipulationQueue.add( + kernelHolder -> + kernelHolder.getStageKernelMap().computeIfAbsent( + workOrder.getStageDefinition().getId(), + ignored -> WorkerStageKernel.create(workOrder) + ) + ); + } + + @Override + public boolean postResultPartitionBoundaries( + final ClusterByPartitions stagePartitionBoundaries, + final String queryId, + final int stageNumber + ) + { + final StageId stageId = new StageId(queryId, stageNumber); + + kernelManipulationQueue.add( + kernelHolder -> { + final WorkerStageKernel stageKernel = kernelHolder.getStageKernelMap().get(stageId); + + // Ignore the update if we don't have a kernel for this stage. + if (stageKernel != null) { + stageKernel.setResultPartitionBoundaries(stagePartitionBoundaries); + } else { + log.warn("Ignored result partition boundaries call for unknown stage [%s]", stageId); + } + } + ); + return true; + } + + @Override + public void postCleanupStage(final StageId stageId) + { + log.info("Cleanup order for stage: [%s] received", stageId); + kernelManipulationQueue.add( + holder -> { + cleanStageOutput(stageId); + // Mark the stage as FINISHED + holder.getStageKernelMap().get(stageId).setStageFinished(); + } + ); + } + + @Override + public void postFinish() + { + kernelManipulationQueue.add(KernelHolder::setDone); + } + + @Override + public CounterSnapshotsTree getCounters() + { + final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); + + for (final Map.Entry entry : stageCounters.entrySet()) { + retVal.put(entry.getKey().getStageNumber(), task().getWorkerNumber(), entry.getValue().snapshot()); + } + + return retVal; + } + + private InputChannelFactory makeBaseInputChannelFactory(final Closer closer) + { + final Supplier> workerTaskList = Suppliers.memoize( + () -> { + try { + return controllerClient.getTaskList(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + )::get; + + if (durableStageStorageEnabled) { + return DurableStorageInputChannelFactory.createStandardImplementation( + task.getControllerTaskId(), + workerTaskList, + MSQTasks.makeStorageConnector(context.injector()), + closer + ); + } else { + return new WorkerOrLocalInputChannelFactory(workerTaskList); + } + } + + private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext frameContext, final int stageNumber) + { + // Use the standard frame size, since we assume this size when computing how much is needed to merge output + // files from different workers. + final int frameSize = frameContext.memoryParameters().getStandardFrameSize(); + + if (durableStageStorageEnabled) { + return DurableStorageOutputChannelFactory.createStandardImplementation( + task.getControllerTaskId(), + id(), + stageNumber, + frameSize, + MSQTasks.makeStorageConnector(context.injector()) + ); + } else { + final File fileChannelDirectory = + new File(context.tempDir(), StringUtils.format("output_stage_%06d", stageNumber)); + + return new FileOutputChannelFactory(fileChannelDirectory, frameSize); + } + } + + /** + * Decorates the server-wide {@link QueryProcessingPool} such that any Callables and Runnables, not just + * {@link PrioritizedCallable} and {@link PrioritizedRunnable}, may be added to it. + * + * In production, the underlying {@link QueryProcessingPool} pool is set up by + * {@link org.apache.druid.guice.DruidProcessingModule}. + */ + private ListeningExecutorService makeProcessingPool() + { + final QueryProcessingPool queryProcessingPool = context.injector().getInstance(QueryProcessingPool.class); + final int priority = 0; + + return new DecoratedExecutorService( + queryProcessingPool, + new DecoratedExecutorService.Decorator() + { + @Override + public Callable decorateCallable(Callable callable) + { + return new PrioritizedCallable() + { + @Override + public int getPriority() + { + return priority; + } + + @Override + public T call() throws Exception + { + return callable.call(); + } + }; + } + + @Override + public Runnable decorateRunnable(Runnable runnable) + { + return new PrioritizedRunnable() + { + @Override + public int getPriority() + { + return priority; + } + + @Override + public void run() + { + runnable.run(); + } + }; + } + } + ); + } + + /** + * Posts all counters for this worker to the controller. + */ + private void postCountersToController() throws IOException + { + final CounterSnapshotsTree snapshotsTree = getCounters(); + + if (controllerAlive && !snapshotsTree.isEmpty()) { + controllerClient.postCounters(snapshotsTree); + } + } + + /** + * Cleans up the stage outputs corresponding to the provided stage id. It essentially calls {@code doneReading()} on + * the readable channels corresponding to all the partitions for that stage, and removes it from the {@code stageOutputs} + * map + */ + private void cleanStageOutput(final StageId stageId) + { + // This code is thread-safe because remove() on ConcurrentHashMap will remove and return the removed channel only for + // one thread. For the other threads it will return null, therefore we will call doneReading for a channel only once + final ConcurrentHashMap partitionOutputsForStage = stageOutputs.remove(stageId); + // Check for null, this can be the case if this method is called simultaneously from multiple threads. + if (partitionOutputsForStage == null) { + return; + } + for (final int partition : partitionOutputsForStage.keySet()) { + final ReadableFrameChannel output = partitionOutputsForStage.remove(partition); + if (output == null) { + continue; + } + output.close(); + + // One caveat with this approach is that in case of a worker crash, while the MM/Indexer systems will delete their + // temp directories where intermediate results were stored, it won't be the case for the external storage. + // Therefore, the logic for cleaning the stage output in case of a worker/machine crash has to be external. + // We currently take care of this in the controller. + if (durableStageStorageEnabled) { + final String fileName = DurableStorageOutputChannelFactory.getPartitionFileName( + task.getControllerTaskId(), + task.getId(), + stageId.getStageNumber(), + partition + ); + try { + MSQTasks.makeStorageConnector(context.injector()).deleteFile(fileName); + } + catch (Exception e) { + // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup + log.warn(e, "Error while cleaning up temporary files at path " + fileName); + } + } + } + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + @Nullable + private SettableFuture startWorkOrder( + final WorkerStageKernel kernel, + final InputChannelFactory inputChannelFactory, + final CounterTracker counters, + final FrameProcessorExecutor exec, + final String cancellationId, + final int parallelism, + final FrameContext frameContext, + final MSQWarningReportPublisher MSQWarningReportPublisher + ) throws IOException + { + final WorkOrder workOrder = kernel.getWorkOrder(); + final int workerNumber = workOrder.getWorkerNumber(); + final StageDefinition stageDef = workOrder.getStageDefinition(); + + final InputChannels inputChannels = + new InputChannelsImpl( + workOrder.getQueryDefinition(), + InputSlices.allReadablePartitions(workOrder.getInputs()), + inputChannelFactory, + () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()), + exec, + cancellationId + ); + + final InputSliceReader inputSliceReader = makeInputSliceReader( + workOrder.getQueryDefinition(), + inputChannels, + frameContext.tempDir(), + frameContext.dataSegmentProvider() + ); + + final OutputChannelFactory workerOutputChannelFactory; + + if (stageDef.doesShuffle()) { + // Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame + // size, since we may be writing to a SuperSorter, and we'll generate fewer temp files if we use larger frames. + // Note: it's not *guaranteed* that we're writing to a SuperSorter, but it's harmless to use large frames + // even if not. + workerOutputChannelFactory = + new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getLargeFrameSize()); + } else { + // Writing stage output. + workerOutputChannelFactory = makeStageOutputChannelFactory(frameContext, stageDef.getStageNumber()); + } + + final ResultAndChannels workerResultAndOutputChannels = + makeAndRunWorkers( + workerNumber, + workOrder.getStageDefinition().getProcessorFactory(), + workOrder.getExtraInfo(), + new CountingOutputChannelFactory( + workerOutputChannelFactory, + counters.channel(CounterNames.outputChannel()) + ), + stageDef, + workOrder.getInputs(), + inputSliceReader, + frameContext, + exec, + cancellationId, + parallelism, + processorBouncer, + counters, + MSQWarningReportPublisher + ); + + final ListenableFuture stagePartitionBoundariesFuture; + final ListenableFuture outputChannelsFuture; + + if (stageDef.doesShuffle()) { + final ClusterBy clusterBy = workOrder.getStageDefinition().getShuffleSpec().get().getClusterBy(); + + final CountingOutputChannelFactory shuffleOutputChannelFactory = + new CountingOutputChannelFactory( + makeStageOutputChannelFactory(frameContext, stageDef.getStageNumber()), + counters.channel(CounterNames.shuffleChannel()) + ); + + if (stageDef.doesSortDuringShuffle()) { + if (stageDef.mustGatherResultKeyStatistics()) { + stagePartitionBoundariesFuture = SettableFuture.create(); + } else { + stagePartitionBoundariesFuture = Futures.immediateFuture(kernel.getResultPartitionBoundaries()); + } + + outputChannelsFuture = superSortOutputChannels( + workOrder.getStageDefinition(), + clusterBy, + workerResultAndOutputChannels.getOutputChannels(), + stagePartitionBoundariesFuture, + shuffleOutputChannelFactory, + exec, + cancellationId, + frameContext.memoryParameters(), + context, + kernelManipulationQueue, + counters.sortProgress() + ); + } else { + // No sorting, just combining all outputs into one big partition. Use a muxer to get everything into one file. + // Note: even if there is only one output channel, we'll run it through the muxer anyway, to ensure the data + // gets written to a file. (httpGetChannelData requires files.) + final OutputChannel outputChannel = shuffleOutputChannelFactory.openChannel(0); + + final FrameChannelMuxer muxer = + new FrameChannelMuxer( + workerResultAndOutputChannels.getOutputChannels() + .getAllChannels() + .stream() + .map(OutputChannel::getReadableChannel) + .collect(Collectors.toList()), + outputChannel.getWritableChannel() + ); + + //noinspection unchecked, rawtypes + outputChannelsFuture = Futures.transform( + exec.runFully(muxer, cancellationId), + (Function) ignored -> OutputChannels.wrap(Collections.singletonList(outputChannel.readOnly())) + ); + + stagePartitionBoundariesFuture = null; + } + } else { + stagePartitionBoundariesFuture = null; + + // Retain read-only versions to reduce memory footprint. + outputChannelsFuture = Futures.immediateFuture(workerResultAndOutputChannels.getOutputChannels().readOnly()); + } + + // Output channels and future are all constructed. Sanity check, record them, and set up callbacks. + Futures.addCallback( + Futures.allAsList( + Arrays.asList( + workerResultAndOutputChannels.getResultFuture(), + Futures.transform( + outputChannelsFuture, + new Function() + { + @Override + public OutputChannels apply(final OutputChannels channels) + { + sanityCheckOutputChannels(channels); + return channels; + } + } + ) + ) + ), + new FutureCallback>() + { + @Override + public void onSuccess(final List workerResultAndOutputChannelsResolved) + { + Object resultObject = workerResultAndOutputChannelsResolved.get(0); + final OutputChannels outputChannels = (OutputChannels) workerResultAndOutputChannelsResolved.get(1); + + for (OutputChannel channel : outputChannels.getAllChannels()) { + stageOutputs.computeIfAbsent(stageDef.getId(), ignored1 -> new ConcurrentHashMap<>()) + .computeIfAbsent(channel.getPartitionNumber(), ignored2 -> channel.getReadableChannel()); + } + kernelManipulationQueue.add(holder -> holder.getStageKernelMap() + .get(stageDef.getId()) + .setResultsComplete(resultObject)); + } + + @Override + public void onFailure(final Throwable t) + { + kernelManipulationQueue.add( + kernelHolder -> + kernelHolder.getStageKernelMap().get(stageDef.getId()).fail(t) + ); + } + } + ); + + // Return settable result-key-statistics future, so callers can set it and unblock the supersorter if needed. + return stageDef.mustGatherResultKeyStatistics() + ? (SettableFuture) stagePartitionBoundariesFuture + : null; + } + + private static , I, WorkerClass extends FrameProcessor, T, R> ResultAndChannels makeAndRunWorkers( + final int workerNumber, + final FactoryType processorFactory, + final I processorFactoryExtraInfo, + final OutputChannelFactory outputChannelFactory, + final StageDefinition stageDefinition, + final List inputSlices, + final InputSliceReader inputSliceReader, + final FrameContext frameContext, + final FrameProcessorExecutor exec, + final String cancellationId, + final int parallelism, + final Bouncer processorBouncer, + final CounterTracker counters, + final MSQWarningReportPublisher warningPublisher + ) throws IOException + { + final ProcessorsAndChannels processors = + processorFactory.makeProcessors( + stageDefinition, + workerNumber, + inputSlices, + inputSliceReader, + processorFactoryExtraInfo, + outputChannelFactory, + frameContext, + parallelism, + counters, + e -> warningPublisher.publishException(stageDefinition.getStageNumber(), e) + ); + + final Sequence processorSequence = processors.processors(); + + final int maxOutstandingProcessors; + + if (processors.getOutputChannels().getAllChannels().isEmpty()) { + // No output channels: run up to "parallelism" processors at once. + maxOutstandingProcessors = Math.max(1, parallelism); + } else { + // If there are output channels, that acts as a ceiling on the number of processors that can run at once. + maxOutstandingProcessors = + Math.max(1, Math.min(parallelism, processors.getOutputChannels().getAllChannels().size())); + } + + final ListenableFuture workResultFuture = exec.runAllFully( + processorSequence, + processorFactory.newAccumulatedResult(), + processorFactory::accumulateResult, + maxOutstandingProcessors, + processorBouncer, + cancellationId + ); + + return new ResultAndChannels<>(workResultFuture, processors.getOutputChannels()); + } + + private static InputSliceReader makeInputSliceReader( + final QueryDefinition queryDef, + final InputChannels inputChannels, + final File temporaryDirectory, + final DataSegmentProvider segmentProvider + ) + { + return new MapInputSliceReader( + ImmutableMap., InputSliceReader>builder() + .put(NilInputSlice.class, NilInputSliceReader.INSTANCE) + .put(StageInputSlice.class, new StageInputSliceReader(queryDef.getQueryId(), inputChannels)) + .put(ExternalInputSlice.class, new ExternalInputSliceReader(temporaryDirectory)) + .put(SegmentsInputSlice.class, new SegmentsInputSliceReader(segmentProvider)) + .build() + ); + } + + private static ListenableFuture superSortOutputChannels( + final StageDefinition stageDefinition, + final ClusterBy clusterBy, + final OutputChannels processorOutputChannels, + final ListenableFuture stagePartitionBoundariesFuture, + final OutputChannelFactory outputChannelFactory, + final FrameProcessorExecutor exec, + final String cancellationId, + final WorkerMemoryParameters memoryParameters, + final WorkerContext context, + final BlockingQueue> kernelManipulationQueue, + final SuperSorterProgressTracker superSorterProgressTracker + ) throws IOException + { + if (!stageDefinition.doesShuffle()) { + throw new ISE("Output channels do not need shuffling"); + } + + final List channelsToSuperSort; + + if (processorOutputChannels.getAllChannels().isEmpty()) { + // No data coming out of this processor. Report empty statistics, if the kernel is expecting statistics. + if (stageDefinition.mustGatherResultKeyStatistics()) { + kernelManipulationQueue.add( + holder -> + holder.getStageKernelMap().get(stageDefinition.getId()) + .setResultKeyStatisticsSnapshot(ClusterByStatisticsSnapshot.empty()) + ); + } + + // Process one empty channel so the SuperSorter has something to do. + final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); + channel.writable().close(); + channelsToSuperSort = Collections.singletonList(channel.readable()); + } else if (stageDefinition.mustGatherResultKeyStatistics()) { + channelsToSuperSort = collectKeyStatistics( + stageDefinition, + clusterBy, + processorOutputChannels, + exec, + cancellationId, + kernelManipulationQueue + ); + } else { + channelsToSuperSort = processorOutputChannels.getAllChannels() + .stream() + .map(OutputChannel::getReadableChannel) + .collect(Collectors.toList()); + } + + final File sorterTmpDir = new File(context.tempDir(), "super-sort-" + UUID.randomUUID()); + FileUtils.mkdirp(sorterTmpDir); + if (!sorterTmpDir.isDirectory()) { + throw new IOException("Cannot create directory: " + sorterTmpDir); + } + + final SuperSorter sorter = new SuperSorter( + channelsToSuperSort, + stageDefinition.getFrameReader(), + clusterBy, + stagePartitionBoundariesFuture, + exec, + sorterTmpDir, + outputChannelFactory, + () -> ArenaMemoryAllocator.createOnHeap(memoryParameters.getLargeFrameSize()), + memoryParameters.getSuperSorterMaxActiveProcessors(), + memoryParameters.getSuperSorterMaxChannelsPerProcessor(), + -1, + cancellationId, + superSorterProgressTracker + ); + + return sorter.run(); + } + + private static List collectKeyStatistics( + final StageDefinition stageDefinition, + final ClusterBy clusterBy, + final OutputChannels processorOutputChannels, + final FrameProcessorExecutor exec, + final String cancellationId, + final BlockingQueue> kernelManipulationQueue + ) + { + final List retVal = new ArrayList<>(); + final List processors = new ArrayList<>(); + + for (final OutputChannel outputChannel : processorOutputChannels.getAllChannels()) { + final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); + retVal.add(channel.readable()); + + processors.add( + new KeyStatisticsCollectionProcessor( + outputChannel.getReadableChannel(), + channel.writable(), + stageDefinition.getFrameReader(), + clusterBy, + stageDefinition.createResultKeyStatisticsCollector() + ) + ); + } + + final ListenableFuture clusterByStatisticsCollectorFuture = + exec.runAllFully( + Sequences.simple(processors), + stageDefinition.createResultKeyStatisticsCollector(), + ClusterByStatisticsCollector::addAll, + // Run all processors simultaneously. They are lightweight and this keeps things moving. + processors.size(), + Bouncer.unlimited(), + cancellationId + ); + + Futures.addCallback( + clusterByStatisticsCollectorFuture, + new FutureCallback() + { + @Override + public void onSuccess(final ClusterByStatisticsCollector result) + { + kernelManipulationQueue.add( + holder -> + holder.getStageKernelMap().get(stageDefinition.getId()) + .setResultKeyStatisticsSnapshot(result.snapshot()) + ); + } + + @Override + public void onFailure(Throwable t) + { + kernelManipulationQueue.add( + holder -> { + log.noStackTrace() + .warn(t, "Failed to gather clusterBy statistics for stage [%s]", stageDefinition.getId()); + holder.getStageKernelMap().get(stageDefinition.getId()).fail(t); + } + ); + } + } + ); + + return retVal; + } + + private static void sanityCheckOutputChannels(final OutputChannels outputChannels) + { + // Verify there is exactly one channel per partition. + for (int partitionNumber : outputChannels.getPartitionNumbers()) { + final List outputChannelsForPartition = + outputChannels.getChannelsForPartition(partitionNumber); + + Preconditions.checkState(partitionNumber >= 0, "Expected partitionNumber >= 0, but got [%s]", partitionNumber); + Preconditions.checkState( + outputChannelsForPartition.size() == 1, + "Expected one channel for partition [%s], but got [%s]", + partitionNumber, + outputChannelsForPartition.size() + ); + } + } + + /** + * Log (at DEBUG level) a string explaining the status of all work assigned to this worker. + */ + private static void logKernelStatus(final Collection kernels) + { + if (log.isDebugEnabled()) { + log.debug( + "Stages: %s", + kernels.stream() + .sorted(Comparator.comparing(k -> k.getStageDefinition().getStageNumber())) + .map(WorkerImpl::makeKernelStageStatusString) + .collect(Collectors.joining("; ")) + ); + } + } + + /** + * Helper used by {@link #logKernelStatus}. + */ + private static String makeKernelStageStatusString(final WorkerStageKernel kernel) + { + final String inputPartitionNumbers = + StreamSupport.stream(InputSlices.allReadablePartitions(kernel.getWorkOrder().getInputs()).spliterator(), false) + .map(ReadablePartition::getPartitionNumber) + .sorted() + .map(String::valueOf) + .collect(Collectors.joining(",")); + + // String like ">50" if shuffling to 50 partitions, ">?" if shuffling to unknown number of partitions. + final String shuffleStatus = + kernel.getStageDefinition().doesShuffle() + ? ">" + (kernel.hasResultPartitionBoundaries() ? kernel.getResultPartitionBoundaries().size() : "?") + : ""; + + return StringUtils.format( + "S%d:W%d:P[%s]%s:%s:%s", + kernel.getStageDefinition().getStageNumber(), + kernel.getWorkOrder().getWorkerNumber(), + inputPartitionNumbers, + shuffleStatus, + kernel.getStageDefinition().doesShuffle() ? "SHUFFLE" : "RETAIN", + kernel.getPhase() + ); + } + + /** + * An {@link InputChannelFactory} that loads data locally when possible, and otherwise connects directly to other + * workers. Used when durable shuffle storage is off. + */ + private class WorkerOrLocalInputChannelFactory implements InputChannelFactory + { + private final Supplier> taskList; + private final WorkerInputChannelFactory workerInputChannelFactory; + + public WorkerOrLocalInputChannelFactory(final Supplier> taskList) + { + this.workerInputChannelFactory = new WorkerInputChannelFactory(workerClient, taskList); + this.taskList = taskList; + } + + @Override + public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) + { + final String taskId = taskList.get().get(workerNumber); + if (taskId.equals(id())) { + final ConcurrentMap partitionOutputsForStage = stageOutputs.get(stageId); + if (partitionOutputsForStage == null) { + throw new ISE("Unable to find outputs for stage: [%s]", stageId); + } + + final ReadableFrameChannel myChannel = partitionOutputsForStage.get(partitionNumber); + + if (myChannel instanceof ReadableFileFrameChannel) { + // Must duplicate the channel to avoid double-closure upon task cleanup. + final FrameFile frameFile = ((ReadableFileFrameChannel) myChannel).newFrameFileReference(); + return new ReadableFileFrameChannel(frameFile); + } else if (myChannel instanceof ReadableNilFrameChannel) { + return myChannel; + } else { + throw new ISE("Output for stage: [%s] are stored in an instance of %s which is not " + + "supported", stageId, myChannel.getClass()); + } + } else { + return workerInputChannelFactory.openChannel(stageId, workerNumber, partitionNumber); + } + } + } + + private static class KernelHolder + { + private final Map stageKernelMap = new HashMap<>(); + private boolean done = false; + + public Map getStageKernelMap() + { + return stageKernelMap; + } + + public boolean isDone() + { + return done; + } + + public void setDone() + { + this.done = true; + } + } + + private static class ResultAndChannels + { + private final ListenableFuture resultFuture; + private final OutputChannels outputChannels; + + public ResultAndChannels( + ListenableFuture resultFuture, + OutputChannels outputChannels + ) + { + this.resultFuture = resultFuture; + this.outputChannels = outputChannels; + } + + public ListenableFuture getResultFuture() + { + return resultFuture; + } + + public OutputChannels getOutputChannels() + { + return outputChannels; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java new file mode 100644 index 00000000000..7d320b111ff --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java @@ -0,0 +1,57 @@ +/* + * 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.druid.msq.exec; + +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.msq.indexing.MSQWorkerTask; + +import java.io.Closeable; +import java.util.Map; +import java.util.Set; + +/** + * Generic interface to the "worker manager" mechanism which starts, cancels and monitors worker tasks. + */ +public interface WorkerManagerClient extends Closeable +{ + String run(String controllerId, MSQWorkerTask task); + + /** + * @param workerId the task ID + * + * @return a {@code TaskLocation} associated with the task or + * {@code TaskLocation.unknown()} if no associated entry could be found + */ + TaskLocation location(String workerId); + + /** + * Fetches status map corresponding to a group of task ids + */ + Map statuses(Set taskIds); + + /** + * Cancel the task corresponding to the provided workerId + */ + void cancel(String workerId); + + @Override + void close(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java new file mode 100644 index 00000000000..9f003311917 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java @@ -0,0 +1,294 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.primitives.Ints; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault; +import org.apache.druid.msq.indexing.error.TooManyWorkersFault; + +import java.util.Objects; + +/** + * Class for determining how much JVM heap to allocate to various purposes. + * + * First, we take {@link #USABLE_MEMORY_FRACTION} out of the total JVM heap and split it into "bundles" of + * equal size. The number of bundles is based entirely on server configuration; this makes the calculation + * robust to different queries running simultaneously in the same JVM. + * + * Then, we split up the resources for each bundle in two different ways: one assuming it'll be used for a + * {@link org.apache.druid.frame.processor.SuperSorter}, and one assuming it'll be used for a regular + * processor. Callers can then use whichever set of allocations makes sense. (We assume no single bundle + * will be used for both purposes.) + */ +public class WorkerMemoryParameters +{ + /** + * Percent of memory that we allocate to bundles. It is less than 100% because we need to leave some space + * left over for miscellaneous other stuff, and to ensure that GC pressure does not get too high. + */ + private static final double USABLE_MEMORY_FRACTION = 0.75; + + /** + * Percent of each bundle's memory that we allocate to appenderators. It is less than 100% because appenderators + * unfortunately have a variety of unaccounted-for memory usage. + */ + static final double APPENDERATOR_MEMORY_FRACTION = 0.67; + + /** + * Size for "standard frames", which are used for most purposes, except inputs to super-sorters. + * + * In particular, frames that travel between workers are always the minimum size. This is helpful because it makes + * it easier to compute the amount of memory needed to merge input streams. + */ + private static final int STANDARD_FRAME_SIZE = 1_000_000; + + /** + * Size for "large frames", which are used for inputs and inner channels in to super-sorters. + * + * This is helpful because it minimizes the number of temporary files needed during super-sorting. + */ + private static final int LARGE_FRAME_SIZE = 8_000_000; + + /** + * Minimum amount of bundle memory available for processing (i.e., total bundle size minus the amount + * needed for input channels). This memory is guaranteed to be available for things like segment generation + * and broadcast data. + */ + public static final long PROCESSING_MINIMUM_BYTES = 25_000_000; + + /** + * Maximum amount of parallelism for the super-sorter. Higher amounts of concurrency tend to be wasteful. + */ + private static final int MAX_SUPER_SORTER_PROCESSORS = 4; + + /** + * Each super-sorter must have at least 1 processor with 2 input frames and 1 output frame. That's 3 total. + */ + private static final int MIN_SUPER_SORTER_FRAMES = 3; + + /** + * (Very) rough estimate of the on-heap overhead of reading a column. + */ + private static final int APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN = 3_000; + + /** + * Fraction of free memory per bundle that can be used by {@link org.apache.druid.msq.querykit.BroadcastJoinHelper} + * to store broadcast data on-heap. This is used to limit the total size of input frames, which we expect to + * expand on-heap. Expansion can potentially be somewhat over 2x: for example, strings are UTF-8 in frames, but are + * UTF-16 on-heap, which is a 2x expansion, and object and index overhead must be considered on top of that. So + * we use a value somewhat lower than 0.5. + */ + static final double BROADCAST_JOIN_MEMORY_FRACTION = 0.3; + + private final int superSorterMaxActiveProcessors; + private final int superSorterMaxChannelsPerProcessor; + private final long appenderatorMemory; + private final long broadcastJoinMemory; + + WorkerMemoryParameters( + final int superSorterMaxActiveProcessors, + final int superSorterMaxChannelsPerProcessor, + final long appenderatorMemory, + final long broadcastJoinMemory + ) + { + this.superSorterMaxActiveProcessors = superSorterMaxActiveProcessors; + this.superSorterMaxChannelsPerProcessor = superSorterMaxChannelsPerProcessor; + this.appenderatorMemory = appenderatorMemory; + this.broadcastJoinMemory = broadcastJoinMemory; + } + + /** + * Returns an object specifying memory-usage parameters for a stage in a worker. + * + * Throws a {@link MSQException} with an appropriate fault if the provided combination of parameters cannot + * yield a workable memory situation. + * + * @param maxMemoryInJvm memory available in the entire JVM. This will be divided amongst processors. + * @param numWorkersInJvm number of workers that can run concurrently in this JVM. Generally equal to + * the task capacity. + * @param numProcessingThreadsInJvm size of the processing thread pool in the JVM. + * @param numInputWorkers number of workers across input stages that need to be merged together. + */ + public static WorkerMemoryParameters compute( + final long maxMemoryInJvm, + final int numWorkersInJvm, + final int numProcessingThreadsInJvm, + final int numInputWorkers + ) + { + final long bundleMemory = memoryPerBundle(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); + final long bundleMemoryForInputChannels = memoryNeededForInputChannels(numInputWorkers); + final long bundleMemoryForProcessing = bundleMemory - bundleMemoryForInputChannels; + + if (bundleMemoryForProcessing < PROCESSING_MINIMUM_BYTES) { + final int maxWorkers = computeMaxWorkers(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); + + if (maxWorkers > 0) { + throw new MSQException(new TooManyWorkersFault(numInputWorkers, Math.min(Limits.MAX_WORKERS, maxWorkers))); + } else { + // Not enough memory for even one worker. More of a NotEnoughMemory situation than a TooManyWorkers situation. + throw new MSQException( + new NotEnoughMemoryFault( + maxMemoryInJvm, + numWorkersInJvm, + numProcessingThreadsInJvm + ) + ); + } + } + + // Compute memory breakdown for super-sorting bundles. + final int maxNumFramesForSuperSorter = Ints.checkedCast(bundleMemory / WorkerMemoryParameters.LARGE_FRAME_SIZE); + + if (maxNumFramesForSuperSorter < MIN_SUPER_SORTER_FRAMES) { + throw new MSQException(new NotEnoughMemoryFault(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm)); + } + + final int superSorterMaxActiveProcessors = Math.min( + numProcessingThreadsInJvm, + Math.min( + maxNumFramesForSuperSorter / MIN_SUPER_SORTER_FRAMES, + MAX_SUPER_SORTER_PROCESSORS + ) + ); + + // Apportion max frames to all processors equally, then subtract one to account for an output frame. + final int superSorterMaxChannelsPerProcessor = maxNumFramesForSuperSorter / superSorterMaxActiveProcessors - 1; + + return new WorkerMemoryParameters( + superSorterMaxActiveProcessors, + superSorterMaxChannelsPerProcessor, + (long) (bundleMemoryForProcessing * APPENDERATOR_MEMORY_FRACTION), + (long) (bundleMemoryForProcessing * BROADCAST_JOIN_MEMORY_FRACTION) + ); + } + + public int getSuperSorterMaxActiveProcessors() + { + return superSorterMaxActiveProcessors; + } + + public int getSuperSorterMaxChannelsPerProcessor() + { + return superSorterMaxChannelsPerProcessor; + } + + public long getAppenderatorMaxBytesInMemory() + { + // Half for indexing, half for merging. + return Math.max(1, appenderatorMemory / 2); + } + + public int getAppenderatorMaxColumnsToMerge() + { + // Half for indexing, half for merging. + return Ints.checkedCast(Math.max(2, appenderatorMemory / 2 / APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN)); + } + + public int getStandardFrameSize() + { + return STANDARD_FRAME_SIZE; + } + + public int getLargeFrameSize() + { + return LARGE_FRAME_SIZE; + } + + public long getBroadcastJoinMemory() + { + return broadcastJoinMemory; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkerMemoryParameters that = (WorkerMemoryParameters) o; + return superSorterMaxActiveProcessors == that.superSorterMaxActiveProcessors + && superSorterMaxChannelsPerProcessor == that.superSorterMaxChannelsPerProcessor + && appenderatorMemory == that.appenderatorMemory + && broadcastJoinMemory == that.broadcastJoinMemory; + } + + @Override + public int hashCode() + { + return Objects.hash( + superSorterMaxActiveProcessors, + superSorterMaxChannelsPerProcessor, + appenderatorMemory, + broadcastJoinMemory + ); + } + + @Override + public String toString() + { + return "WorkerMemoryParameters{" + + "superSorterMaxActiveProcessors=" + superSorterMaxActiveProcessors + + ", superSorterMaxChannelsPerProcessor=" + superSorterMaxChannelsPerProcessor + + ", appenderatorMemory=" + appenderatorMemory + + ", broadcastJoinMemory=" + broadcastJoinMemory + + '}'; + } + + /** + * Computes the highest value of numInputWorkers, for the given parameters, that can be passed to + * {@link #compute} without resulting in a {@link TooManyWorkersFault}. + * + * Returns 0 if no number of workers would be OK. + */ + static int computeMaxWorkers( + final long maxMemoryInJvm, + final int numWorkersInJvm, + final int numProcessingThreadsInJvm + ) + { + final long bundleMemory = memoryPerBundle(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); + + // Inverse of memoryNeededForInputChannels. + return Ints.checkedCast((bundleMemory - PROCESSING_MINIMUM_BYTES) / STANDARD_FRAME_SIZE - 1); + } + + private static long memoryPerBundle( + final long maxMemoryInJvm, + final int numWorkersInJvm, + final int numProcessingThreadsInJvm + ) + { + final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm; + return (long) (maxMemoryInJvm * USABLE_MEMORY_FRACTION) / bundleCount; + } + + private static long memoryNeededForInputChannels(final int numInputWorkers) + { + // Regular processors require input-channel-merging for their inputs. Calculate how much that is. + // Requirement: inputChannelsPerProcessor number of input frames, one output frame. + return (long) STANDARD_FRAME_SIZE * (numInputWorkers + 1); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQDurableStorageModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQDurableStorageModule.java new file mode 100644 index 00000000000..67818532162 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQDurableStorageModule.java @@ -0,0 +1,86 @@ +/* + * 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.druid.msq.guice; + +import com.fasterxml.jackson.databind.Module; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Key; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.StorageConnectorProvider; + +import java.util.List; +import java.util.Properties; + +/** + * Module for functionality related to durable storage for stage output data. + */ +public class MSQDurableStorageModule implements DruidModule +{ + public static final String MSQ_INTERMEDIATE_STORAGE_PREFIX = + String.join(".", MSQIndexingModule.BASE_MSQ_KEY, "intermediate.storage"); + + public static final String MSQ_INTERMEDIATE_STORAGE_ENABLED = + String.join(".", MSQ_INTERMEDIATE_STORAGE_PREFIX, "enable"); + + @Inject + private Properties properties; + + public MSQDurableStorageModule() + { + } + + public MSQDurableStorageModule(Properties properties) + { + this.properties = properties; + } + + @Override + public List getJacksonModules() + { + return ImmutableList.of(); + } + + @Override + public void configure(Binder binder) + { + if (isDurableShuffleStorageEnabled()) { + JsonConfigProvider.bind( + binder, + MSQ_INTERMEDIATE_STORAGE_PREFIX, + StorageConnectorProvider.class, + MultiStageQuery.class + ); + + binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class)) + .toProvider(Key.get(StorageConnectorProvider.class, MultiStageQuery.class)) + .in(LazySingleton.class); + } + } + + private boolean isDurableShuffleStorageEnabled() + { + return Boolean.parseBoolean((String) properties.getOrDefault(MSQ_INTERMEDIATE_STORAGE_ENABLED, "false")); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQExternalDataSourceModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQExternalDataSourceModule.java new file mode 100644 index 00000000000..c2ed663acfa --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQExternalDataSourceModule.java @@ -0,0 +1,52 @@ +/* + * 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.druid.msq.guice; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.sql.calcite.external.ExternalDataSource; + +import java.util.Collections; +import java.util.List; + +/** + * Module that adds {@link ExternalDataSource}, a class that is in core but is not bound by any core modules. + * + * It is only available if this extension is loaded. + */ +public class MSQExternalDataSourceModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return Collections.singletonList( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes(ExternalDataSource.class) + ); + } + + @Override + public void configure(Binder binder) + { + // Nothing to do. + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java new file mode 100644 index 00000000000..313de7cd111 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -0,0 +1,201 @@ +/* + * 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.druid.msq.guice; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Provides; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.counters.CounterSnapshotsSerializer; +import org.apache.druid.msq.counters.SuperSorterProgressTrackerCounter; +import org.apache.druid.msq.counters.WarningCounters; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQWorkerTask; +import org.apache.druid.msq.indexing.SegmentGeneratorFrameProcessorFactory; +import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; +import org.apache.druid.msq.indexing.error.CanceledFault; +import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; +import org.apache.druid.msq.indexing.error.ColumnNameRestrictedFault; +import org.apache.druid.msq.indexing.error.ColumnTypeNotSupportedFault; +import org.apache.druid.msq.indexing.error.DurableStorageConfigurationFault; +import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault; +import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; +import org.apache.druid.msq.indexing.error.InsertCannotOrderByDescendingFault; +import org.apache.druid.msq.indexing.error.InsertCannotReplaceExistingSegmentFault; +import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault; +import org.apache.druid.msq.indexing.error.InsertTimeNullFault; +import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault; +import org.apache.druid.msq.indexing.error.InvalidNullByteFault; +import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault; +import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; +import org.apache.druid.msq.indexing.error.RowTooLargeFault; +import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; +import org.apache.druid.msq.indexing.error.TooManyBucketsFault; +import org.apache.druid.msq.indexing.error.TooManyColumnsFault; +import org.apache.druid.msq.indexing.error.TooManyInputFilesFault; +import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; +import org.apache.druid.msq.indexing.error.TooManyWarningsFault; +import org.apache.druid.msq.indexing.error.TooManyWorkersFault; +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.error.WorkerFailedFault; +import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; +import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.input.NilInputSource; +import org.apache.druid.msq.input.external.ExternalInputSlice; +import org.apache.druid.msq.input.external.ExternalInputSpec; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.input.table.SegmentsInputSlice; +import org.apache.druid.msq.input.table.TableInputSpec; +import org.apache.druid.msq.kernel.NilExtraInfoHolder; +import org.apache.druid.msq.querykit.InputNumberDataSource; +import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.msq.querykit.groupby.GroupByPostShuffleFrameProcessorFactory; +import org.apache.druid.msq.querykit.groupby.GroupByPreShuffleFrameProcessorFactory; +import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessorFactory; +import org.apache.druid.msq.util.PassthroughAggregatorFactory; +import org.apache.druid.query.DruidProcessingConfig; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * Module that adds {@link MSQControllerTask}, {@link MSQWorkerTask}, and dependencies. + */ +public class MSQIndexingModule implements DruidModule +{ + static final String BASE_MSQ_KEY = "druid.msq"; + + public static final List> FAULT_CLASSES = ImmutableList.of( + BroadcastTablesTooLargeFault.class, + CanceledFault.class, + CannotParseExternalDataFault.class, + ColumnTypeNotSupportedFault.class, + ColumnNameRestrictedFault.class, + DurableStorageConfigurationFault.class, + InsertCannotAllocateSegmentFault.class, + InsertCannotBeEmptyFault.class, + InsertCannotOrderByDescendingFault.class, + InsertCannotReplaceExistingSegmentFault.class, + InsertLockPreemptedFault.class, + InsertTimeNullFault.class, + InsertTimeOutOfBoundsFault.class, + InvalidNullByteFault.class, + NotEnoughMemoryFault.class, + QueryNotSupportedFault.class, + RowTooLargeFault.class, + TaskStartTimeoutFault.class, + TooManyBucketsFault.class, + TooManyColumnsFault.class, + TooManyInputFilesFault.class, + TooManyPartitionsFault.class, + TooManyWarningsFault.class, + TooManyWorkersFault.class, + UnknownFault.class, + WorkerFailedFault.class, + WorkerRpcFailedFault.class + ); + + @Override + public List getJacksonModules() + { + final SimpleModule module = new SimpleModule(getClass().getSimpleName()); + + module.registerSubtypes( + // Task classes + MSQControllerTask.class, + MSQWorkerTask.class, + + // FrameChannelWorkerFactory and FrameChannelWorkerFactoryExtraInfoHolder classes + SegmentGeneratorFrameProcessorFactory.class, + SegmentGeneratorFrameProcessorFactory.SegmentGeneratorExtraInfoHolder.class, + ScanQueryFrameProcessorFactory.class, + GroupByPreShuffleFrameProcessorFactory.class, + GroupByPostShuffleFrameProcessorFactory.class, + OffsetLimitFrameProcessorFactory.class, + NilExtraInfoHolder.class, + + // FrameChannelWorkerFactory and FrameChannelWorkerFactoryExtraInfoHolder classes + ScanQueryFrameProcessorFactory.class, + GroupByPreShuffleFrameProcessorFactory.class, + GroupByPostShuffleFrameProcessorFactory.class, + OffsetLimitFrameProcessorFactory.class, + NilExtraInfoHolder.class, + + // DataSource classes (note: ExternalDataSource is in MSQSqlModule) + InputNumberDataSource.class, + + // TaskReport classes + MSQTaskReport.class, + + // QueryCounter.Snapshot classes + ChannelCounters.Snapshot.class, + SuperSorterProgressTrackerCounter.Snapshot.class, + WarningCounters.Snapshot.class, + + // InputSpec classes + ExternalInputSpec.class, + StageInputSpec.class, + TableInputSpec.class, + + // InputSlice classes + ExternalInputSlice.class, + NilInputSlice.class, + SegmentsInputSlice.class, + StageInputSlice.class, + + // Other + PassthroughAggregatorFactory.class, + NilInputSource.class + ); + + FAULT_CLASSES.forEach(module::registerSubtypes); + module.addSerializer(new CounterSnapshotsSerializer()); + return Collections.singletonList(module); + } + + @Override + public void configure(Binder binder) + { + } + + @Provides + @LazySingleton + public Bouncer makeBouncer(final DruidProcessingConfig processingConfig, @Self final Set nodeRoles) + { + if (nodeRoles.contains(NodeRole.PEON) && !nodeRoles.contains(NodeRole.INDEXER)) { + // CliPeon -> use only one thread regardless of configured # of processing threads. This matches the expected + // resource usage pattern for CliPeon-based tasks (one task / one working thread per JVM). + return new Bouncer(1); + } else { + return new Bouncer(processingConfig.getNumThreads()); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQServiceClientModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQServiceClientModule.java new file mode 100644 index 00000000000..436fa7bebbe --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQServiceClientModule.java @@ -0,0 +1,86 @@ +/* + * 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.druid.msq.guice; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Provides; +import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.msq.rpc.CoordinatorServiceClient; +import org.apache.druid.msq.rpc.CoordinatorServiceClientImpl; +import org.apache.druid.rpc.DiscoveryServiceLocator; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocator; +import org.apache.druid.rpc.StandardRetryPolicy; + +import java.util.Collections; +import java.util.List; + +/** + * Module for providing {@link CoordinatorServiceClient}. + */ +public class MSQServiceClientModule implements DruidModule +{ + private static final int COORDINATOR_ATTEMPTS = 6; + + @Override + public List getJacksonModules() + { + return Collections.emptyList(); + } + + @Override + public void configure(Binder binder) + { + // Nothing to do. + } + + @Provides + @ManageLifecycle + @Coordinator + public ServiceLocator makeCoordinatorServiceLocator(final DruidNodeDiscoveryProvider discoveryProvider) + { + return new DiscoveryServiceLocator(discoveryProvider, NodeRole.COORDINATOR); + } + + @Provides + public CoordinatorServiceClient makeCoordinatorServiceClient( + @Json final ObjectMapper jsonMapper, + @EscalatedGlobal final ServiceClientFactory clientFactory, + @Coordinator final ServiceLocator serviceLocator + ) + { + return new CoordinatorServiceClientImpl( + clientFactory.makeClient( + NodeRole.COORDINATOR.getJsonName(), + serviceLocator, + StandardRetryPolicy.builder().maxAttempts(COORDINATOR_ATTEMPTS).build() + ), + jsonMapper + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java new file mode 100644 index 00000000000..9aea33f4072 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.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.druid.msq.guice; + +import com.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import com.google.inject.Inject; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.metadata.input.InputSourceModule; +import org.apache.druid.sql.calcite.external.ExternalOperatorConversion; +import org.apache.druid.sql.guice.SqlBindings; + +import java.util.List; +import java.util.Properties; + +/** + * Module for providing the {@code EXTERN} operator. + */ +@LoadScope(roles = NodeRole.BROKER_JSON_NAME) +public class MSQSqlModule implements DruidModule +{ + @Inject + Properties properties = null; + + @Override + public List getJacksonModules() + { + // We want this module to bring input sources along for the ride. + return new InputSourceModule().getJacksonModules(); + } + + @Override + public void configure(Binder binder) + { + // We want this module to bring InputSourceModule along for the ride. + binder.install(new InputSourceModule()); + + // Set up the EXTERN macro. + SqlBindings.addOperatorConversion(binder, ExternalOperatorConversion.class); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MultiStageQuery.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MultiStageQuery.java new file mode 100644 index 00000000000..954b5fdd14b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MultiStageQuery.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.guice; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface MultiStageQuery +{ +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java new file mode 100644 index 00000000000..d8f8b06f9b9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.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.druid.msq.guice; + +import com.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.msq.sql.SqlTaskResource; + +import java.util.Collections; +import java.util.List; + +/** + * Module for adding the {@link SqlTaskResource} endpoint to the Broker. + */ +@LoadScope(roles = NodeRole.BROKER_JSON_NAME) +public class SqlTaskModule implements DruidModule +{ + @Override + public void configure(Binder binder) + { + // Force eager initialization. + LifecycleModule.register(binder, SqlTaskResource.class); + Jerseys.addResource(binder, SqlTaskResource.class); + } + + @Override + public List getJacksonModules() + { + return Collections.emptyList(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMapping.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMapping.java new file mode 100644 index 00000000000..6e55270d6c2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMapping.java @@ -0,0 +1,82 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.Objects; + +public class ColumnMapping +{ + private final String queryColumn; + private final String outputColumn; + + @JsonCreator + public ColumnMapping( + @JsonProperty("queryColumn") String queryColumn, + @JsonProperty("outputColumn") String outputColumn + ) + { + this.queryColumn = Preconditions.checkNotNull(queryColumn, "queryColumn"); + this.outputColumn = Preconditions.checkNotNull(outputColumn, "outputColumn"); + } + + @JsonProperty + public String getQueryColumn() + { + return queryColumn; + } + + @JsonProperty + public String getOutputColumn() + { + return outputColumn; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColumnMapping that = (ColumnMapping) o; + return Objects.equals(queryColumn, that.queryColumn) && Objects.equals(outputColumn, that.outputColumn); + } + + @Override + public int hashCode() + { + return Objects.hash(queryColumn, outputColumn); + } + + @Override + public String toString() + { + return "ColumnMapping{" + + "queryColumn='" + queryColumn + '\'' + + ", outputColumn='" + outputColumn + '\'' + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMappings.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMappings.java new file mode 100644 index 00000000000..fddfbeefc76 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMappings.java @@ -0,0 +1,144 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.column.RowSignature; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +public class ColumnMappings +{ + private final List mappings; + private final Map outputToQueryColumnMap; + private final Map> queryToOutputColumnsMap; + + @JsonCreator + public ColumnMappings(final List mappings) + { + this.mappings = validateNoDuplicateOutputColumns(Preconditions.checkNotNull(mappings, "mappings")); + this.outputToQueryColumnMap = new HashMap<>(); + this.queryToOutputColumnsMap = new HashMap<>(); + + for (final ColumnMapping mapping : mappings) { + outputToQueryColumnMap.put(mapping.getOutputColumn(), mapping.getQueryColumn()); + queryToOutputColumnsMap.computeIfAbsent(mapping.getQueryColumn(), k -> new ArrayList<>()) + .add(mapping.getOutputColumn()); + } + } + + public static ColumnMappings identity(final RowSignature signature) + { + return new ColumnMappings( + signature.getColumnNames() + .stream() + .map(column -> new ColumnMapping(column, column)) + .collect(Collectors.toList()) + ); + } + + public List getOutputColumnNames() + { + return mappings.stream().map(ColumnMapping::getOutputColumn).collect(Collectors.toList()); + } + + public boolean hasOutputColumn(final String columnName) + { + return outputToQueryColumnMap.containsKey(columnName); + } + + public String getQueryColumnForOutputColumn(final String outputColumn) + { + final String queryColumn = outputToQueryColumnMap.get(outputColumn); + if (queryColumn != null) { + return queryColumn; + } else { + throw new IAE("No such output column [%s]", outputColumn); + } + } + + public List getOutputColumnsForQueryColumn(final String queryColumn) + { + final List outputColumns = queryToOutputColumnsMap.get(queryColumn); + if (outputColumns != null) { + return outputColumns; + } else { + return Collections.emptyList(); + } + } + + @JsonValue + public List getMappings() + { + return mappings; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColumnMappings that = (ColumnMappings) o; + return Objects.equals(mappings, that.mappings); + } + + @Override + public int hashCode() + { + return Objects.hash(mappings); + } + + @Override + public String toString() + { + return "ColumnMappings{" + + "mappings=" + mappings + + '}'; + } + + private static List validateNoDuplicateOutputColumns(final List mappings) + { + final Set encountered = new HashSet<>(); + + for (final ColumnMapping mapping : mappings) { + if (!encountered.add(mapping.getOutputColumn())) { + throw new ISE("Duplicate output column [%s]", mapping.getOutputColumn()); + } + } + + return mappings; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java new file mode 100644 index 00000000000..5561a98a07b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java @@ -0,0 +1,194 @@ +/* + * 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.druid.msq.indexing; + +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.msq.counters.CounterSnapshots; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.segment.realtime.firehose.ChatHandlers; +import org.apache.druid.server.security.Action; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.util.List; +import java.util.Map; + +public class ControllerChatHandler implements ChatHandler +{ + private final Controller controller; + private final MSQControllerTask task; + private final TaskToolbox toolbox; + + public ControllerChatHandler(TaskToolbox toolbox, Controller controller) + { + this.controller = controller; + this.task = controller.task(); + this.toolbox = toolbox; + } + + /** + * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages. + * + * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API. + */ + @POST + @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpPostKeyStatistics( + final Object keyStatisticsObject, + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("workerNumber") final int workerNumber, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + controller.updateStatus(stageNumber, workerNumber, keyStatisticsObject); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * Used by subtasks to post system errors. Note that the errors are organized by taskId, not by query/stage/worker, + * because system errors are associated with a task rather than a specific query/stage/worker execution context. + * + * See {@link ControllerClient#postWorkerError} for the client-side code that calls this API. + */ + @POST + @Path("/workerError/{taskId}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpPostWorkerError( + final MSQErrorReport errorReport, + @PathParam("taskId") final String taskId, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + controller.workerError(errorReport); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * Used by subtasks to post system warnings. + * + * See {@link ControllerClient#postWorkerWarning} for the client-side code that calls this API. + */ + @POST + @Path("/workerWarning/{taskId}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpPostWorkerWarning( + final List errorReport, + @PathParam("taskId") final String taskId, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + controller.workerWarning(errorReport); + return Response.status(Response.Status.ACCEPTED).build(); + } + + + /** + * Used by subtasks to post {@link CounterSnapshots} periodically. + * + * See {@link ControllerClient#postCounters} for the client-side code that calls this API. + */ + @POST + @Path("/counters") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpPostCounters( + final CounterSnapshotsTree snapshotsTree, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + controller.updateCounters(snapshotsTree); + return Response.status(Response.Status.OK).build(); + } + + /** + * Used by subtasks to post notifications that their results are ready. + * + * See {@link ControllerClient#postResultsComplete} for the client-side code that calls this API. + */ + @POST + @Path("/resultsComplete/{queryId}/{stageNumber}/{workerNumber}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpPostResultsComplete( + final Object resultObject, + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("workerNumber") final int workerNumber, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + controller.resultsComplete(queryId, stageNumber, workerNumber, resultObject); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * See {@link ControllerClient#getTaskList} for the client-side code that calls this API. + */ + @GET + @Path("/taskList") + @Produces(MediaType.APPLICATION_JSON) + public Response httpGetTaskList(@Context final HttpServletRequest req) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + + return Response.ok(new MSQTaskList(controller.getTaskIds())).build(); + } + + /** + * See {@link org.apache.druid.indexing.overlord.RemoteTaskRunner#streamTaskReports} for the client-side code that + * calls this API. + */ + @GET + @Path("/liveReports") + @Produces(MediaType.APPLICATION_JSON) + public Response httpGetLiveReports(@Context final HttpServletRequest req) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + final Map reports = controller.liveReports(); + if (reports == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + return Response.ok(reports).build(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java new file mode 100644 index 00000000000..54b8ac9a8cd --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.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.druid.msq.indexing; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.msq.counters.ChannelCounters; + +import java.io.IOException; + +public class CountableInputSourceReader implements InputSourceReader +{ + private final InputSourceReader inputSourceReader; + private final ChannelCounters channelCounters; + + public CountableInputSourceReader( + final InputSourceReader inputSourceReader, + final ChannelCounters channelCounters + ) + { + this.inputSourceReader = inputSourceReader; + this.channelCounters = channelCounters; + } + + @Override + public CloseableIterator read() throws IOException + { + return inputSourceReader.read().map(inputRow -> { + channelCounters.incrementRowCount(); + return inputRow; + }); + } + + @Override + public CloseableIterator sample() throws IOException + { + return inputSourceReader.sample(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java new file mode 100644 index 00000000000..0521389d667 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.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.druid.msq.indexing; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.msq.counters.ChannelCounters; + +import java.io.IOException; + +public class CountingOutputChannelFactory implements OutputChannelFactory +{ + private final OutputChannelFactory baseFactory; + private final ChannelCounters channelCounters; + + public CountingOutputChannelFactory( + final OutputChannelFactory baseFactory, + final ChannelCounters channelCounters + ) + { + this.baseFactory = Preconditions.checkNotNull(baseFactory, "baseFactory"); + this.channelCounters = Preconditions.checkNotNull(channelCounters, "channelCounter"); + } + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + final OutputChannel baseChannel = baseFactory.openChannel(partitionNumber); + + return baseChannel.mapWritableChannel( + baseWritableChannel -> + new CountingWritableFrameChannel( + baseChannel.getWritableChannel(), + channelCounters, + baseChannel.getPartitionNumber() + ) + ); + } + + @Override + public OutputChannel openNilChannel(final int partitionNumber) + { + // No need for counters on nil channels: they never receive input. + return baseFactory.openNilChannel(partitionNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingReadableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingReadableFrameChannel.java new file mode 100644 index 00000000000..6462832acbf --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingReadableFrameChannel.java @@ -0,0 +1,75 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.msq.counters.ChannelCounters; + +public class CountingReadableFrameChannel implements ReadableFrameChannel +{ + private final ReadableFrameChannel baseChannel; + private final ChannelCounters channelCounters; + private final int partitionNumber; + + public CountingReadableFrameChannel( + ReadableFrameChannel baseChannel, + ChannelCounters channelCounters, + int partitionNumber + ) + { + this.baseChannel = baseChannel; + this.channelCounters = channelCounters; + this.partitionNumber = partitionNumber; + } + + @Override + public boolean isFinished() + { + return baseChannel.isFinished(); + } + + @Override + public boolean canRead() + { + return baseChannel.canRead(); + } + + @Override + public Frame read() + { + final Frame frame = baseChannel.read(); + channelCounters.addFrame(partitionNumber, frame); + return frame; + } + + @Override + public ListenableFuture readabilityFuture() + { + return baseChannel.readabilityFuture(); + } + + @Override + public void close() + { + baseChannel.close(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java new file mode 100644 index 00000000000..1daf6ef32ef --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java @@ -0,0 +1,71 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.msq.counters.ChannelCounters; + +import javax.annotation.Nullable; +import java.io.IOException; + +public class CountingWritableFrameChannel implements WritableFrameChannel +{ + private final WritableFrameChannel baseChannel; + private final ChannelCounters channelCounters; + private final int partitionNumber; + + public CountingWritableFrameChannel( + final WritableFrameChannel baseChannel, + final ChannelCounters channelCounters, + final int partitionNumber + ) + { + this.baseChannel = baseChannel; + this.channelCounters = channelCounters; + this.partitionNumber = partitionNumber; + } + + @Override + public void write(FrameWithPartition frame) throws IOException + { + baseChannel.write(frame); + channelCounters.addFrame(partitionNumber, frame.frame()); + } + + @Override + public void fail(@Nullable Throwable cause) throws IOException + { + baseChannel.fail(cause); + } + + @Override + public void close() throws IOException + { + baseChannel.close(); + } + + @Override + public ListenableFuture writabilityFuture() + { + return baseChannel.writabilityFuture(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java new file mode 100644 index 00000000000..6b1a683ff0e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java @@ -0,0 +1,163 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class DataSourceMSQDestination implements MSQDestination +{ + static final String TYPE = "dataSource"; + + private final String dataSource; + private final Granularity segmentGranularity; + private final List segmentSortOrder; + + @Nullable + private final List replaceTimeChunks; + + @JsonCreator + public DataSourceMSQDestination( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segmentGranularity") Granularity segmentGranularity, + @JsonProperty("segmentSortOrder") @Nullable List segmentSortOrder, + @JsonProperty("replaceTimeChunks") @Nullable List replaceTimeChunks + ) + { + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.segmentGranularity = Preconditions.checkNotNull(segmentGranularity, "segmentGranularity"); + this.segmentSortOrder = segmentSortOrder != null ? segmentSortOrder : Collections.emptyList(); + this.replaceTimeChunks = replaceTimeChunks; + + if (replaceTimeChunks != null) { + // Verify that if replaceTimeChunks is provided, it is nonempty. + if (replaceTimeChunks.isEmpty()) { + throw new IAE("replaceTimeChunks must be null or nonempty; cannot be empty"); + } + + // Verify all provided time chunks are aligned with segmentGranularity. + for (final Interval interval : replaceTimeChunks) { + // ETERNITY gets a free pass. + if (!Intervals.ETERNITY.equals(interval)) { + final boolean startIsAligned = + segmentGranularity.bucketStart(interval.getStart()).equals(interval.getStart()); + + final boolean endIsAligned = + segmentGranularity.bucketStart(interval.getEnd()).equals(interval.getEnd()) + || segmentGranularity.increment(segmentGranularity.bucketStart(interval.getEnd())) + .equals(interval.getEnd()); + + if (!startIsAligned || !endIsAligned) { + throw new IAE( + "Time chunk [%s] provided in replaceTimeChunks is not aligned with segmentGranularity [%s]", + interval, + segmentGranularity + ); + } + } + } + } + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Granularity getSegmentGranularity() + { + return segmentGranularity; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public List getSegmentSortOrder() + { + return segmentSortOrder; + } + + /** + * Returns the list of time chunks to replace, or null if {@link #isReplaceTimeChunks()} is false. + * + * Invariants: if nonnull, then this will *also* be nonempty, and all intervals will be aligned + * with {@link #getSegmentGranularity()}. Each interval may comprise multiple time chunks. + */ + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getReplaceTimeChunks() + { + return replaceTimeChunks; + } + + /** + * Whether this object is in replace-existing-time-chunks mode. + */ + public boolean isReplaceTimeChunks() + { + return replaceTimeChunks != null; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DataSourceMSQDestination that = (DataSourceMSQDestination) o; + return Objects.equals(dataSource, that.dataSource) + && Objects.equals(segmentGranularity, that.segmentGranularity) + && Objects.equals(segmentSortOrder, that.segmentSortOrder) + && Objects.equals(replaceTimeChunks, that.replaceTimeChunks); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, segmentGranularity, segmentSortOrder, replaceTimeChunks); + } + + @Override + public String toString() + { + return "DataSourceMSQDestination{" + + "dataSource='" + dataSource + '\'' + + ", segmentGranularity=" + segmentGranularity + + ", segmentSortOrder=" + segmentSortOrder + + ", replaceTimeChunks=" + replaceTimeChunks + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java new file mode 100644 index 00000000000..2c249e26398 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java @@ -0,0 +1,182 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.rpc.IgnoreHttpResponseHandler; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutionException; + +public class IndexerControllerClient implements ControllerClient +{ + private final ServiceClient serviceClient; + private final ObjectMapper jsonMapper; + private final Closeable baggage; + + public IndexerControllerClient( + final ServiceClient serviceClient, + final ObjectMapper jsonMapper, + final Closeable baggage + ) + { + this.serviceClient = serviceClient; + this.jsonMapper = jsonMapper; + this.baggage = baggage; + } + + @Override + public void postKeyStatistics( + StageId stageId, + int workerNumber, + ClusterByStatisticsSnapshot keyStatistics + ) throws IOException + { + final String path = StringUtils.format( + "/keyStatistics/%s/%s/%d", + StringUtils.urlEncode(stageId.getQueryId()), + stageId.getStageNumber(), + workerNumber + ); + + doRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, keyStatistics), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public void postCounters(CounterSnapshotsTree snapshotsTree) throws IOException + { + doRequest( + new RequestBuilder(HttpMethod.POST, "/counters") + .jsonContent(jsonMapper, snapshotsTree), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public void postResultsComplete(StageId stageId, int workerNumber, @Nullable Object resultObject) throws IOException + { + final String path = StringUtils.format( + "/resultsComplete/%s/%s/%d", + StringUtils.urlEncode(stageId.getQueryId()), + stageId.getStageNumber(), + workerNumber + ); + + doRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, resultObject), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public void postWorkerError(String workerId, MSQErrorReport errorWrapper) throws IOException + { + final String path = StringUtils.format( + "/workerError/%s", + StringUtils.urlEncode(workerId) + ); + + doRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, errorWrapper), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public void postWorkerWarning(String workerId, List MSQErrorReports) throws IOException + { + final String path = StringUtils.format( + "/workerWarning/%s", + StringUtils.urlEncode(workerId) + ); + + doRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, MSQErrorReports), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public List getTaskList() throws IOException + { + final BytesFullResponseHolder retVal = doRequest( + new RequestBuilder(HttpMethod.GET, "/taskList"), + new BytesFullResponseHandler() + ); + + final MSQTaskList taskList = jsonMapper.readValue(retVal.getContent(), MSQTaskList.class); + return taskList.getTaskIds(); + } + + @Override + public void close() + { + try { + baggage.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Similar to {@link ServiceClient#request}, but preserves IOExceptions rather than wrapping them in + * {@link ExecutionException}. + */ + private FinalType doRequest( + RequestBuilder requestBuilder, + HttpResponseHandler handler + ) throws IOException + { + try { + return FutureUtils.get(serviceClient.asyncRequest(requestBuilder, handler), true); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + catch (ExecutionException e) { + throw new IOException(e.getCause()); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java new file mode 100644 index 00000000000..cc767626ecb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java @@ -0,0 +1,123 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Injector; +import com.google.inject.Key; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.exec.WorkerManagerClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.server.DruidNode; + +import java.util.Map; + +/** + * Implementation for {@link ControllerContext} required to run multi-stage queries as indexing tasks. + */ +public class IndexerControllerContext implements ControllerContext +{ + private final TaskToolbox toolbox; + private final Injector injector; + private final ServiceClientFactory clientFactory; + private final OverlordClient overlordClient; + private final WorkerManagerClient workerManager; + + public IndexerControllerContext( + final TaskToolbox toolbox, + final Injector injector, + final ServiceClientFactory clientFactory, + final OverlordClient overlordClient + ) + { + this.toolbox = toolbox; + this.injector = injector; + this.clientFactory = clientFactory; + this.overlordClient = overlordClient; + this.workerManager = new IndexerWorkerManagerClient(overlordClient); + } + + @Override + public ObjectMapper jsonMapper() + { + return toolbox.getJsonMapper(); + } + + @Override + public Injector injector() + { + return injector; + } + + @Override + public DruidNode selfNode() + { + return injector.getInstance(Key.get(DruidNode.class, Self.class)); + } + + @Override + public CoordinatorClient coordinatorClient() + { + return toolbox.getCoordinatorClient(); + } + + @Override + public TaskActionClient taskActionClient() + { + return toolbox.getTaskActionClient(); + } + + @Override + public WorkerClient taskClientFor(Controller controller) + { + // Ignore controller parameter. + return new IndexerWorkerClient(clientFactory, overlordClient, jsonMapper()); + } + + @Override + public void registerController(Controller controller, final Closer closer) + { + ChatHandler chatHandler = new ControllerChatHandler(toolbox, controller); + toolbox.getChatHandlerProvider().register(controller.id(), chatHandler, false); + closer.register(() -> toolbox.getChatHandlerProvider().unregister(controller.id())); + } + + @Override + public WorkerManagerClient workerManager() + { + return workerManager; + } + + @Override + public void writeReports(String controllerTaskId, Map reports) + { + toolbox.getTaskReportFileWriter().write(controllerTaskId, reports); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java new file mode 100644 index 00000000000..65e55545026 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -0,0 +1,120 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.loading.DataSegmentPusher; + +import java.io.File; + +public class IndexerFrameContext implements FrameContext +{ + private final IndexerWorkerContext context; + private final IndexIO indexIO; + private final DataSegmentProvider dataSegmentProvider; + private final WorkerMemoryParameters memoryParameters; + + public IndexerFrameContext( + IndexerWorkerContext context, + IndexIO indexIO, + DataSegmentProvider dataSegmentProvider, + WorkerMemoryParameters memoryParameters + ) + { + this.context = context; + this.indexIO = indexIO; + this.dataSegmentProvider = dataSegmentProvider; + this.memoryParameters = memoryParameters; + } + + @Override + public JoinableFactory joinableFactory() + { + return context.injector().getInstance(JoinableFactory.class); + } + + @Override + public GroupByStrategySelector groupByStrategySelector() + { + return context.injector().getInstance(GroupByStrategySelector.class); + } + + @Override + public RowIngestionMeters rowIngestionMeters() + { + return context.toolbox().getRowIngestionMetersFactory().createRowIngestionMeters(); + } + + @Override + public DataSegmentProvider dataSegmentProvider() + { + return dataSegmentProvider; + } + + @Override + public File tempDir() + { + return context.tempDir(); + } + + @Override + public ObjectMapper jsonMapper() + { + return context.jsonMapper(); + } + + @Override + public IndexIO indexIO() + { + return indexIO; + } + + @Override + public File persistDir() + { + return context.toolbox().getPersistDir(); + } + + @Override + public DataSegmentPusher segmentPusher() + { + return context.toolbox().getSegmentPusher(); + } + + @Override + public IndexMergerV9 indexMerger() + { + return context.toolbox().getIndexMergerV9(); + } + + @Override + public WorkerMemoryParameters memoryParameters() + { + return memoryParameters; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java new file mode 100644 index 00000000000..430c264291f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java @@ -0,0 +1,271 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; +import org.apache.druid.frame.file.FrameFileHttpResponseHandler; +import org.apache.druid.frame.file.FrameFilePartialFetch; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.rpc.IgnoreHttpResponseHandler; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.rpc.indexing.SpecificTaskRetryPolicy; +import org.apache.druid.rpc.indexing.SpecificTaskServiceLocator; +import org.apache.druid.utils.CloseableUtils; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import javax.annotation.Nonnull; +import javax.ws.rs.core.HttpHeaders; +import java.io.Closeable; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class IndexerWorkerClient implements WorkerClient +{ + private final ServiceClientFactory clientFactory; + private final OverlordClient overlordClient; + private final ObjectMapper jsonMapper; + + @GuardedBy("clientMap") + private final Map> clientMap = new HashMap<>(); + + public IndexerWorkerClient( + final ServiceClientFactory clientFactory, + final OverlordClient overlordClient, + final ObjectMapper jsonMapper + ) + { + this.clientFactory = clientFactory; + this.overlordClient = overlordClient; + this.jsonMapper = jsonMapper; + } + + + @Nonnull + public static String getStagePartitionPath(StageId stageId, int partitionNumber) + { + return StringUtils.format( + "/channels/%s/%d/%d", + StringUtils.urlEncode(stageId.getQueryId()), + stageId.getStageNumber(), + partitionNumber + ); + } + + @Override + public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workOrder) + { + return getClient(workerTaskId).asyncRequest( + new RequestBuilder(HttpMethod.POST, "/workOrder") + .jsonContent(jsonMapper, workOrder), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public ListenableFuture postResultPartitionBoundaries( + String workerTaskId, + StageId stageId, + ClusterByPartitions partitionBoundaries + ) + { + final String path = StringUtils.format( + "/resultPartitionBoundaries/%s/%d", + StringUtils.urlEncode(stageId.getQueryId()), + stageId.getStageNumber() + ); + + return getClient(workerTaskId).asyncRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, partitionBoundaries), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + /** + * Client-side method for {@link WorkerChatHandler#httpPostCleanupStage}. + */ + @Override + public ListenableFuture postCleanupStage( + final String workerTaskId, + final StageId stageId + ) + { + final String path = StringUtils.format( + "/cleanupStage/%s/%d", + StringUtils.urlEncode(stageId.getQueryId()), + stageId.getStageNumber() + ); + + return getClient(workerTaskId).asyncRequest( + new RequestBuilder(HttpMethod.POST, path), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public ListenableFuture postFinish(String workerTaskId) + { + return getClient(workerTaskId).asyncRequest( + new RequestBuilder(HttpMethod.POST, "/finish"), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + @Override + public ListenableFuture getCounters(String workerTaskId) + { + return FutureUtils.transform( + getClient(workerTaskId).asyncRequest( + new RequestBuilder(HttpMethod.GET, "/counters"), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, new TypeReference() {}) + ); + } + + private static final Logger log = new Logger(IndexerWorkerClient.class); + + @Override + public ListenableFuture fetchChannelData( + String workerTaskId, + StageId stageId, + int partitionNumber, + long offset, + ReadableByteChunksFrameChannel channel + ) + { + final ServiceClient client = getClient(workerTaskId); + final String path = getStagePartitionPath(stageId, partitionNumber); + + final SettableFuture retVal = SettableFuture.create(); + final ListenableFuture clientFuture = + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, StringUtils.format("%s?offset=%d", path, offset)) + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), // Data is compressed at app level + new FrameFileHttpResponseHandler(channel) + ); + + Futures.addCallback( + clientFuture, + new FutureCallback() + { + @Override + public void onSuccess(FrameFilePartialFetch partialFetch) + { + if (partialFetch.isExceptionCaught()) { + // Exception while reading channel. Recoverable. + log.noStackTrace().info( + partialFetch.getExceptionCaught(), + "Encountered exception while reading channel [%s]", + channel.getId() + ); + } + + // Empty fetch means this is the last fetch for the channel. + partialFetch.backpressureFuture().addListener( + () -> retVal.set(partialFetch.isLastFetch()), + Execs.directExecutor() + ); + } + + @Override + public void onFailure(Throwable t) + { + retVal.setException(t); + } + } + ); + + return retVal; + } + + @Override + public void close() throws IOException + { + synchronized (clientMap) { + try { + final List closeables = + clientMap.values().stream().map(pair -> pair.rhs).collect(Collectors.toList()); + CloseableUtils.closeAll(closeables); + } + finally { + clientMap.clear(); + } + } + } + + private ServiceClient getClient(final String workerTaskId) + { + synchronized (clientMap) { + return clientMap.computeIfAbsent( + workerTaskId, + id -> { + final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(id, overlordClient); + final ServiceClient client = clientFactory.makeClient( + id, + locator, + new SpecificTaskRetryPolicy(workerTaskId, StandardRetryPolicy.unlimited()) + ); + return Pair.of(client, locator); + } + ).lhs; + } + } + + /** + * Deserialize a {@link BytesFullResponseHolder} as JSON. + * + * It would be reasonable to move this to {@link BytesFullResponseHolder} itself, or some shared utility class. + */ + private T deserialize(final BytesFullResponseHolder bytesHolder, final TypeReference typeReference) + { + try { + return jsonMapper.readValue(bytesHolder.getContent(), typeReference); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java new file mode 100644 index 00000000000..0536785562a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -0,0 +1,296 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Injector; +import com.google.inject.Key; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.exec.TaskDataSegmentProvider; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.exec.WorkerContext; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.input.InputSpecs; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.rpc.CoordinatorServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocations; +import org.apache.druid.rpc.ServiceLocator; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.rpc.indexing.SpecificTaskRetryPolicy; +import org.apache.druid.rpc.indexing.SpecificTaskServiceLocator; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; +import org.apache.druid.server.DruidNode; + +import java.io.File; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; + +public class IndexerWorkerContext implements WorkerContext +{ + private static final Logger log = new Logger(IndexerWorkerContext.class); + private static final long FREQUENCY_CHECK_MILLIS = 1000; + private static final long FREQUENCY_CHECK_JITTER = 30; + + private final TaskToolbox toolbox; + private final Injector injector; + private final IndexIO indexIO; + private final TaskDataSegmentProvider dataSegmentProvider; + private final ServiceClientFactory clientFactory; + + @GuardedBy("this") + private OverlordClient overlordClient; + + @GuardedBy("this") + private ServiceLocator controllerLocator; + + public IndexerWorkerContext( + final TaskToolbox toolbox, + final Injector injector, + final IndexIO indexIO, + final TaskDataSegmentProvider dataSegmentProvider, + final ServiceClientFactory clientFactory + ) + { + this.toolbox = toolbox; + this.injector = injector; + this.indexIO = indexIO; + this.dataSegmentProvider = dataSegmentProvider; + this.clientFactory = clientFactory; + } + + public static IndexerWorkerContext createProductionInstance(final TaskToolbox toolbox, final Injector injector) + { + final IndexIO indexIO = injector.getInstance(IndexIO.class); + final CoordinatorServiceClient coordinatorServiceClient = + injector.getInstance(CoordinatorServiceClient.class).withRetryPolicy(StandardRetryPolicy.unlimited()); + final SegmentCacheManager segmentCacheManager = + injector.getInstance(SegmentCacheManagerFactory.class) + .manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch")); + final ServiceClientFactory serviceClientFactory = + injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); + + return new IndexerWorkerContext( + toolbox, + injector, + indexIO, + new TaskDataSegmentProvider(coordinatorServiceClient, segmentCacheManager, indexIO), + serviceClientFactory + ); + } + + public TaskToolbox toolbox() + { + return toolbox; + } + + @Override + public ObjectMapper jsonMapper() + { + return toolbox.getJsonMapper(); + } + + @Override + public Injector injector() + { + return injector; + } + + @Override + public void registerWorker(Worker worker, Closer closer) + { + WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + toolbox.getChatHandlerProvider().register(worker.id(), chatHandler, false); + closer.register(() -> toolbox.getChatHandlerProvider().unregister(worker.id())); + closer.register(() -> { + synchronized (this) { + if (controllerLocator != null) { + controllerLocator.close(); + } + } + }); + + // Register the periodic controller checker + final ExecutorService periodicControllerCheckerExec = Execs.singleThreaded("controller-status-checker-%s"); + closer.register(periodicControllerCheckerExec::shutdownNow); + final ServiceLocator controllerLocator = makeControllerLocator(worker.task().getControllerTaskId()); + periodicControllerCheckerExec.submit(() -> controllerCheckerRunnable(controllerLocator, worker)); + } + + @VisibleForTesting + void controllerCheckerRunnable(final ServiceLocator controllerLocator, final Worker worker) + { + while (true) { + // Add some randomness to the frequency of the loop to avoid requests from simultaneously spun up tasks bunching + // up and stagger them randomly + long sleepTimeMillis = FREQUENCY_CHECK_MILLIS + ThreadLocalRandom.current().nextLong( + -FREQUENCY_CHECK_JITTER, + 2 * FREQUENCY_CHECK_JITTER + ); + final ServiceLocations controllerLocations; + try { + controllerLocations = controllerLocator.locate().get(); + } + catch (Throwable e) { + // Service locator exceptions are not recoverable. + log.noStackTrace().warn( + e, + "Periodic fetch of controller location encountered an exception. Worker task [%s] will exit.", + worker.id() + ); + worker.controllerFailed(); + break; + } + + if (controllerLocations.isClosed() || controllerLocations.getLocations().isEmpty()) { + log.warn( + "Periodic fetch of controller location returned [%s]. Worker task [%s] will exit.", + controllerLocations, + worker.id() + ); + worker.controllerFailed(); + break; + } + + try { + Thread.sleep(sleepTimeMillis); + } + catch (InterruptedException ignored) { + // Do nothing: an interrupt means we were shut down. Status checker should exit quietly. + } + } + } + + @Override + public File tempDir() + { + return toolbox.getIndexingTmpDir(); + } + + @Override + public ControllerClient makeControllerClient(String controllerId) + { + final ServiceLocator locator = makeControllerLocator(controllerId); + + return new IndexerControllerClient( + clientFactory.makeClient( + controllerId, + locator, + new SpecificTaskRetryPolicy(controllerId, StandardRetryPolicy.unlimited()) + ), + jsonMapper(), + locator + ); + } + + @Override + public WorkerClient makeWorkerClient() + { + // Ignore workerId parameter. The workerId is passed into each method of WorkerClient individually. + return new IndexerWorkerClient(clientFactory, makeOverlordClient(), jsonMapper()); + } + + @Override + public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) + { + final int numWorkersInJvm; + + // Determine the max number of workers in JVM for memory allocations. + if (toolbox.getAppenderatorsManager() instanceof UnifiedIndexerAppenderatorsManager) { + // CliIndexer + numWorkersInJvm = injector.getInstance(WorkerConfig.class).getCapacity(); + } else { + // CliPeon + numWorkersInJvm = 1; + } + + final IntSet inputStageNumbers = + InputSpecs.getStageNumbers(queryDef.getStageDefinition(stageNumber).getInputSpecs()); + final int numInputWorkers = + inputStageNumbers.intStream() + .map(inputStageNumber -> queryDef.getStageDefinition(inputStageNumber).getMaxWorkerCount()) + .sum(); + + return new IndexerFrameContext( + this, + indexIO, + dataSegmentProvider, + WorkerMemoryParameters.compute( + Runtime.getRuntime().maxMemory(), + numWorkersInJvm, + processorBouncer().getMaxCount(), + numInputWorkers + ) + ); + } + + @Override + public int threadCount() + { + return processorBouncer().getMaxCount(); + } + + @Override + public DruidNode selfNode() + { + return injector.getInstance(Key.get(DruidNode.class, Self.class)); + } + + @Override + public Bouncer processorBouncer() + { + return injector.getInstance(Bouncer.class); + } + + private synchronized OverlordClient makeOverlordClient() + { + if (overlordClient == null) { + overlordClient = injector.getInstance(OverlordClient.class) + .withRetryPolicy(StandardRetryPolicy.unlimited()); + } + return overlordClient; + } + + private synchronized ServiceLocator makeControllerLocator(final String controllerId) + { + if (controllerLocator == null) { + controllerLocator = new SpecificTaskServiceLocator(controllerId, makeOverlordClient()); + } + + return controllerLocator; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java new file mode 100644 index 00000000000..ac1e71a5525 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java @@ -0,0 +1,81 @@ +/* + * 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.druid.msq.indexing; + +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.msq.exec.WorkerManagerClient; +import org.apache.druid.rpc.indexing.OverlordClient; + +import java.util.Map; +import java.util.Set; + +/** + * Worker manager client backed by the Indexer service. Glues together + * three different mechanisms to provide the single multi-stage query interface. + */ +public class IndexerWorkerManagerClient implements WorkerManagerClient +{ + private final OverlordClient overlordClient; + + public IndexerWorkerManagerClient(final OverlordClient overlordClient) + { + this.overlordClient = overlordClient; + } + + @Override + public String run(String controllerId, MSQWorkerTask task) + { + FutureUtils.getUnchecked(overlordClient.runTask(controllerId, task), true); + return controllerId; + } + + @Override + public void cancel(String taskId) + { + FutureUtils.getUnchecked(overlordClient.cancelTask(taskId), true); + } + + @Override + public Map statuses(Set taskIds) + { + return FutureUtils.getUnchecked(overlordClient.taskStatuses(taskIds), true); + } + + @Override + public TaskLocation location(String workerId) + { + final TaskStatusResponse response = FutureUtils.getUnchecked(overlordClient.taskStatus(workerId), true); + + if (response.getStatus() != null) { + return response.getStatus().getLocation(); + } else { + return TaskLocation.unknown(); + } + } + + @Override + public void close() + { + // Nothing to do. The OverlordServiceClient is closed by the JVM lifecycle. + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java new file mode 100644 index 00000000000..d6b02810c2c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.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.druid.msq.indexing; + +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.msq.kernel.StageId; + +import java.io.IOException; + +/** + * Creates {@link ReadableFrameChannel} to fetch frames corresponding to a particular stage and partition from the + * provided worker id + */ +public interface InputChannelFactory +{ + /** + * Given stageId, partitionNumber and workerNumber, this method opens the ReadableFrameChannel to fetch the + * corresponding frames + */ + ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException; +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java new file mode 100644 index 00000000000..1c01d480e33 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java @@ -0,0 +1,211 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.collect.Iterables; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.processor.FrameChannelMerger; +import org.apache.druid.frame.processor.FrameChannelMuxer; +import org.apache.druid.frame.processor.FrameProcessorExecutor; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.msq.input.stage.InputChannels; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.StagePartition; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +/** + * Implementation of {@link InputChannels}. + */ +public class InputChannelsImpl implements InputChannels +{ + private final QueryDefinition queryDefinition; + private final InputChannelFactory channelFactory; + private final Supplier allocatorMaker; + private final FrameProcessorExecutor exec; + private final String cancellationId; + private final Map readablePartitionMap; + + public InputChannelsImpl( + final QueryDefinition queryDefinition, + final ReadablePartitions readablePartitions, + final InputChannelFactory channelFactory, + final Supplier allocatorMaker, + final FrameProcessorExecutor exec, + final String cancellationId + ) + { + this.queryDefinition = queryDefinition; + this.readablePartitionMap = new HashMap<>(); + this.channelFactory = channelFactory; + this.allocatorMaker = allocatorMaker; + this.exec = exec; + this.cancellationId = cancellationId; + + for (final ReadablePartition readablePartition : readablePartitions) { + readablePartitionMap.put( + new StagePartition( + new StageId(queryDefinition.getQueryId(), readablePartition.getStageNumber()), + readablePartition.getPartitionNumber() + ), + readablePartition + ); + } + } + + @Override + public ReadableFrameChannel openChannel(final StagePartition stagePartition) throws IOException + { + final StageDefinition stageDef = queryDefinition.getStageDefinition(stagePartition.getStageId()); + final ReadablePartition readablePartition = readablePartitionMap.get(stagePartition); + final ClusterBy inputClusterBy = stageDef.getClusterBy(); + final boolean isSorted = inputClusterBy.getBucketByCount() != inputClusterBy.getColumns().size(); + + if (isSorted) { + return openSorted(stageDef, readablePartition); + } else { + return openUnsorted(stageDef, readablePartition); + } + } + + @Override + public FrameReader frameReader(final int stageNumber) + { + return queryDefinition.getStageDefinition(stageNumber).getFrameReader(); + } + + private ReadableFrameChannel openSorted( + final StageDefinition stageDefinition, + final ReadablePartition readablePartition + ) throws IOException + { + // Note: this method uses a single FrameChannelMerger, not a SuperSorter, for efficiency. (Currently, SuperSorter + // is always multi-level and always uses disk.) + final BlockingQueueFrameChannel queueChannel = BlockingQueueFrameChannel.minimal(); + + final List channels = openChannels( + stageDefinition.getId(), + readablePartition + ); + + if (channels.size() == 1) { + return Iterables.getOnlyElement(channels); + } else { + final FrameChannelMerger merger = new FrameChannelMerger( + channels, + stageDefinition.getFrameReader(), + queueChannel.writable(), + FrameWriters.makeFrameWriterFactory( + FrameType.ROW_BASED, + allocatorMaker.get(), + stageDefinition.getFrameReader().signature(), + + // No sortColumns, because FrameChannelMerger generates frames that are sorted all on its own + Collections.emptyList() + ), + stageDefinition.getClusterBy(), + null, + -1 + ); + + // Discard future, since there is no need to keep it. We aren't interested in its return value. If it fails, + // downstream processors are notified through fail(e) on in-memory channels. If we need to cancel it, we use + // the cancellationId. + exec.runFully(merger, cancellationId); + + return queueChannel.readable(); + } + } + + private ReadableFrameChannel openUnsorted( + final StageDefinition stageDefinition, + final ReadablePartition readablePartition + ) throws IOException + { + final List channels = openChannels( + stageDefinition.getId(), + readablePartition + ); + + if (channels.size() == 1) { + return Iterables.getOnlyElement(channels); + } else { + final BlockingQueueFrameChannel queueChannel = BlockingQueueFrameChannel.minimal(); + final FrameChannelMuxer muxer = new FrameChannelMuxer(channels, queueChannel.writable()); + + // Discard future, since there is no need to keep it. We aren't interested in its return value. If it fails, + // downstream processors are notified through fail(e) on in-memory channels. If we need to cancel it, we use + // the cancellationId. + exec.runFully(muxer, cancellationId); + + return queueChannel.readable(); + } + } + + private List openChannels( + final StageId stageId, + final ReadablePartition readablePartition + ) throws IOException + { + final List channels = new ArrayList<>(); + + try { + for (final int workerNumber : readablePartition.getWorkerNumbers()) { + channels.add( + channelFactory.openChannel( + stageId, + workerNumber, + readablePartition.getPartitionNumber() + ) + ); + } + + return channels; + } + catch (Exception e) { + // Close all channels opened so far before throwing the exception. + for (final ReadableFrameChannel channel : channels) { + try { + channel.close(); + } + catch (Exception e2) { + e.addSuppressed(e2); + } + } + + throw e; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java new file mode 100644 index 00000000000..af528f6ecbe --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java @@ -0,0 +1,183 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.primitives.Ints; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.datasketches.memory.Memory; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.allocation.MemoryRange; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.FrameComparisonWidget; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.read.FrameReaderUtils; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.sql.MSQTaskQueryMaker; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.function.IntSupplier; +import java.util.function.Supplier; + +public class KeyStatisticsCollectionProcessor implements FrameProcessor +{ + /** + * Constant chosen such that a column full of "standard" values, with row count + * {@link MSQTaskQueryMaker#DEFAULT_ROWS_PER_SEGMENT}, and *some* redundancy between + * rows (therefore: some "reasonable" compression) will not have any columns greater than 2GB in size. + */ + private static final int STANDARD_VALUE_SIZE = 1000; + + /** + * Constant chosen such that a segment full of "standard" rows, with row count + * {@link MSQTaskQueryMaker#DEFAULT_ROWS_PER_SEGMENT}, and *some* redundancy between + * rows (therefore: some "reasonable" compression) will not be larger than 5GB in size. + */ + private static final int STANDARD_ROW_SIZE = 2000; + + private final ReadableFrameChannel inputChannel; + private final WritableFrameChannel outputChannel; + private final FrameReader frameReader; + private final ClusterBy clusterBy; + + private ClusterByStatisticsCollector clusterByStatisticsCollector; + + public KeyStatisticsCollectionProcessor( + final ReadableFrameChannel inputChannel, + final WritableFrameChannel outputChannel, + final FrameReader frameReader, + final ClusterBy clusterBy, + final ClusterByStatisticsCollector clusterByStatisticsCollector + ) + { + this.inputChannel = inputChannel; + this.outputChannel = outputChannel; + this.frameReader = frameReader; + this.clusterBy = clusterBy; + this.clusterByStatisticsCollector = clusterByStatisticsCollector; + } + + @Override + public List inputChannels() + { + return Collections.singletonList(inputChannel); + } + + @Override + public List outputChannels() + { + return Collections.singletonList(outputChannel); + } + + @Override + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + { + if (readableInputs.isEmpty()) { + return ReturnOrAwait.awaitAll(1); + } + + if (inputChannel.isFinished()) { + return ReturnOrAwait.returnObject(clusterByStatisticsCollector); + } + + final Frame frame = inputChannel.read(); + final Cursor cursor = FrameProcessors.makeCursor(frame, frameReader); + final IntSupplier rowWeightSupplier = makeRowWeightSupplier(frameReader, cursor.getColumnSelectorFactory()); + final FrameComparisonWidget comparisonWidget = frameReader.makeComparisonWidget(frame, clusterBy.getColumns()); + + for (int i = 0; i < frame.numRows(); i++, cursor.advance()) { + final RowKey key = comparisonWidget.readKey(i); + clusterByStatisticsCollector.add(key, rowWeightSupplier.getAsInt()); + } + + // Clears partition info (uses NO_PARTITION), but that's OK, because it isn't needed downstream of this processor. + outputChannel.write(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION)); + return ReturnOrAwait.awaitAll(1); + } + + @Override + public void cleanup() throws IOException + { + FrameProcessors.closeAll( + inputChannels(), + outputChannels(), + () -> clusterByStatisticsCollector = null + ); + } + + private IntSupplier makeRowWeightSupplier( + final FrameReader frameReader, + final ColumnSelectorFactory columnSelectorFactory + ) + { + final Supplier> rowMemorySupplier = + FrameReaderUtils.makeRowMemorySupplier(columnSelectorFactory, frameReader.signature()); + + final int numFields = frameReader.signature().size(); + + if (rowMemorySupplier == null) { + // Can't access row memory. + throw new ISE("Can't read row memory from frame. Wrong frame type or signature?"); + } + + return () -> { + final MemoryRange rowMemory = rowMemorySupplier.get(); + + if (rowMemory == null) { + // Can't access row memory. + throw new ISE("Can't read row memory from frame. Wrong type or signature?"); + } + + long maxValueLength = 0; + long totalLength = 0; + long currentValueStartPosition = (long) Integer.BYTES * numFields; + + for (int i = 0; i < numFields; i++) { + final long currentValueEndPosition = rowMemory.memory().getInt(rowMemory.start() + (long) Integer.BYTES * i); + final long valueLength = currentValueEndPosition - currentValueStartPosition; + + if (valueLength > maxValueLength) { + maxValueLength = valueLength; + } + + totalLength += valueLength; + currentValueStartPosition = currentValueEndPosition; + } + + return 1 + Ints.checkedCast( + Math.max( + maxValueLength / STANDARD_VALUE_SIZE, + totalLength / STANDARD_ROW_SIZE + ) + ); + }; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java new file mode 100644 index 00000000000..ff4c8c19ed0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -0,0 +1,222 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.inject.Injector; +import com.google.inject.Key; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.exec.ControllerImpl; +import org.apache.druid.msq.exec.MSQTasks; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +@JsonTypeName(MSQControllerTask.TYPE) +public class MSQControllerTask extends AbstractTask +{ + public static final String TYPE = "query_controller"; + public static final String DUMMY_DATASOURCE_FOR_SELECT = "__query_select"; + + private final MSQSpec querySpec; + + // Enables users, and the web console, to see the original SQL query (if any). Not used by anything else in Druid. + @Nullable + private final String sqlQuery; + + // Enables users, and the web console, to see the original SQL context (if any). Not used by any other Druid logic. + @Nullable + private final Map sqlQueryContext; + + // Enables users, and the web console, to see the original SQL type names (if any). Not used by any other Druid logic. + @Nullable + private final List sqlTypeNames; + @Nullable + private final ExecutorService remoteFetchExecutorService; + + // Using an Injector directly because tasks do not have a way to provide their own Guice modules. + @JacksonInject + private Injector injector; + + private volatile Controller controller; + + @JsonCreator + public MSQControllerTask( + @JsonProperty("id") @Nullable String id, + @JsonProperty("spec") MSQSpec querySpec, + @JsonProperty("sqlQuery") @Nullable String sqlQuery, + @JsonProperty("sqlQueryContext") @Nullable Map sqlQueryContext, + @JsonProperty("sqlTypeNames") @Nullable List sqlTypeNames, + @JsonProperty("context") @Nullable Map context + ) + { + super( + id != null ? id : MSQTasks.controllerTaskId(null), + id, + null, + getDataSourceForTaskMetadata(querySpec), + context + ); + + this.querySpec = querySpec; + this.sqlQuery = sqlQuery; + this.sqlQueryContext = sqlQueryContext; + this.sqlTypeNames = sqlTypeNames; + + if (MultiStageQueryContext.isDurableStorageEnabled(querySpec.getQuery().getContext())) { + this.remoteFetchExecutorService = + Executors.newCachedThreadPool(Execs.makeThreadFactory(getId() + "-remote-fetcher-%d")); + } else { + this.remoteFetchExecutorService = null; + } + + addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); + } + + @Override + public String getType() + { + return TYPE; + } + + @JsonProperty("spec") + public MSQSpec getQuerySpec() + { + return querySpec; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getSqlQuery() + { + return sqlQuery; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getSqlQueryContext() + { + return sqlQueryContext; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getSqlTypeNames() + { + return sqlTypeNames; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + // If we're in replace mode, acquire locks for all intervals before declaring the task ready. + if (isIngestion(querySpec) && ((DataSourceMSQDestination) querySpec.getDestination()).isReplaceTimeChunks()) { + final List intervals = + ((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks(); + + for (final Interval interval : intervals) { + final TaskLock taskLock = + taskActionClient.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)); + + if (taskLock == null) { + return false; + } else if (taskLock.isRevoked()) { + throw new ISE(StringUtils.format("Lock for interval [%s] was revoked", interval)); + } + } + } + + return true; + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + final ServiceClientFactory clientFactory = + injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); + final OverlordClient overlordClient = injector.getInstance(OverlordClient.class) + .withRetryPolicy(StandardRetryPolicy.unlimited()); + final ControllerContext context = new IndexerControllerContext( + toolbox, + injector, + clientFactory, + overlordClient + ); + controller = new ControllerImpl(this, context); + return controller.run(); + } + + @Override + public void stopGracefully(final TaskConfig taskConfig) + { + if (controller != null) { + controller.stopGracefully(); + } + if (remoteFetchExecutorService != null) { + // This is to make sure we don't leak connections. + remoteFetchExecutorService.shutdownNow(); + } + } + + private static String getDataSourceForTaskMetadata(final MSQSpec querySpec) + { + final MSQDestination destination = querySpec.getDestination(); + + if (destination instanceof DataSourceMSQDestination) { + return ((DataSourceMSQDestination) destination).getDataSource(); + } else { + return DUMMY_DATASOURCE_FOR_SELECT; + } + } + + public static boolean isIngestion(final MSQSpec querySpec) + { + return querySpec.getDestination() instanceof DataSourceMSQDestination; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQDestination.java new file mode 100644 index 00000000000..d84a1635edc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQDestination.java @@ -0,0 +1,33 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = DataSourceMSQDestination.TYPE, value = DataSourceMSQDestination.class), + @JsonSubTypes.Type(name = TaskReportMSQDestination.TYPE, value = TaskReportMSQDestination.class) +}) +public interface MSQDestination +{ + // No methods. Just a marker interface for deserialization. +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQPartitionAssignment.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQPartitionAssignment.java new file mode 100644 index 00000000000..722d58787f1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQPartitionAssignment.java @@ -0,0 +1,96 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; + +import java.util.Map; +import java.util.Objects; + +public class MSQPartitionAssignment +{ + private final ClusterByPartitions partitions; + private final Map allocations; + + @JsonCreator + public MSQPartitionAssignment( + @JsonProperty("partitions") ClusterByPartitions partitions, + @JsonProperty("allocations") Map allocations + ) + { + this.partitions = Preconditions.checkNotNull(partitions, "partitions"); + this.allocations = Preconditions.checkNotNull(allocations, "allocations"); + + // Sanity checks. + for (final int partitionNumber : allocations.keySet()) { + if (partitionNumber < 0 || partitionNumber >= partitions.size()) { + throw new IAE("Partition [%s] out of bounds", partitionNumber); + } + } + } + + @JsonProperty + public ClusterByPartitions getPartitions() + { + return partitions; + } + + @JsonProperty + public Map getAllocations() + { + return allocations; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MSQPartitionAssignment that = (MSQPartitionAssignment) o; + return Objects.equals(partitions, that.partitions) && Objects.equals( + allocations, + that.allocations + ); + } + + @Override + public int hashCode() + { + return Objects.hash(partitions, allocations); + } + + @Override + public String toString() + { + return "MSQPartitionAssignment{" + + "partitions=" + partitions + + ", allocations=" + allocations + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSpec.java new file mode 100644 index 00000000000..b5f616b9640 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSpec.java @@ -0,0 +1,160 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.query.Query; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class MSQSpec +{ + private final Query query; + private final ColumnMappings columnMappings; + private final MSQDestination destination; + private final WorkerAssignmentStrategy assignmentStrategy; + private final MSQTuningConfig tuningConfig; + + @JsonCreator + public MSQSpec( + @JsonProperty("query") Query query, + @JsonProperty("columnMappings") @Nullable ColumnMappings columnMappings, + @JsonProperty("destination") MSQDestination destination, + @JsonProperty("assignmentStrategy") WorkerAssignmentStrategy assignmentStrategy, + @JsonProperty("tuningConfig") MSQTuningConfig tuningConfig + ) + { + this.query = Preconditions.checkNotNull(query, "query"); + this.columnMappings = Preconditions.checkNotNull(columnMappings, "columnMappings"); + this.destination = Preconditions.checkNotNull(destination, "destination"); + this.assignmentStrategy = Preconditions.checkNotNull(assignmentStrategy, "assignmentStrategy"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + } + + public static Builder builder() + { + return new Builder(); + } + + @JsonProperty + public Query getQuery() + { + return query; + } + + @JsonProperty("columnMappings") + public ColumnMappings getColumnMappings() + { + return columnMappings; + } + + @JsonProperty + public MSQDestination getDestination() + { + return destination; + } + + @JsonProperty + public WorkerAssignmentStrategy getAssignmentStrategy() + { + return assignmentStrategy; + } + + @JsonProperty + public MSQTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MSQSpec that = (MSQSpec) o; + return Objects.equals(query, that.query) + && Objects.equals(columnMappings, that.columnMappings) + && Objects.equals(destination, that.destination) + && Objects.equals(assignmentStrategy, that.assignmentStrategy) + && Objects.equals(tuningConfig, that.tuningConfig); + } + + @Override + public int hashCode() + { + return Objects.hash(query, columnMappings, destination, assignmentStrategy, tuningConfig); + } + + public static class Builder + { + private Query query; + private ColumnMappings columnMappings; + private MSQDestination destination = TaskReportMSQDestination.instance(); + private WorkerAssignmentStrategy assignmentStrategy = WorkerAssignmentStrategy.MAX; + private MSQTuningConfig tuningConfig; + + public Builder query(Query query) + { + this.query = query; + return this; + } + + public Builder columnMappings(final ColumnMappings columnMappings) + { + this.columnMappings = columnMappings; + return this; + } + + public Builder destination(final MSQDestination destination) + { + this.destination = destination; + return this; + } + + public Builder assignmentStrategy(final WorkerAssignmentStrategy assignmentStrategy) + { + this.assignmentStrategy = assignmentStrategy; + return this; + } + + public Builder tuningConfig(final MSQTuningConfig tuningConfig) + { + this.tuningConfig = tuningConfig; + return this; + } + + public MSQSpec build() + { + if (destination == null) { + destination = TaskReportMSQDestination.instance(); + } + + return new MSQSpec(query, columnMappings, destination, assignmentStrategy, tuningConfig); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTaskList.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTaskList.java new file mode 100644 index 00000000000..1d2a4e31d54 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTaskList.java @@ -0,0 +1,71 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Objects; + +public class MSQTaskList +{ + private final List taskIds; + + @JsonCreator + public MSQTaskList(@JsonProperty("taskIds") List taskIds) + { + this.taskIds = Preconditions.checkNotNull(taskIds, "taskIds"); + } + + @JsonProperty + public List getTaskIds() + { + return taskIds; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MSQTaskList that = (MSQTaskList) o; + return Objects.equals(taskIds, that.taskIds); + } + + @Override + public int hashCode() + { + return Objects.hash(taskIds); + } + + @Override + public String toString() + { + return "MSQTaskList{" + + "taskIds=" + taskIds + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java new file mode 100644 index 00000000000..391bb9d674d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQTuningConfig.java @@ -0,0 +1,140 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexer.partitions.PartitionsSpec; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Tuning parameters for multi-stage query tasks: {@link MSQControllerTask} and {@link MSQWorkerTask}. + */ +public class MSQTuningConfig +{ + /** + * Lower than {@link org.apache.druid.segment.indexing.TuningConfig#DEFAULT_MAX_ROWS_IN_MEMORY} to minimize the + * impact of per-row overheads that are not accounted for by OnheapIncrementalIndex. For example: overheads + * related to creating bitmaps during persist. + * + * If this value proves to work well, we'll update {@link org.apache.druid.segment.indexing.TuningConfig} to bring + * the two values in line with each other. + */ + private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 100000; + + /** + * One worker task. + */ + private static final int DEFAULT_MAX_NUM_TASKS = 1; + + @Nullable + private final Integer maxNumWorkers; + + @Nullable + private final Integer maxRowsInMemory; + + @Nullable + private final Integer rowsPerSegment; + + public MSQTuningConfig( + @JsonProperty("maxNumWorkers") @Nullable final Integer maxNumWorkers, + @JsonProperty("maxRowsInMemory") @Nullable final Integer maxRowsInMemory, + @JsonProperty("rowsPerSegment") @Nullable final Integer rowsPerSegment + ) + { + this.maxNumWorkers = maxNumWorkers; + this.maxRowsInMemory = maxRowsInMemory; + this.rowsPerSegment = rowsPerSegment; + } + + public static MSQTuningConfig defaultConfig() + { + return new MSQTuningConfig(null, null, null); + } + + @JsonProperty("maxNumWorkers") + @JsonInclude(JsonInclude.Include.NON_NULL) + Integer getMaxNumWorkersForSerialization() + { + return maxNumWorkers; + } + + @JsonProperty("maxRowsInMemory") + @JsonInclude(JsonInclude.Include.NON_NULL) + Integer getMaxRowsInMemoryForSerialization() + { + return maxRowsInMemory; + } + + @JsonProperty("rowsPerSegment") + @JsonInclude(JsonInclude.Include.NON_NULL) + Integer getRowsPerSegmentForSerialization() + { + return rowsPerSegment; + } + + public int getMaxNumWorkers() + { + return maxNumWorkers != null ? maxNumWorkers : DEFAULT_MAX_NUM_TASKS; + } + + public int getMaxRowsInMemory() + { + return maxRowsInMemory != null ? maxRowsInMemory : DEFAULT_MAX_ROWS_IN_MEMORY; + } + + public int getRowsPerSegment() + { + return rowsPerSegment != null ? rowsPerSegment : PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MSQTuningConfig that = (MSQTuningConfig) o; + return Objects.equals(maxNumWorkers, that.maxNumWorkers) + && Objects.equals(maxRowsInMemory, that.maxRowsInMemory) + && Objects.equals(rowsPerSegment, that.rowsPerSegment); + } + + @Override + public int hashCode() + { + return Objects.hash(maxNumWorkers, maxRowsInMemory, rowsPerSegment); + } + + @Override + public String toString() + { + return "MSQTuningConfig{" + + "maxNumWorkers=" + maxNumWorkers + + ", maxRowsInMemory=" + maxRowsInMemory + + ", rowsPerSegment=" + rowsPerSegment + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java new file mode 100644 index 00000000000..5ce5b229060 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java @@ -0,0 +1,115 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Injector; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.msq.exec.MSQTasks; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.exec.WorkerContext; +import org.apache.druid.msq.exec.WorkerImpl; + +import java.util.Map; + +@JsonTypeName(MSQWorkerTask.TYPE) +public class MSQWorkerTask extends AbstractTask +{ + public static final String TYPE = "query_worker"; + + private final String controllerTaskId; + private final int workerNumber; + + // Using an Injector directly because tasks do not have a way to provide their own Guice modules. + @JacksonInject + private Injector injector; + + private volatile Worker worker; + + @JsonCreator + @VisibleForTesting + public MSQWorkerTask( + @JsonProperty("controllerTaskId") final String controllerTaskId, + @JsonProperty("dataSource") final String dataSource, + @JsonProperty("workerNumber") final int workerNumber, + @JsonProperty("context") final Map context + ) + { + super( + MSQTasks.workerTaskId(controllerTaskId, workerNumber), + controllerTaskId, + null, + dataSource, + context + ); + + this.controllerTaskId = controllerTaskId; + this.workerNumber = workerNumber; + } + + @JsonProperty + public String getControllerTaskId() + { + return controllerTaskId; + } + + @JsonProperty + public int getWorkerNumber() + { + return workerNumber; + } + + @Override + public String getType() + { + return TYPE; + } + + + @Override + public boolean isReady(final TaskActionClient taskActionClient) + { + return true; + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + final WorkerContext context = IndexerWorkerContext.createProductionInstance(toolbox, injector); + worker = new WorkerImpl(this, context); + return worker.run(); + } + + @Override + public void stopGracefully(TaskConfig taskConfig) + { + if (worker != null) { + worker.stopGracefully(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java new file mode 100644 index 00000000000..15c260bdfb2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -0,0 +1,487 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.exec.ControllerImpl; +import org.apache.druid.msq.exec.WorkerManagerClient; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.error.WorkerFailedFault; +import org.apache.druid.msq.util.MultiStageQueryContext; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.OptionalLong; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.IntStream; + +/** + * Like {@link org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor}, but different. + */ +public class MSQWorkerTaskLauncher +{ + private static final Logger log = new Logger(MSQWorkerTaskLauncher.class); + private static final long HIGH_FREQUENCY_CHECK_MILLIS = 100; + private static final long LOW_FREQUENCY_CHECK_MILLIS = 2000; + private static final long SWITCH_TO_LOW_FREQUENCY_CHECK_AFTER_MILLIS = 10000; + private static final long SHUTDOWN_TIMEOUT_MS = Duration.ofMinutes(1).toMillis(); + + // States for "state" variable. + private enum State + { + NEW, + STARTED, + STOPPED + } + + private final String controllerTaskId; + private final String dataSource; + private final ControllerContext context; + private final ExecutorService exec; + private final long maxTaskStartDelayMillis; + private final boolean durableStageStorageEnabled; + + // Mutable state meant to be accessible by threads outside the main loop. + private final SettableFuture stopFuture = SettableFuture.create(); + private final AtomicReference state = new AtomicReference<>(State.NEW); + private final AtomicBoolean cancelTasksOnStop = new AtomicBoolean(); + + @GuardedBy("taskIds") + private int desiredTaskCount = 0; + + // Worker number -> task ID. + @GuardedBy("taskIds") + private final List taskIds = new ArrayList<>(); + + // Worker number -> whether the task has fully started up or not. + @GuardedBy("taskIds") + private final IntSet fullyStartedTasks = new IntOpenHashSet(); + + // Mutable state accessible only to the main loop. LinkedHashMap since order of key set matters. Tasks are added + // here once they are submitted for running, but before they are fully started up. + private final Map taskTrackers = new LinkedHashMap<>(); + + // Set of tasks which are issued a cancel request by the controller. + private final Set canceledWorkerTasks = ConcurrentHashMap.newKeySet(); + + public MSQWorkerTaskLauncher( + final String controllerTaskId, + final String dataSource, + final ControllerContext context, + final boolean durableStageStorageEnabled, + final long maxTaskStartDelayMillis + ) + { + this.controllerTaskId = controllerTaskId; + this.dataSource = dataSource; + this.context = context; + this.exec = Execs.singleThreaded( + "multi-stage-query-task-launcher[" + StringUtils.encodeForFormat(controllerTaskId) + "]-%s" + ); + this.durableStageStorageEnabled = durableStageStorageEnabled; + this.maxTaskStartDelayMillis = maxTaskStartDelayMillis; + } + + /** + * Launches tasks, blocking until they are all in RUNNING state. Returns a future that resolves successfully when + * all tasks end successfully or are canceled. The returned future resolves to an exception if one of the tasks fails + * without being explicitly canceled, or if something else goes wrong. + */ + public ListenableFuture start() + { + if (state.compareAndSet(State.NEW, State.STARTED)) { + exec.submit(() -> { + try { + mainLoop(); + } + catch (Throwable e) { + log.warn(e, "Error encountered in main loop. Abandoning worker tasks."); + } + }); + } + + // Return an "everything is done" future that callers can wait for. + return stopFuture; + } + + /** + * Stops all tasks, blocking until they exit. Returns quietly, no matter whether there was an exception + * associated with the future from {@link #start()} or not. + */ + public void stop(final boolean interrupt) + { + if (state.compareAndSet(State.STARTED, State.STOPPED)) { + if (interrupt) { + cancelTasksOnStop.set(true); + } + + synchronized (taskIds) { + // Wake up sleeping mainLoop. + taskIds.notifyAll(); + } + + // Only shutdown the executor when transitioning from STARTED. + exec.shutdown(); + } else if (state.get() == State.STOPPED) { + // interrupt = true is sticky: don't reset on interrupt = false. + if (interrupt) { + cancelTasksOnStop.set(true); + } + } else { + throw new ISE("Cannot stop(%s) from state [%s]", interrupt, state.get()); + } + + // Block until stopped. + try { + FutureUtils.getUnchecked(stopFuture, false); + } + catch (Throwable ignored) { + // Suppress. + } + } + + /** + * Get the list of currently-active tasks. + */ + public List getTaskList() + { + synchronized (taskIds) { + return ImmutableList.copyOf(taskIds); + } + } + + /** + * Launch additional tasks, if needed, to bring the size of {@link #taskIds} up to {@code taskCount}. If enough + * tasks are already running, this method does nothing. + */ + public void launchTasksIfNeeded(final int taskCount) throws InterruptedException + { + synchronized (taskIds) { + if (taskCount > desiredTaskCount) { + desiredTaskCount = taskCount; + } + + while (taskIds.size() < taskCount || !IntStream.range(0, taskCount).allMatch(fullyStartedTasks::contains)) { + if (stopFuture.isDone() || stopFuture.isCancelled()) { + FutureUtils.getUnchecked(stopFuture, false); + throw new ISE("Stopped"); + } + + taskIds.wait(); + } + } + } + + /** + * Checks if the controller has canceled the input taskId. This method is used in {@link ControllerImpl} + * to figure out if the worker taskId is canceled by the controller. If yes, the errors from that worker taskId + * are ignored for the error reports. + * + * @return true if task is canceled by the controller, else false + */ + public boolean isTaskCanceledByController(String taskId) + { + return canceledWorkerTasks.contains(taskId); + } + + private void mainLoop() + { + try { + Throwable caught = null; + + while (state.get() == State.STARTED) { + final long loopStartTime = System.currentTimeMillis(); + + try { + runNewTasks(); + updateTaskTrackersAndTaskIds(); + checkForErroneousTasks(); + } + catch (Throwable e) { + state.set(State.STOPPED); + cancelTasksOnStop.set(true); + caught = e; + break; + } + + // Sleep for a bit, maybe. + sleep(computeSleepTime(System.currentTimeMillis() - loopStartTime), false); + } + + // Only valid transition out of STARTED. + assert state.get() == State.STOPPED; + + final long stopStartTime = System.currentTimeMillis(); + + while (taskTrackers.values().stream().anyMatch(tracker -> !tracker.isComplete())) { + final long loopStartTime = System.currentTimeMillis(); + + if (cancelTasksOnStop.get()) { + shutDownTasks(); + } + + updateTaskTrackersAndTaskIds(); + + // Sleep for a bit, maybe. + final long now = System.currentTimeMillis(); + + if (now > stopStartTime + SHUTDOWN_TIMEOUT_MS) { + if (caught != null) { + throw caught; + } else { + throw new ISE("Task shutdown timed out (limit = %,dms)", SHUTDOWN_TIMEOUT_MS); + } + } + + sleep(computeSleepTime(now - loopStartTime), true); + } + + if (caught != null) { + throw caught; + } + + stopFuture.set(null); + } + catch (Throwable e) { + if (!stopFuture.isDone()) { + stopFuture.setException(e); + } + } + + synchronized (taskIds) { + // notify taskIds so launchWorkersIfNeeded can wake up, if it is sleeping, and notice stopFuture is done. + taskIds.notifyAll(); + } + } + + /** + * Used by the main loop to launch new tasks up to {@link #desiredTaskCount}. Adds trackers to {@link #taskTrackers} + * for newly launched tasks. + */ + private void runNewTasks() + { + final Map taskContext = new HashMap<>(); + + if (durableStageStorageEnabled) { + taskContext.put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, true); + } + + final int firstTask; + final int taskCount; + + synchronized (taskIds) { + firstTask = taskIds.size(); + taskCount = desiredTaskCount; + } + + for (int i = firstTask; i < taskCount; i++) { + final MSQWorkerTask task = new MSQWorkerTask( + controllerTaskId, + dataSource, + i, + taskContext + ); + + taskTrackers.put(task.getId(), new TaskTracker(i)); + context.workerManager().run(task.getId(), task); + + synchronized (taskIds) { + taskIds.add(task.getId()); + taskIds.notifyAll(); + } + } + } + + /** + * Used by the main loop to update {@link #taskTrackers} and {@link #fullyStartedTasks}. + */ + private void updateTaskTrackersAndTaskIds() + { + final Set taskStatusesNeeded = new HashSet<>(); + for (final Map.Entry taskEntry : taskTrackers.entrySet()) { + if (!taskEntry.getValue().isComplete()) { + taskStatusesNeeded.add(taskEntry.getKey()); + } + } + + if (!taskStatusesNeeded.isEmpty()) { + final WorkerManagerClient workerManager = context.workerManager(); + final Map statuses = workerManager.statuses(taskStatusesNeeded); + + for (Map.Entry statusEntry : statuses.entrySet()) { + final String taskId = statusEntry.getKey(); + final TaskTracker tracker = taskTrackers.get(taskId); + tracker.status = statusEntry.getValue(); + + if (!tracker.status.getStatusCode().isComplete() && tracker.unknownLocation()) { + // Look up location if not known. Note: this location is not used to actually contact the task. For that, + // we have SpecificTaskServiceLocator. This location is only used to determine if a task has started up. + tracker.initialLocation = workerManager.location(taskId); + } + + if (tracker.status.getStatusCode() == TaskState.RUNNING && !tracker.unknownLocation()) { + synchronized (taskIds) { + fullyStartedTasks.add(tracker.workerNumber); + taskIds.notifyAll(); + } + } + } + } + } + + /** + * Used by the main loop to generate exceptions if any tasks have failed, have taken too long to start up, or + * have gone inexplicably missing. + * + * Throws an exception if some task is erroneous. + */ + private void checkForErroneousTasks() + { + final int numTasks = taskTrackers.size(); + + for (final Map.Entry taskEntry : taskTrackers.entrySet()) { + final String taskId = taskEntry.getKey(); + final TaskTracker tracker = taskEntry.getValue(); + + if (tracker.status == null) { + throw new MSQException(UnknownFault.forMessage(StringUtils.format("Task [%s] status missing", taskId))); + } + + if (tracker.didRunTimeOut(maxTaskStartDelayMillis) && !canceledWorkerTasks.contains(taskId)) { + throw new MSQException(new TaskStartTimeoutFault(numTasks + 1)); + } + + if (tracker.didFail() && !canceledWorkerTasks.contains(taskId)) { + throw new MSQException(new WorkerFailedFault(taskId, tracker.status.getErrorMsg())); + } + } + } + + private void shutDownTasks() + { + for (final Map.Entry taskEntry : taskTrackers.entrySet()) { + final String taskId = taskEntry.getKey(); + final TaskTracker tracker = taskEntry.getValue(); + if (!canceledWorkerTasks.contains(taskId) + && (tracker.status == null || !tracker.status.getStatusCode().isComplete())) { + canceledWorkerTasks.add(taskId); + context.workerManager().cancel(taskId); + } + } + } + + /** + * Used by the main loop to decide how often to check task status. + */ + private long computeSleepTime(final long loopDurationMs) + { + final OptionalLong maxTaskStartTime = + taskTrackers.values().stream().mapToLong(tracker -> tracker.startTimeMs).max(); + + if (maxTaskStartTime.isPresent() && + System.currentTimeMillis() - maxTaskStartTime.getAsLong() < SWITCH_TO_LOW_FREQUENCY_CHECK_AFTER_MILLIS) { + return HIGH_FREQUENCY_CHECK_MILLIS - loopDurationMs; + } else { + return LOW_FREQUENCY_CHECK_MILLIS - loopDurationMs; + } + } + + private void sleep(final long sleepMillis, final boolean shuttingDown) throws InterruptedException + { + if (sleepMillis > 0) { + if (shuttingDown) { + Thread.sleep(sleepMillis); + } else { + // wait on taskIds so we can wake up early if needed. + synchronized (taskIds) { + taskIds.wait(sleepMillis); + } + } + } else { + // No wait, but check interrupted status anyway. + if (Thread.interrupted()) { + throw new InterruptedException(); + } + } + } + + /** + * Tracker for information about a worker. Mutable. + */ + private static class TaskTracker + { + private final int workerNumber; + private final long startTimeMs = System.currentTimeMillis(); + private TaskStatus status; + private TaskLocation initialLocation; + + public TaskTracker(int workerNumber) + { + this.workerNumber = workerNumber; + } + + public boolean unknownLocation() + { + return initialLocation == null || TaskLocation.unknown().equals(initialLocation); + } + + public boolean isComplete() + { + return status != null && status.getStatusCode().isComplete(); + } + + public boolean didFail() + { + return status != null && status.getStatusCode().isFailure(); + } + + public boolean didRunTimeOut(final long maxTaskStartDelayMillis) + { + return (status == null || status.getStatusCode() == TaskState.RUNNING) + && unknownLocation() + && System.currentTimeMillis() - startTimeMs > maxTaskStartDelayMillis; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessor.java new file mode 100644 index 00000000000..36400ef2667 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessor.java @@ -0,0 +1,295 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.ListenableFuture; +import it.unimi.dsi.fastutil.ints.IntSet; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.Rows; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.segment.FrameStorageAdapter; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.MSQTasks; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.util.SequenceUtils; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +public class SegmentGeneratorFrameProcessor implements FrameProcessor +{ + private static final Logger log = new Logger(SegmentGeneratorFrameProcessor.class); + + private final ReadableFrameChannel inChannel; + private final FrameReader frameReader; + private final Appenderator appenderator; + private final SegmentIdWithShardSpec segmentIdWithShardSpec; + private final List dimensionsForInputRows; + private final Object2IntMap outputColumnNameToFrameColumnNumberMap; + + private boolean firstRun = true; + private long rowsWritten = 0L; + + SegmentGeneratorFrameProcessor( + final ReadableInput readableInput, + final ColumnMappings columnMappings, + final List dimensionsForInputRows, + final Appenderator appenderator, + final SegmentIdWithShardSpec segmentIdWithShardSpec + ) + { + this.inChannel = readableInput.getChannel(); + this.frameReader = readableInput.getChannelFrameReader(); + this.appenderator = appenderator; + this.segmentIdWithShardSpec = segmentIdWithShardSpec; + this.dimensionsForInputRows = dimensionsForInputRows; + + outputColumnNameToFrameColumnNumberMap = new Object2IntOpenHashMap<>(); + outputColumnNameToFrameColumnNumberMap.defaultReturnValue(-1); + + for (final ColumnMapping columnMapping : columnMappings.getMappings()) { + outputColumnNameToFrameColumnNumberMap.put( + columnMapping.getOutputColumn(), + frameReader.signature().indexOf(columnMapping.getQueryColumn()) + ); + } + } + + @Override + public List inputChannels() + { + return Collections.singletonList(inChannel); + } + + @Override + public List outputChannels() + { + return Collections.emptyList(); + } + + @Override + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws InterruptedException + { + if (firstRun) { + log.debug("Starting job for segment [%s].", segmentIdWithShardSpec.asSegmentId()); + appenderator.startJob(); + firstRun = false; + } + + if (readableInputs.isEmpty()) { + return ReturnOrAwait.awaitAll(1); + } + + if (inChannel.isFinished()) { + if (rowsWritten == 0) { + log.debug("Finished reading. No data for segment [%s], skipping.", segmentIdWithShardSpec.asSegmentId()); + return ReturnOrAwait.returnObject(null); + } else { + log.debug("Finished reading. Pushing segment [%s].", segmentIdWithShardSpec.asSegmentId()); + + // This is a blocking action which violates the FrameProcessor's contract. + // useUniquePath = false because this class is meant to be used by batch jobs. + final ListenableFuture pushFuture = + appenderator.push(Collections.singletonList(segmentIdWithShardSpec), null, false); + + final SegmentsAndCommitMetadata metadata; + + try { + metadata = FutureUtils.get(pushFuture, true); + } + catch (ExecutionException e) { + throw new RuntimeException(e.getCause()); + } + + appenderator.clear(); + + log.debug("Finished work for segment [%s].", segmentIdWithShardSpec.asSegmentId()); + return ReturnOrAwait.returnObject(Iterables.getOnlyElement(metadata.getSegments())); + } + } else { + if (appenderator.getSegments().isEmpty()) { + log.debug("Received first frame for segment [%s].", segmentIdWithShardSpec.asSegmentId()); + } + + addFrame(inChannel.read()); + return ReturnOrAwait.awaitAll(1); + } + } + + @Override + public void cleanup() throws IOException + { + FrameProcessors.closeAll(inputChannels(), outputChannels(), appenderator::close); + } + + private void addFrame(final Frame frame) + { + final RowSignature signature = frameReader.signature(); + + // Reuse input row to avoid redoing allocations. + final MSQInputRow inputRow = new MSQInputRow(); + + final Sequence cursorSequence = + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) + .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null); + + SequenceUtils.forEach( + cursorSequence, + cursor -> { + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + //noinspection rawtypes + @SuppressWarnings("rawtypes") + final List selectors = + frameReader.signature() + .getColumnNames() + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); + + while (!cursor.isDone()) { + for (int j = 0; j < signature.size(); j++) { + inputRow.getBackingArray()[j] = selectors.get(j).getObject(); + } + + try { + rowsWritten++; + appenderator.add(segmentIdWithShardSpec, inputRow, null); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + cursor.advance(); + } + } + ); + } + + private class MSQInputRow implements InputRow + { + private final Object[] backingArray; + private final int timeColumnNumber = outputColumnNameToFrameColumnNumberMap.getInt(ColumnHolder.TIME_COLUMN_NAME); + + public MSQInputRow() + { + this.backingArray = new Object[frameReader.signature().size()]; + } + + @Override + public long getTimestampFromEpoch() + { + if (timeColumnNumber < 0) { + return 0; + } else { + return MSQTasks.primaryTimestampFromObjectForInsert(backingArray[timeColumnNumber]); + } + } + + @Override + public DateTime getTimestamp() + { + return DateTimes.utc(getTimestampFromEpoch()); + } + + @Override + public List getDimensions() + { + return dimensionsForInputRows; + } + + @Nullable + @Override + public Object getRaw(String columnName) + { + final int columnNumber = outputColumnNameToFrameColumnNumberMap.getInt(columnName); + if (columnNumber < 0) { + return null; + } else { + return backingArray[columnNumber]; + } + } + + @Override + public List getDimension(String columnName) + { + return Rows.objectToStrings(getRaw(columnName)); + } + + @Nullable + @Override + public Number getMetric(String columnName) + { + return Rows.objectToNumber(columnName, getRaw(columnName), true); + } + + @Override + public int compareTo(Row other) + { + // Not used during indexing. + throw new UnsupportedOperationException(); + } + + private Object[] getBackingArray() + { + return backingArray; + } + + @Override + public String toString() + { + return "MSQInputRow{" + + "backingArray=" + Arrays.toString(backingArray) + + ", timeColumnNumber=" + timeColumnNumber + + '}'; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessorFactory.java new file mode 100644 index 00000000000..6019f4bffd7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessorFactory.java @@ -0,0 +1,373 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.kernel.ExtraInfoHolder; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.FrameProcessorFactory; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.CompressionFactory; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.Consumer; + +@JsonTypeName("segmentGenerator") +public class SegmentGeneratorFrameProcessorFactory + implements FrameProcessorFactory, SegmentGeneratorFrameProcessor, DataSegment, Set> +{ + private final DataSchema dataSchema; + private final ColumnMappings columnMappings; + private final MSQTuningConfig tuningConfig; + + @JsonCreator + public SegmentGeneratorFrameProcessorFactory( + @JsonProperty("dataSchema") final DataSchema dataSchema, + @JsonProperty("columnMappings") final ColumnMappings columnMappings, + @JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig + ) + { + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.columnMappings = Preconditions.checkNotNull(columnMappings, "columnMappings"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public ColumnMappings getColumnMappings() + { + return columnMappings; + } + + @JsonProperty + public MSQTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @Override + public ProcessorsAndChannels makeProcessors( + StageDefinition stageDefinition, + int workerNumber, + List inputSlices, + InputSliceReader inputSliceReader, + @Nullable List extra, + OutputChannelFactory outputChannelFactory, + FrameContext frameContext, + int maxOutstandingProcessors, + CounterTracker counters, + Consumer warningPublisher + ) + { + final RowIngestionMeters meters = frameContext.rowIngestionMeters(); + + final ParseExceptionHandler parseExceptionHandler = new ParseExceptionHandler( + meters, + TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS, + TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS, + TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + ); + + // Expect a single input slice. + final InputSlice slice = Iterables.getOnlyElement(inputSlices); + final Sequence> inputSequence = + Sequences.simple(Iterables.transform( + inputSliceReader.attach(0, slice, counters, warningPublisher), + new Function>() + { + int i = 0; + + @Override + public Pair apply(ReadableInput readableInput) + { + return Pair.of(i++, readableInput); + } + } + )); + + final Sequence workers = inputSequence.map( + readableInputPair -> { + final StagePartition stagePartition = Preconditions.checkNotNull(readableInputPair.rhs.getStagePartition()); + final SegmentIdWithShardSpec segmentIdWithShardSpec = extra.get(readableInputPair.lhs); + final String idString = StringUtils.format("%s:%s", stagePartition, workerNumber); + final File persistDirectory = new File( + frameContext.persistDir(), + segmentIdWithShardSpec.asSegmentId().toString() + ); + + // Create directly, without using AppenderatorsManager, because we need different memory overrides due to + // using one Appenderator per processing thread instead of per task. + // Note: "createOffline" ignores the batchProcessingMode and always acts like CLOSED_SEGMENTS_SINKS. + final Appenderator appenderator = + Appenderators.createOffline( + idString, + dataSchema, + makeAppenderatorConfig( + tuningConfig, + persistDirectory, + frameContext.memoryParameters() + ), + new FireDepartmentMetrics(), // We should eventually expose the FireDepartmentMetrics + frameContext.segmentPusher(), + frameContext.jsonMapper(), + frameContext.indexIO(), + frameContext.indexMerger(), + meters, + parseExceptionHandler, + true + ); + + return new SegmentGeneratorFrameProcessor( + readableInputPair.rhs, + columnMappings, + dataSchema.getDimensionsSpec().getDimensionNames(), + appenderator, + segmentIdWithShardSpec + ); + } + ); + + return new ProcessorsAndChannels<>(workers, OutputChannels.none()); + } + + @Override + public TypeReference> getAccumulatedResultTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public Set newAccumulatedResult() + { + return new HashSet<>(); + } + + @Nullable + @Override + public Set accumulateResult(Set accumulated, DataSegment current) + { + if (current != null) { + accumulated.add(current); + } + + return accumulated; + } + + @Nullable + @Override + public Set mergeAccumulatedResult(Set accumulated, Set otherAccumulated) + { + accumulated.addAll(otherAccumulated); + return accumulated; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentGeneratorFrameProcessorFactory that = (SegmentGeneratorFrameProcessorFactory) o; + return Objects.equals(dataSchema, that.dataSchema) + && Objects.equals(columnMappings, that.columnMappings) + && Objects.equals(tuningConfig, that.tuningConfig); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSchema, columnMappings, tuningConfig); + } + + @Override + @SuppressWarnings("rawtypes") + public ExtraInfoHolder makeExtraInfoHolder(final List segmentIdsWithShardSpecs) + { + return new SegmentGeneratorExtraInfoHolder(segmentIdsWithShardSpecs); + } + + private static AppenderatorConfig makeAppenderatorConfig( + final MSQTuningConfig tuningConfig, + final File persistDirectory, + final WorkerMemoryParameters memoryParameters + ) + { + return new AppenderatorConfig() + { + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return TuningConfig.DEFAULT_APPENDABLE_INDEX; + } + + @Override + public int getMaxRowsInMemory() + { + // No need to apportion this amongst memoryParameters.getAppenderatorCount(), because it only exists + // to minimize the impact of per-row overheads that are not accounted for by OnheapIncrementalIndex's + // maxBytesInMemory handling. For example: overheads related to creating bitmaps during persist. + return tuningConfig.getMaxRowsInMemory(); + } + + @Override + public long getMaxBytesInMemory() + { + return memoryParameters.getAppenderatorMaxBytesInMemory(); + } + + @Override + public PartitionsSpec getPartitionsSpec() + { + // We're not actually doing dynamic partitioning. This segment generator is going to get exactly one segment. + return new DynamicPartitionsSpec(Integer.MAX_VALUE, Long.MAX_VALUE); + } + + @Override + public IndexSpec getIndexSpec() + { + return new IndexSpec(); + } + + @Override + public IndexSpec getIndexSpecForIntermediatePersists() + { + // Disable compression for intermediate persists to reduce direct memory usage. + return new IndexSpec( + null, + CompressionStrategy.UNCOMPRESSED, // Dimensions don't support NONE, so use UNCOMPRESSED + CompressionStrategy.NONE, // NONE is more efficient than UNCOMPRESSED + CompressionFactory.LongEncodingStrategy.LONGS, + null + ); + } + + @Override + public boolean isReportParseExceptions() + { + return true; + } + + @Override + public int getMaxPendingPersists() + { + return 0; + } + + @Override + public boolean isSkipBytesInMemoryOverheadCheck() + { + return TuningConfig.DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK; + } + + @Override + public Period getIntermediatePersistPeriod() + { + // Intermediate persist doesn't make much sense for batch jobs. + return new Period(Integer.MAX_VALUE); + } + + @Override + public File getBasePersistDirectory() + { + return persistDirectory; + } + + @Override + public AppenderatorConfig withBasePersistDirectory(final File basePersistDirectory) + { + // Not used. + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + // Default SegmentWriteOutMediumFactory. + return null; + } + + @Override + public int getMaxColumnsToMerge() + { + return memoryParameters.getAppenderatorMaxColumnsToMerge(); + } + }; + } + + @JsonTypeName("segmentGenerator") + public static class SegmentGeneratorExtraInfoHolder extends ExtraInfoHolder> + { + @JsonCreator + public SegmentGeneratorExtraInfoHolder(@Nullable @JsonProperty(INFO_KEY) final List extra) + { + super(extra); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/TaskReportMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/TaskReportMSQDestination.java new file mode 100644 index 00000000000..3038f5d65dd --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/TaskReportMSQDestination.java @@ -0,0 +1,45 @@ +/* + * 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.druid.msq.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; + +public class TaskReportMSQDestination implements MSQDestination +{ + public static final TaskReportMSQDestination INSTANCE = new TaskReportMSQDestination(); + static final String TYPE = "taskReport"; + + private TaskReportMSQDestination() + { + // Singleton. + } + + @JsonCreator + public static TaskReportMSQDestination instance() + { + return INSTANCE; + } + + @Override + public String toString() + { + return "TaskReportMSQDestination{}"; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java new file mode 100644 index 00000000000..06fd8f36caf --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java @@ -0,0 +1,222 @@ +/* + * 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.druid.msq.indexing; + +import it.unimi.dsi.fastutil.bytes.ByteArrays; +import org.apache.commons.lang.mutable.MutableLong; +import org.apache.druid.frame.file.FrameFileHttpResponseHandler; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.segment.realtime.firehose.ChatHandlers; +import org.apache.druid.server.security.Action; +import org.apache.druid.utils.CloseableUtils; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.StreamingOutput; +import java.io.IOException; +import java.io.InputStream; + +public class WorkerChatHandler implements ChatHandler +{ + private static final Logger log = new Logger(WorkerChatHandler.class); + + /** + * Callers must be able to store an entire chunk in memory. It can't be too large. + */ + private static final long CHANNEL_DATA_CHUNK_SIZE = 1_000_000; + + private final Worker worker; + private final MSQWorkerTask task; + private final TaskToolbox toolbox; + + public WorkerChatHandler(TaskToolbox toolbox, Worker worker) + { + this.worker = worker; + this.task = worker.task(); + this.toolbox = toolbox; + } + + /** + * Returns up to {@link #CHANNEL_DATA_CHUNK_SIZE} bytes of stage output data. + * + * See {@link org.apache.druid.msq.exec.WorkerClient#fetchChannelData} for the client-side code that calls this API. + */ + @GET + @Path("/channels/{queryId}/{stageNumber}/{partitionNumber}") + @Produces(MediaType.APPLICATION_OCTET_STREAM) + public Response httpGetChannelData( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("partitionNumber") final int partitionNumber, + @QueryParam("offset") final long offset, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + try { + final InputStream inputStream = worker.readChannel(queryId, stageNumber, partitionNumber, offset); + if (inputStream == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + final Response.ResponseBuilder responseBuilder = Response.ok(); + + final byte[] readBuf = new byte[8192]; + final MutableLong bytesReadTotal = new MutableLong(0L); + final int firstRead = inputStream.read(readBuf); + + if (firstRead == -1) { + // Empty read means we're at the end of the channel. Set the last fetch header so the client knows this. + inputStream.close(); + return responseBuilder + .header( + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME, + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE + ) + .entity(ByteArrays.EMPTY_ARRAY) + .build(); + } + + return Response.ok((StreamingOutput) output -> { + try { + int bytesReadThisCall = firstRead; + do { + final int bytesToWrite = + (int) Math.min(CHANNEL_DATA_CHUNK_SIZE - bytesReadTotal.longValue(), bytesReadThisCall); + output.write(readBuf, 0, bytesToWrite); + bytesReadTotal.add(bytesReadThisCall); + } while (bytesReadTotal.longValue() < CHANNEL_DATA_CHUNK_SIZE + && (bytesReadThisCall = inputStream.read(readBuf)) != -1); + } + catch (Throwable e) { + // Suppress the exception to ensure nothing gets written over the wire once we've sent a 200. The client + // will resume from where it left off. + log.noStackTrace().warn( + e, + "Error writing channel for query [%s] stage [%s] partition [%s] offset [%,d] to [%s]", + queryId, + stageNumber, + partitionNumber, + offset, + req.getRemoteAddr() + ); + } + finally { + CloseableUtils.closeAll(inputStream, output); + } + }).build(); + } + catch (IOException e) { + return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); + } + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postWorkOrder} for the client-side code that calls this API. + */ + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/workOrder") + public Response httpPostWorkOrder(final WorkOrder workOrder, @Context final HttpServletRequest req) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + worker.postWorkOrder(workOrder); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postResultPartitionBoundaries} for the client-side code that calls this API. + */ + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/resultPartitionBoundaries/{queryId}/{stageNumber}") + public Response httpPostResultPartitionBoundaries( + final ClusterByPartitions stagePartitionBoundaries, + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + if (worker.postResultPartitionBoundaries(stagePartitionBoundaries, queryId, stageNumber)) { + return Response.status(Response.Status.ACCEPTED).build(); + } else { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API. + */ + @POST + @Path("/cleanupStage/{queryId}/{stageNumber}") + public Response httpPostCleanupStage( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + worker.postCleanupStage(new StageId(queryId, stageNumber)); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postFinish} for the client-side code that calls this API. + */ + @POST + @Path("/finish") + public Response httpPostFinish(@Context final HttpServletRequest req) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + worker.postFinish(); + return Response.status(Response.Status.ACCEPTED).build(); + } + + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#getCounters} for the client-side code that calls this API. + */ + @GET + @Produces(MediaType.APPLICATION_JSON) + @Path("/counters") + public Response httpGetCounters(@Context final HttpServletRequest req) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); + return Response.status(Response.Status.OK).entity(worker.getCounters()).build(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java new file mode 100644 index 00000000000..a510852c95a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java @@ -0,0 +1,121 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Base implementation of {@link MSQFault}. + * + * Implements {@link #equals}, {@link #hashCode()}, and {@link #toString()} using {@link #errorCode} and + * {@link #errorMessage}, so faults must either encode all relevant information in the message, or provide + * their own implementation of these methods. + */ +public abstract class BaseMSQFault implements MSQFault +{ + private final String errorCode; + + @Nullable + private final String errorMessage; + + BaseMSQFault(final String errorCode, @Nullable final String errorMessage) + { + this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode"); + this.errorMessage = errorMessage; + } + + BaseMSQFault( + final String errorCode, + final String errorMessageFormat, + final Object errorMessageFirstArg, + final Object... errorMessageOtherArgs + ) + { + this(errorCode, format(errorMessageFormat, errorMessageFirstArg, errorMessageOtherArgs)); + } + + BaseMSQFault(final String errorCode) + { + this(errorCode, null); + } + + @Override + public String getErrorCode() + { + return errorCode; + } + + @Override + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getErrorMessage() + { + return errorMessage; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BaseMSQFault that = (BaseMSQFault) o; + return Objects.equals(errorCode, that.errorCode) && Objects.equals(errorMessage, that.errorMessage); + } + + @Override + public int hashCode() + { + return Objects.hash(errorCode, errorMessage); + } + + @Override + public String toString() + { + return getCodeWithMessage(); + } + + private static String format( + final String formatString, + final Object firstArg, + final Object... otherArgs + ) + { + final Object[] args = new Object[1 + (otherArgs != null ? otherArgs.length : 0)]; + + args[0] = firstArg; + + if (otherArgs != null) { + System.arraycopy(otherArgs, 0, args, 1, otherArgs.length); + } + + return StringUtils.format(formatString, args); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BroadcastTablesTooLargeFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BroadcastTablesTooLargeFault.java new file mode 100644 index 00000000000..edc33865cf8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BroadcastTablesTooLargeFault.java @@ -0,0 +1,72 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(BroadcastTablesTooLargeFault.CODE) +public class BroadcastTablesTooLargeFault extends BaseMSQFault +{ + static final String CODE = "BroadcastTablesTooLarge"; + + private final long maxBroadcastTablesSize; + + @JsonCreator + public BroadcastTablesTooLargeFault(@JsonProperty("maxBroadcastTablesSize") final long maxBroadcastTablesSize) + { + super(CODE, + "Size of the broadcast tables exceed the memory reserved for them (memory reserved for broadcast tables = %d bytes)", + maxBroadcastTablesSize + ); + this.maxBroadcastTablesSize = maxBroadcastTablesSize; + } + + @JsonProperty + public long getMaxBroadcastTablesSize() + { + return maxBroadcastTablesSize; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + BroadcastTablesTooLargeFault that = (BroadcastTablesTooLargeFault) o; + return maxBroadcastTablesSize == that.maxBroadcastTablesSize; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), maxBroadcastTablesSize); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java new file mode 100644 index 00000000000..4f3508cbde1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java @@ -0,0 +1,41 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; + +@JsonTypeName(CanceledFault.CODE) +public class CanceledFault extends BaseMSQFault +{ + public static final CanceledFault INSTANCE = new CanceledFault(); + static final String CODE = "Canceled"; + + CanceledFault() + { + super(CODE, "Query canceled by user or by task shutdown."); + } + + @JsonCreator + public static CanceledFault instance() + { + return INSTANCE; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CannotParseExternalDataFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CannotParseExternalDataFault.java new file mode 100644 index 00000000000..7668b68ea03 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CannotParseExternalDataFault.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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; + +@JsonTypeName(CannotParseExternalDataFault.CODE) +public class CannotParseExternalDataFault extends BaseMSQFault +{ + public static final String CODE = "CannotParseExternalData"; + + public CannotParseExternalDataFault(@JsonProperty("errorMessage") String message) + { + super(CODE, Preconditions.checkNotNull(message, "errorMessage")); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnNameRestrictedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnNameRestrictedFault.java new file mode 100644 index 00000000000..c2c4617292e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnNameRestrictedFault.java @@ -0,0 +1,75 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.StringUtils; + +import java.util.Objects; + +@JsonTypeName(ColumnNameRestrictedFault.CODE) +public class ColumnNameRestrictedFault extends BaseMSQFault +{ + static final String CODE = "ColumnNameRestricted"; + + private final String columnName; + + @JsonCreator + public ColumnNameRestrictedFault( + @JsonProperty("columnName") final String columnName + ) + { + super(CODE, StringUtils.format( + "[%s] column name is reserved for MSQ's internal purpose. Please retry the query after renaming the column.", + columnName)); + this.columnName = Preconditions.checkNotNull(columnName, "columnName"); + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + ColumnNameRestrictedFault that = (ColumnNameRestrictedFault) o; + return Objects.equals(columnName, that.columnName); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), columnName); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnTypeNotSupportedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnTypeNotSupportedFault.java new file mode 100644 index 00000000000..91764b4b398 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/ColumnTypeNotSupportedFault.java @@ -0,0 +1,89 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.druid.frame.write.UnsupportedColumnTypeException; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; +import java.util.Objects; + +@JsonTypeName(ColumnTypeNotSupportedFault.CODE) +public class ColumnTypeNotSupportedFault extends BaseMSQFault +{ + static final String CODE = "ColumnTypeNotSupported"; + + private final String columnName; + + @Nullable + private final ColumnType columnType; + + @JsonCreator + public ColumnTypeNotSupportedFault( + @JsonProperty("columnName") final String columnName, + @JsonProperty("columnType") @Nullable final ColumnType columnType + ) + { + super(CODE, UnsupportedColumnTypeException.message(columnName, columnType)); + this.columnName = Preconditions.checkNotNull(columnName, "columnName"); + this.columnType = columnType; + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public ColumnType getColumnType() + { + return columnType; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + ColumnTypeNotSupportedFault that = (ColumnTypeNotSupportedFault) o; + return Objects.equals(columnName, that.columnName) && Objects.equals(columnType, that.columnType); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), columnName, columnType); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.java new file mode 100644 index 00000000000..2fe0d6eccc7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.msq.guice.MSQDurableStorageModule; +import org.apache.druid.msq.util.MultiStageQueryContext; + +import java.util.Objects; + +@JsonTypeName(DurableStorageConfigurationFault.CODE) +public class DurableStorageConfigurationFault extends BaseMSQFault +{ + static final String CODE = "DurableStorageConfiguration"; + + private final String errorMessage; + + @JsonCreator + public DurableStorageConfigurationFault(@JsonProperty("message") final String errorMessage) + { + super( + CODE, + "Durable storage mode can only be enabled when %s is set to true and " + + "the connector is configured correctly. " + + "Check the documentation on how to enable durable storage mode. " + + "If you want to still query without durable storage mode, set %s to false in the query context. Got error %s", + MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_ENABLED, + MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, + errorMessage + ); + this.errorMessage = errorMessage; + } + + @JsonProperty + public String getMessage() + { + return errorMessage; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + DurableStorageConfigurationFault that = (DurableStorageConfigurationFault) o; + return Objects.equals(errorMessage, that.errorMessage); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), errorMessage); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/FaultsExceededChecker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/FaultsExceededChecker.java new file mode 100644 index 00000000000..db8f1a2306d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/FaultsExceededChecker.java @@ -0,0 +1,92 @@ +/* + * 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.druid.msq.indexing.error; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterSnapshots; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.counters.WarningCounters; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * Keeps a track of the warnings that have been so far and returns if any type has exceeded their designated limit + * This class is thread safe + */ +public class FaultsExceededChecker +{ + final Map maxFaultsAllowedCount; + + public FaultsExceededChecker(final Map maxFaultsAllowedCount) + { + maxFaultsAllowedCount.forEach( + (warning, count) -> + Preconditions.checkArgument( + count >= 0 || count == -1, + StringUtils.format("Invalid limit of %d supplied for warnings of type %s. " + + "Limit can be greater than or equal to -1.", count, warning) + ) + ); + this.maxFaultsAllowedCount = maxFaultsAllowedCount; + } + + /** + * @param snapshotsTree WorkerCounters have the count of the warnings generated per worker + * + * @return An optional which is empty if the faults count in the present in the task counters don't exceed their + * prescribed limit, else it contains the errorCode and the maximum allowed faults for that errorCode + */ + public Optional> addFaultsAndCheckIfExceeded(CounterSnapshotsTree snapshotsTree) + { + final Map> snapshotsMap = snapshotsTree.copyMap(); + + Map allWarnings = new HashMap<>(); + for (Map.Entry> stageEntry : snapshotsMap.entrySet()) { + for (Map.Entry workerEntry : stageEntry.getValue().entrySet()) { + final WarningCounters.Snapshot warningsSnapshot = + (WarningCounters.Snapshot) workerEntry.getValue().getMap().get(CounterNames.warnings()); + + if (warningsSnapshot != null) { + for (Map.Entry entry : warningsSnapshot.getWarningCountMap().entrySet()) { + allWarnings.compute( + entry.getKey(), + (ignored, value) -> value == null ? entry.getValue() : value + entry.getValue() + ); + } + } + } + } + + for (Map.Entry totalWarningCountEntry : allWarnings.entrySet()) { + long limit = maxFaultsAllowedCount.getOrDefault(totalWarningCountEntry.getKey(), -1L); + boolean passed = limit == -1 || totalWarningCountEntry.getValue() <= limit; + if (!passed) { + return Optional.of(Pair.of(totalWarningCountEntry.getKey(), limit)); + } + } + return Optional.empty(); + } + +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java new file mode 100644 index 00000000000..403af37d9bf --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java @@ -0,0 +1,82 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.joda.time.Interval; + +import java.util.Objects; + +@JsonTypeName(InsertCannotAllocateSegmentFault.CODE) +public class InsertCannotAllocateSegmentFault extends BaseMSQFault +{ + static final String CODE = "InsertCannotAllocateSegment"; + + private final String dataSource; + private final Interval interval; + + @JsonCreator + public InsertCannotAllocateSegmentFault( + @JsonProperty("dataSource") final String dataSource, + @JsonProperty("interval") final Interval interval + ) + { + super(CODE, "Cannot allocate segment for dataSource [%s], interval [%s]", dataSource, interval); + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.interval = Preconditions.checkNotNull(interval, "interval"); + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + InsertCannotAllocateSegmentFault that = (InsertCannotAllocateSegmentFault) o; + return Objects.equals(dataSource, that.dataSource) && Objects.equals(interval, that.interval); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), dataSource, interval); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java new file mode 100644 index 00000000000..4285ace4180 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java @@ -0,0 +1,72 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; + +import java.util.Objects; + +@JsonTypeName(InsertCannotBeEmptyFault.CODE) +public class InsertCannotBeEmptyFault extends BaseMSQFault +{ + static final String CODE = "InsertCannotBeEmpty"; + + private final String dataSource; + + @JsonCreator + public InsertCannotBeEmptyFault( + @JsonProperty("dataSource") final String dataSource + ) + { + super(CODE, "No rows to insert for dataSource [%s]", dataSource); + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + InsertCannotBeEmptyFault that = (InsertCannotBeEmptyFault) o; + return Objects.equals(dataSource, that.dataSource); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), dataSource); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java new file mode 100644 index 00000000000..43b50e87827 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java @@ -0,0 +1,72 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; + +import java.util.Objects; + +@JsonTypeName(InsertCannotOrderByDescendingFault.CODE) +public class InsertCannotOrderByDescendingFault extends BaseMSQFault +{ + static final String CODE = "InsertCannotOrderByDescending"; + + private final String columnName; + + @JsonCreator + public InsertCannotOrderByDescendingFault( + @JsonProperty("columnName") final String columnName + ) + { + super(CODE, "Cannot ingest column [%s] in descending order", columnName); + this.columnName = Preconditions.checkNotNull(columnName, "columnName"); + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + InsertCannotOrderByDescendingFault that = (InsertCannotOrderByDescendingFault) o; + return Objects.equals(columnName, that.columnName); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), columnName); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotReplaceExistingSegmentFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotReplaceExistingSegmentFault.java new file mode 100644 index 00000000000..ed1d14fcc00 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotReplaceExistingSegmentFault.java @@ -0,0 +1,76 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.timeline.SegmentId; + +import java.util.Objects; + +public class InsertCannotReplaceExistingSegmentFault extends BaseMSQFault +{ + static final String CODE = "InsertCannotReplaceExistingSegment"; + + private final String segmentId; + + public InsertCannotReplaceExistingSegmentFault(@JsonProperty("segmentId") String segmentId) + { + super( + CODE, + "Cannot replace existing segment [%s] because it is not within the " + + "bounds specified by replaceExistingTimeChunks", + segmentId + ); + this.segmentId = segmentId; + } + + public InsertCannotReplaceExistingSegmentFault(final SegmentId segmentId) + { + this(segmentId.toString()); + } + + @JsonProperty + public String getSegmentId() + { + return segmentId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + InsertCannotReplaceExistingSegmentFault that = (InsertCannotReplaceExistingSegmentFault) o; + return Objects.equals(segmentId, that.segmentId); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), segmentId); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFault.java new file mode 100644 index 00000000000..83bd9ad8e62 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFault.java @@ -0,0 +1,45 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; + +@JsonTypeName(InsertLockPreemptedFault.CODE) +public class InsertLockPreemptedFault extends BaseMSQFault +{ + public static final InsertLockPreemptedFault INSTANCE = new InsertLockPreemptedFault(); + static final String CODE = "InsertLockPreempted"; + + InsertLockPreemptedFault() + { + super( + CODE, + "Insert lock preempted while trying to ingest the data." + + " This can occur if there are higher priority jobs like real-time ingestion running on same time chunks." + ); + } + + @JsonCreator + public static InsertLockPreemptedFault instance() + { + return INSTANCE; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeNullFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeNullFault.java new file mode 100644 index 00000000000..bcdc592eefc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeNullFault.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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.segment.column.ColumnHolder; + +@JsonTypeName(InsertTimeNullFault.CODE) +public class InsertTimeNullFault extends BaseMSQFault +{ + public static final InsertTimeNullFault INSTANCE = new InsertTimeNullFault(); + static final String CODE = "InsertTimeNull"; + + InsertTimeNullFault() + { + super(CODE, "Null timestamp (%s) encountered during INSERT or REPLACE.", ColumnHolder.TIME_COLUMN_NAME); + } + + @JsonCreator + public static InsertTimeNullFault instance() + { + return INSTANCE; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java new file mode 100644 index 00000000000..c15a3fb6d17 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java @@ -0,0 +1,66 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; + +import java.util.Objects; + +public class InsertTimeOutOfBoundsFault extends BaseMSQFault +{ + static final String CODE = "InsertTimeOutOfBounds"; + + private final Interval interval; + + public InsertTimeOutOfBoundsFault(@JsonProperty("interval") Interval interval) + { + super(CODE, "Query generated time chunk [%s] out of bounds specified by replaceExistingTimeChunks", interval); + this.interval = interval; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + InsertTimeOutOfBoundsFault that = (InsertTimeOutOfBoundsFault) o; + return Objects.equals(interval, that.interval); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), interval); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java new file mode 100644 index 00000000000..894ba5f7e6c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java @@ -0,0 +1,71 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(InvalidNullByteFault.CODE) +public class InvalidNullByteFault extends BaseMSQFault +{ + static final String CODE = "InvalidNullByte"; + + private final String column; + + @JsonCreator + public InvalidNullByteFault( + @JsonProperty("column") final String column + ) + { + super(CODE, "Invalid null byte in string column [%s]", column); + this.column = column; + } + + @JsonProperty + public String getColumn() + { + return column; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + InvalidNullByteFault that = (InvalidNullByteFault) o; + return Objects.equals(column, that.column); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), column); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java new file mode 100644 index 00000000000..31bc2753aa9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java @@ -0,0 +1,200 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.apache.druid.frame.processor.FrameRowTooLargeException; +import org.apache.druid.frame.write.UnsupportedColumnTypeException; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.msq.statistics.TooManyBucketsException; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class MSQErrorReport +{ + private final String taskId; + @Nullable + private final String host; + @Nullable + private final Integer stageNumber; + private final MSQFault error; + @Nullable + private final String exceptionStackTrace; + + @JsonCreator + MSQErrorReport( + @JsonProperty("taskId") final String taskId, + @JsonProperty("host") @Nullable final String host, + @JsonProperty("stageNumber") final Integer stageNumber, + @JsonProperty("error") final MSQFault fault, + @JsonProperty("exceptionStackTrace") @Nullable final String exceptionStackTrace + ) + { + this.taskId = Preconditions.checkNotNull(taskId, "taskId"); + this.host = host; + this.stageNumber = stageNumber; + this.error = Preconditions.checkNotNull(fault, "error"); + this.exceptionStackTrace = exceptionStackTrace; + } + + public static MSQErrorReport fromFault( + final String taskId, + @Nullable final String host, + @Nullable final Integer stageNumber, + final MSQFault fault + ) + { + return new MSQErrorReport(taskId, host, stageNumber, fault, null); + } + + public static MSQErrorReport fromException( + final String taskId, + @Nullable final String host, + @Nullable final Integer stageNumber, + final Throwable e + ) + { + return new MSQErrorReport( + taskId, + host, + stageNumber, + getFaultFromException(e), + Throwables.getStackTraceAsString(e) + ); + } + + @JsonProperty + public String getTaskId() + { + return taskId; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getHost() + { + return host; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Integer getStageNumber() + { + return stageNumber; + } + + @JsonProperty("error") + public MSQFault getFault() + { + return error; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getExceptionStackTrace() + { + return exceptionStackTrace; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MSQErrorReport that = (MSQErrorReport) o; + return Objects.equals(taskId, that.taskId) + && Objects.equals(host, that.host) + && Objects.equals(stageNumber, that.stageNumber) + && Objects.equals(error, that.error) + && Objects.equals(exceptionStackTrace, that.exceptionStackTrace); + } + + @Override + public int hashCode() + { + return Objects.hash(taskId, host, error, exceptionStackTrace); + } + + @Override + public String toString() + { + return "MSQErrorReport{" + + "taskId='" + taskId + '\'' + + ", host='" + host + '\'' + + ", stageNumber=" + stageNumber + + ", error=" + error + + ", exceptionStackTrace='" + exceptionStackTrace + '\'' + + '}'; + } + + /** + * Magical code that extracts a useful fault from an exception, even if that exception is not necessarily a + * {@link MSQException}. This method walks through the causal chain, and also "knows" about various exception + * types thrown by other Druid code. + */ + public static MSQFault getFaultFromException(@Nullable final Throwable e) + { + // Unwrap exception wrappers to find an underlying fault. The assumption here is that the topmost recognizable + // exception should be used to generate the fault code for the entire report. + + Throwable cause = e; + + // This method will grow as we try to add more faults and exceptions + // One way of handling this would be to extend the faults to have a method like + // public MSQFault fromException(@Nullable Throwable e) which returns the specific fault if it can be reconstructed + // from the exception or null. Then instead of having a case per exception, we can have a case per fault, which + // should be cool because there is a 1:1 mapping between faults and exceptions (apart from the more geeneric + // UnknownFaults and MSQExceptions) + while (cause != null) { + + if (cause instanceof MSQException) { + return ((MSQException) cause).getFault(); + } else if (cause instanceof ParseException) { + return new CannotParseExternalDataFault(cause.getMessage()); + } else if (cause instanceof UnsupportedColumnTypeException) { + final UnsupportedColumnTypeException unsupportedColumnTypeException = (UnsupportedColumnTypeException) cause; + return new ColumnTypeNotSupportedFault( + unsupportedColumnTypeException.getColumnName(), + unsupportedColumnTypeException.getColumnType() + ); + } else if (cause instanceof TooManyBucketsException) { + return new TooManyBucketsFault(((TooManyBucketsException) cause).getMaxBuckets()); + } else if (cause instanceof FrameRowTooLargeException) { + return new RowTooLargeFault(((FrameRowTooLargeException) cause).getMaxFrameSize()); + } else { + cause = cause.getCause(); + } + } + + return UnknownFault.forException(e); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.java new file mode 100644 index 00000000000..b50e430f18c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.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.druid.msq.indexing.error; + +import com.google.common.base.Preconditions; + +import javax.annotation.Nullable; + +/** + * An unchecked exception that holds a {@link MSQFault}. + */ +public class MSQException extends RuntimeException +{ + private final MSQFault fault; + + public MSQException( + @Nullable final Throwable cause, + final MSQFault fault + ) + { + super(fault.getCodeWithMessage(), cause); + this.fault = Preconditions.checkNotNull(fault, "fault"); + } + + public MSQException(final MSQFault fault) + { + this(null, fault); + } + + public MSQFault getFault() + { + return fault; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java new file mode 100644 index 00000000000..36f5910d3ee --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java @@ -0,0 +1,49 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +import javax.annotation.Nullable; + +/** + * Error code for multi-stage queries. + * + * See {@link MSQErrorReport#getFaultFromException} for a mapping of exception type to error code. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "errorCode") +public interface MSQFault +{ + String getErrorCode(); + + @Nullable + String getErrorMessage(); + + default String getCodeWithMessage() + { + final String message = getErrorMessage(); + + if (message != null && !message.isEmpty()) { + return getErrorCode() + ": " + message; + } else { + return getErrorCode(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java new file mode 100644 index 00000000000..c72d70206ab --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java @@ -0,0 +1,96 @@ +/* + * 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.druid.msq.indexing.error; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.msq.exec.Limits; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Limits the number of exceptions that get published to the underlying delegate publisher. This helps + * in preventing the spam of exceptions from the worker task to the published source. As such, any implementation + * of {@link MSQWarningReportPublisher} that is wrapped in this class cannot be sure that the warning handed off + * is trully published + */ +public class MSQWarningReportLimiterPublisher implements MSQWarningReportPublisher +{ + + final MSQWarningReportPublisher delegate; + final long totalLimit; + final Map errorCodeToLimit; + final ConcurrentHashMap errorCodeToCurrentCount = new ConcurrentHashMap<>(); + + long totalCount = 0L; + + final Object lock = new Object(); + + public MSQWarningReportLimiterPublisher(MSQWarningReportPublisher delegate) + { + this( + delegate, + Limits.MAX_VERBOSE_WARNINGS, + ImmutableMap.of( + CannotParseExternalDataFault.CODE, Limits.MAX_VERBOSE_PARSE_EXCEPTIONS + ) + ); + } + + public MSQWarningReportLimiterPublisher( + MSQWarningReportPublisher delegate, + long totalLimit, + Map errorCodeToLimit + ) + { + this.delegate = delegate; + this.errorCodeToLimit = errorCodeToLimit; + this.totalLimit = totalLimit; + } + + @Override + public void publishException(int stageNumber, Throwable e) + { + String errorCode = MSQErrorReport.getFaultFromException(e).getErrorCode(); + synchronized (lock) { + totalCount = totalCount + 1; + errorCodeToCurrentCount.compute(errorCode, (ignored, count) -> count == null ? 1L : count + 1); + + if (totalLimit != -1 && totalCount > totalLimit) { + return; + } + } + + long limitForFault = errorCodeToLimit.getOrDefault(errorCode, -1L); + synchronized (lock) { + if (limitForFault != -1 && errorCodeToCurrentCount.getOrDefault(errorCode, 0L) > limitForFault) { + return; + } + } + delegate.publishException(stageNumber, e); + } + + @Override + public void close() throws IOException + { + delegate.close(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.java new file mode 100644 index 00000000000..9dbb51b6569 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.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.druid.msq.indexing.error; + +import java.io.Closeable; +import java.io.IOException; + +/** + * Provides an interface for a worker to publish warnings to an external source. + * For example, the worker uses this interface to send warnings to the controller. + */ +public interface MSQWarningReportPublisher extends Closeable +{ + void publishException(int stageNumber, Throwable e); + + @Override + void close() throws IOException; +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java new file mode 100644 index 00000000000..3c07a163aa3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java @@ -0,0 +1,72 @@ +/* + * 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.druid.msq.indexing.error; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.msq.exec.ControllerClient; + +import javax.annotation.Nullable; +import java.io.IOException; + +/** + * Publishes the warning report to the {@link ControllerClient#postWorkerWarning} as is without any buffering/batching. + */ +public class MSQWarningReportSimplePublisher implements MSQWarningReportPublisher +{ + + final String workerId; + final ControllerClient controllerClient; + final String taskId; + @Nullable + final String host; + + public MSQWarningReportSimplePublisher( + final String workerId, + final ControllerClient controllerClient, + final String taskId, + @Nullable final String host + ) + { + this.workerId = workerId; + this.controllerClient = controllerClient; + this.taskId = taskId; + this.host = host; + } + + + @Override + public void publishException(int stageNumber, Throwable e) + { + final MSQErrorReport warningReport = MSQErrorReport.fromException(taskId, host, stageNumber, e); + + try { + controllerClient.postWorkerWarning(workerId, ImmutableList.of(warningReport)); + } + catch (IOException e2) { + throw new RuntimeException(e2); + } + } + + @Override + public void close() + { + + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarnings.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarnings.java new file mode 100644 index 00000000000..7b75940df34 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarnings.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing.error; + +/** + * Helper class for defining parameters used by the multi-stage query engine's "warning framework" + */ +public class MSQWarnings +{ + /** + * Query context key for limiting the maximum number of parse exceptions that a multi-stage query can generate + */ + public static final String CTX_MAX_PARSE_EXCEPTIONS_ALLOWED = "maxParseExceptions"; + + /** + * Default number of parse exceptions permissible for a multi-stage query + */ + public static final Long DEFAULT_MAX_PARSE_EXCEPTIONS_ALLOWED = -1L; +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java new file mode 100644 index 00000000000..b353c03df10 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java @@ -0,0 +1,108 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(NotEnoughMemoryFault.CODE) +public class NotEnoughMemoryFault extends BaseMSQFault +{ + static final String CODE = "NotEnoughMemory"; + + private final long serverMemory; + private final int serverWorkers; + private final int serverThreads; + + @JsonCreator + public NotEnoughMemoryFault( + @JsonProperty("serverMemory") final long serverMemory, + @JsonProperty("serverWorkers") final int serverWorkers, + @JsonProperty("serverThreads") final int serverThreads + ) + { + super( + CODE, + "Not enough memory (available = %,d; server workers = %,d; server threads = %,d)", + serverMemory, + serverWorkers, + serverThreads + ); + + this.serverMemory = serverMemory; + this.serverWorkers = serverWorkers; + this.serverThreads = serverThreads; + } + + @JsonProperty + public long getServerMemory() + { + return serverMemory; + } + + @JsonProperty + public int getServerWorkers() + { + return serverWorkers; + } + + @JsonProperty + public int getServerThreads() + { + return serverThreads; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + NotEnoughMemoryFault that = (NotEnoughMemoryFault) o; + return serverMemory == that.serverMemory + && serverWorkers == that.serverWorkers + && serverThreads == that.serverThreads; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), serverMemory, serverWorkers, serverThreads); + } + + @Override + public String toString() + { + return "NotEnoughMemoryFault{" + + "serverMemory=" + serverMemory + + ", serverWorkers=" + serverWorkers + + ", serverThreads=" + serverThreads + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/QueryNotSupportedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/QueryNotSupportedFault.java new file mode 100644 index 00000000000..bba058cd588 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/QueryNotSupportedFault.java @@ -0,0 +1,41 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; + +@JsonTypeName(QueryNotSupportedFault.CODE) +public class QueryNotSupportedFault extends BaseMSQFault +{ + public static final QueryNotSupportedFault INSTANCE = new QueryNotSupportedFault(); + static final String CODE = "QueryNotSupported"; + + QueryNotSupportedFault() + { + super(CODE); + } + + @JsonCreator + public static QueryNotSupportedFault instance() + { + return INSTANCE; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/RowTooLargeFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/RowTooLargeFault.java new file mode 100644 index 00000000000..865cb6b0c1d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/RowTooLargeFault.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(RowTooLargeFault.CODE) +public class RowTooLargeFault extends BaseMSQFault +{ + static final String CODE = "RowTooLarge"; + + private final long maxFrameSize; + + @JsonCreator + public RowTooLargeFault(@JsonProperty("maxFrameSize") final long maxFrameSize) + { + super(CODE, "Encountered row that cannot fit in a single frame (max frame size = %,d)", maxFrameSize); + this.maxFrameSize = maxFrameSize; + } + + @JsonProperty + public long getMaxFrameSize() + { + return maxFrameSize; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + RowTooLargeFault that = (RowTooLargeFault) o; + return maxFrameSize == that.maxFrameSize; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), maxFrameSize); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TaskStartTimeoutFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TaskStartTimeoutFault.java new file mode 100644 index 00000000000..dceb730393d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TaskStartTimeoutFault.java @@ -0,0 +1,76 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.msq.util.MultiStageQueryContext; + +import java.util.Objects; + +@JsonTypeName(TaskStartTimeoutFault.CODE) +public class TaskStartTimeoutFault extends BaseMSQFault +{ + static final String CODE = "TaskStartTimeout"; + + private final int numTasks; + + @JsonCreator + public TaskStartTimeoutFault(@JsonProperty("numTasks") int numTasks) + { + super( + CODE, + "Unable to launch all the worker tasks in time. There might be insufficient available slots to start all the worker tasks simultaneously." + + " Try lowering '%s' in your query context to lower than [%d] tasks, or increasing capacity.", + MultiStageQueryContext.CTX_MAX_NUM_TASKS, + numTasks + ); + this.numTasks = numTasks; + } + + @JsonProperty + public int getNumTasks() + { + return numTasks; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TaskStartTimeoutFault that = (TaskStartTimeoutFault) o; + return numTasks == that.numTasks; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), numTasks); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java new file mode 100644 index 00000000000..8af20d09191 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java @@ -0,0 +1,77 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(TooManyBucketsFault.CODE) +public class TooManyBucketsFault extends BaseMSQFault +{ + static final String CODE = "TooManyBuckets"; + + private final int maxBuckets; + + @JsonCreator + public TooManyBucketsFault(@JsonProperty("maxBuckets") final int maxBuckets) + { + // Currently, partition buckets are only used for segmentGranularity during ingestion queries. So it's fair + // to assume that a TooManyBuckets error happened due to a too-fine segmentGranularity, even though we don't + // technically have proof of that. + super( + CODE, + "Too many partition buckets (max = %,d); try breaking your query up into smaller queries or " + + "using a wider segmentGranularity", + maxBuckets + ); + this.maxBuckets = maxBuckets; + } + + @JsonProperty + public int getMaxBuckets() + { + return maxBuckets; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TooManyBucketsFault that = (TooManyBucketsFault) o; + return maxBuckets == that.maxBuckets; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), maxBuckets); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyColumnsFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyColumnsFault.java new file mode 100644 index 00000000000..05b9d0d197a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyColumnsFault.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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(TooManyColumnsFault.CODE) +public class TooManyColumnsFault extends BaseMSQFault +{ + static final String CODE = "TooManyColumns"; + + private final int numColumns; + private final int maxColumns; + + @JsonCreator + public TooManyColumnsFault( + @JsonProperty("numColumns") final int numColumns, + @JsonProperty("maxColumns") final int maxColumns + ) + { + super(CODE, "Too many output columns (requested = %d, max = %d)", numColumns, maxColumns); + this.numColumns = numColumns; + this.maxColumns = maxColumns; + } + + @JsonProperty + public int getNumColumns() + { + return numColumns; + } + + @JsonProperty + public int getMaxColumns() + { + return maxColumns; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TooManyColumnsFault that = (TooManyColumnsFault) o; + return numColumns == that.numColumns && maxColumns == that.maxColumns; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), numColumns, maxColumns); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyInputFilesFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyInputFilesFault.java new file mode 100644 index 00000000000..ceb1367ffd8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyInputFilesFault.java @@ -0,0 +1,102 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.msq.util.MultiStageQueryContext; + +import java.util.Objects; + +@JsonTypeName(TooManyInputFilesFault.CODE) +public class TooManyInputFilesFault extends BaseMSQFault +{ + static final String CODE = "TooManyInputFiles"; + + private final int numInputFiles; + private final int maxInputFiles; + private final int minNumWorkers; + + + @JsonCreator + public TooManyInputFilesFault( + @JsonProperty("numInputFiles") final int numInputFiles, + @JsonProperty("maxInputFiles") final int maxInputFiles, + @JsonProperty("minNumWorkers") final int minNumWorkers + ) + { + super( + CODE, + "Too many input files/segments [%d] encountered. Maximum input files/segments per worker is set to [%d]. Try" + + " breaking your query up into smaller queries, or increasing the number of workers to at least [%d] by" + + " setting %s in your query context", + numInputFiles, + maxInputFiles, + minNumWorkers, + MultiStageQueryContext.CTX_MAX_NUM_TASKS + ); + this.numInputFiles = numInputFiles; + this.maxInputFiles = maxInputFiles; + this.minNumWorkers = minNumWorkers; + } + + @JsonProperty + public int getNumInputFiles() + { + return numInputFiles; + } + + @JsonProperty + public int getMaxInputFiles() + { + return maxInputFiles; + } + + @JsonProperty + public int getMinNumWorkers() + { + return minNumWorkers; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TooManyInputFilesFault that = (TooManyInputFilesFault) o; + return numInputFiles == that.numInputFiles + && maxInputFiles == that.maxInputFiles + && minNumWorkers == that.minNumWorkers; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), numInputFiles, maxInputFiles, minNumWorkers); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyPartitionsFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyPartitionsFault.java new file mode 100644 index 00000000000..e9d91a42d4e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyPartitionsFault.java @@ -0,0 +1,74 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(TooManyPartitionsFault.CODE) +public class TooManyPartitionsFault extends BaseMSQFault +{ + static final String CODE = "TooManyPartitions"; + + private final int maxPartitions; + + @JsonCreator + public TooManyPartitionsFault(@JsonProperty("maxPartitions") final int maxPartitions) + { + super( + CODE, + "Too many partitions (max = %d); try breaking your query up into smaller queries or " + + "using a larger target size", + maxPartitions + ); + this.maxPartitions = maxPartitions; + } + + @JsonProperty + public int getMaxPartitions() + { + return maxPartitions; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TooManyPartitionsFault that = (TooManyPartitionsFault) o; + return maxPartitions == that.maxPartitions; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), maxPartitions); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWarningsFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWarningsFault.java new file mode 100644 index 00000000000..57663cb0ddb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWarningsFault.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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(TooManyWarningsFault.CODE) +public class TooManyWarningsFault extends BaseMSQFault +{ + static final String CODE = "TooManyWarnings"; + + private final int maxWarnings; + private final String rootErrorCode; + + @JsonCreator + public TooManyWarningsFault( + @JsonProperty("maxWarnings") final int maxWarnings, + @JsonProperty("rootErrorCode") final String rootErrorCode + ) + { + super(CODE, "Too many warnings of type %s generated (max = %d)", rootErrorCode, maxWarnings); + this.maxWarnings = maxWarnings; + this.rootErrorCode = rootErrorCode; + } + + @JsonProperty + public int getMaxWarnings() + { + return maxWarnings; + } + + @JsonProperty + public String getRootErrorCode() + { + return rootErrorCode; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TooManyWarningsFault that = (TooManyWarningsFault) o; + return maxWarnings == that.maxWarnings && Objects.equals(rootErrorCode, that.rootErrorCode); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), maxWarnings, rootErrorCode); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkersFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkersFault.java new file mode 100644 index 00000000000..42376ca8cb8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkersFault.java @@ -0,0 +1,89 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(TooManyWorkersFault.CODE) +public class TooManyWorkersFault extends BaseMSQFault +{ + static final String CODE = "TooManyWorkers"; + + private final int workers; + private final int maxWorkers; + + @JsonCreator + public TooManyWorkersFault( + @JsonProperty("workers") final int workers, + @JsonProperty("maxWorkers") final int maxWorkers + ) + { + super(CODE, "Too many workers (current = %d; max = %d)", workers, maxWorkers); + this.workers = workers; + this.maxWorkers = maxWorkers; + } + + @JsonProperty + public int getWorkers() + { + return workers; + } + + @JsonProperty + public int getMaxWorkers() + { + return maxWorkers; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TooManyWorkersFault that = (TooManyWorkersFault) o; + return workers == that.workers && maxWorkers == that.maxWorkers; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), workers, maxWorkers); + } + + @Override + public String toString() + { + return "TooManyWorkersFault{" + + "workers=" + workers + + ", maxWorkers=" + maxWorkers + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/UnknownFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/UnknownFault.java new file mode 100644 index 00000000000..3a14683613d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/UnknownFault.java @@ -0,0 +1,84 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import javax.annotation.Nullable; +import java.util.Objects; + +@JsonTypeName(UnknownFault.CODE) +public class UnknownFault extends BaseMSQFault +{ + public static final String CODE = "UnknownError"; + + @Nullable + private final String message; + + private UnknownFault(@Nullable final String message) + { + super(CODE, message); + this.message = message; + } + + @JsonCreator + public static UnknownFault forMessage(@JsonProperty("message") @Nullable final String message) + { + return new UnknownFault(message); + } + + public static UnknownFault forException(@Nullable final Throwable t) + { + return new UnknownFault(t == null ? null : t.toString()); + } + + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + @JsonProperty + public String getMessage() + { + return message; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + UnknownFault that = (UnknownFault) o; + return Objects.equals(message, that.message); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), message); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerFailedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerFailedFault.java new file mode 100644 index 00000000000..b16260a601a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerFailedFault.java @@ -0,0 +1,86 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import javax.annotation.Nullable; +import java.util.Objects; + +@JsonTypeName(WorkerFailedFault.CODE) +public class WorkerFailedFault extends BaseMSQFault +{ + public static final String CODE = "WorkerFailed"; + + private final String workerTaskId; + + @Nullable + private final String errorMsg; + + @JsonCreator + public WorkerFailedFault( + @JsonProperty("workerTaskId") final String workerTaskId, + @JsonProperty("errorMsg") @Nullable final String errorMsg + ) + { + super(CODE, "Worker task failed: [%s]%s", workerTaskId, errorMsg != null ? " (" + errorMsg + ")" : ""); + this.workerTaskId = workerTaskId; + this.errorMsg = errorMsg; + } + + @JsonProperty + public String getWorkerTaskId() + { + return workerTaskId; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getErrorMsg() + { + return errorMsg; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + WorkerFailedFault that = (WorkerFailedFault) o; + return Objects.equals(workerTaskId, that.workerTaskId) && Objects.equals(errorMsg, that.errorMsg); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), workerTaskId, errorMsg); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java new file mode 100644 index 00000000000..0fb888d8fc0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(WorkerRpcFailedFault.CODE) +public class WorkerRpcFailedFault extends BaseMSQFault +{ + static final String CODE = "WorkerRpcFailed"; + + private final String workerTaskId; + + @JsonCreator + public WorkerRpcFailedFault(@JsonProperty("workerTaskId") final String workerTaskId) + { + super(CODE, "RPC call to task failed unrecoverably: [%s]", workerTaskId); + this.workerTaskId = workerTaskId; + } + + @JsonProperty + public String getWorkerTaskId() + { + return workerTaskId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + WorkerRpcFailedFault that = (WorkerRpcFailedFault) o; + return Objects.equals(workerTaskId, that.workerTaskId); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), workerTaskId); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java new file mode 100644 index 00000000000..3d75e3986ca --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java @@ -0,0 +1,84 @@ +/* + * 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.druid.msq.indexing.report; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.util.List; + +public class MSQResultsReport +{ + private final RowSignature signature; + @Nullable + private final List sqlTypeNames; + private final Yielder resultYielder; + + public MSQResultsReport( + final RowSignature signature, + @Nullable final List sqlTypeNames, + final Yielder resultYielder + ) + { + this.signature = Preconditions.checkNotNull(signature, "signature"); + this.sqlTypeNames = sqlTypeNames; + this.resultYielder = Preconditions.checkNotNull(resultYielder, "resultYielder"); + } + + /** + * Method that enables Jackson deserialization. + */ + @JsonCreator + static MSQResultsReport fromJson( + @JsonProperty("signature") final RowSignature signature, + @JsonProperty("sqlTypeNames") @Nullable final List sqlTypeNames, + @JsonProperty("results") final List results + ) + { + return new MSQResultsReport(signature, sqlTypeNames, Yielders.each(Sequences.simple(results))); + } + + @JsonProperty("signature") + public RowSignature getSignature() + { + return signature; + } + + @Nullable + @JsonProperty("sqlTypeNames") + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getSqlTypeNames() + { + return sqlTypeNames; + } + + @JsonProperty("results") + public Yielder getResultYielder() + { + return resultYielder; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStagesReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStagesReport.java new file mode 100644 index 00000000000..422d8235fe2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStagesReport.java @@ -0,0 +1,248 @@ +/* + * 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.druid.msq.indexing.report; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.controller.ControllerStagePhase; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class MSQStagesReport +{ + private final List stages; + + @JsonCreator + public MSQStagesReport(final List stages) + { + this.stages = Preconditions.checkNotNull(stages, "stages"); + } + + public static MSQStagesReport create( + final QueryDefinition queryDef, + final Map stagePhaseMap, + final Map stageRuntimeMap, + final Map stageWorkerCountMap, + final Map stagePartitionCountMap + ) + { + final List stages = new ArrayList<>(); + + final int[] stageNumbers = + queryDef.getStageDefinitions().stream().mapToInt(StageDefinition::getStageNumber).toArray(); + Arrays.sort(stageNumbers); + + for (final int stageNumber : stageNumbers) { + final StageDefinition stageDef = queryDef.getStageDefinition(stageNumber); + + final int workerCount = stageWorkerCountMap.getOrDefault(stageNumber, 0); + final int partitionCount = stagePartitionCountMap.getOrDefault(stageNumber, 0); + final Interval stageRuntimeInterval = stageRuntimeMap.get(stageNumber); + final DateTime stageStartTime = stageRuntimeInterval == null ? null : stageRuntimeInterval.getStart(); + final long stageDuration = stageRuntimeInterval == null ? 0 : stageRuntimeInterval.toDurationMillis(); + + final Stage stage = new Stage( + stageNumber, + stageDef, + stagePhaseMap.get(stageNumber), + workerCount, + partitionCount, + stageStartTime, + stageDuration + ); + stages.add(stage); + } + + return new MSQStagesReport(stages); + } + + @JsonValue + public List getStages() + { + return stages; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MSQStagesReport that = (MSQStagesReport) o; + return Objects.equals(stages, that.stages); + } + + @Override + public int hashCode() + { + return Objects.hash(stages); + } + + @Override + public String toString() + { + return "MSQStagesReport{" + + "stages=" + stages + + '}'; + } + + public static class Stage + { + private final int stageNumber; + private final StageDefinition stageDef; + @Nullable + private final ControllerStagePhase phase; + private final int workerCount; + private final int partitionCount; + private final DateTime startTime; + private final long duration; + + @JsonCreator + private Stage( + @JsonProperty("stageNumber") final int stageNumber, + @JsonProperty("definition") final StageDefinition stageDef, + @JsonProperty("phase") @Nullable final ControllerStagePhase phase, + @JsonProperty("workerCount") final int workerCount, + @JsonProperty("partitionCount") final int partitionCount, + @JsonProperty("startTime") @Nullable final DateTime startTime, + @JsonProperty("duration") final long duration + ) + { + this.stageNumber = stageNumber; + this.stageDef = stageDef; + this.phase = phase; + this.workerCount = workerCount; + this.partitionCount = partitionCount; + this.startTime = startTime; + this.duration = duration; + } + + @JsonProperty + public int getStageNumber() + { + return stageNumber; + } + + @JsonProperty("definition") + public StageDefinition getStageDefinition() + { + return stageDef; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public ControllerStagePhase getPhase() + { + // Null if the stage has not yet been started. + return phase; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public int getWorkerCount() + { + return workerCount; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public int getPartitionCount() + { + return partitionCount; + } + + @JsonProperty("sort") + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean isSorting() + { + // Field written out, but not read, because it is derived from "definition". + return stageDef.doesSortDuringShuffle(); + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public DateTime getStartTime() + { + return startTime; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public long getDuration() + { + return duration; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Stage stage = (Stage) o; + return stageNumber == stage.stageNumber + && workerCount == stage.workerCount + && partitionCount == stage.partitionCount + && duration == stage.duration + && Objects.equals(stageDef, stage.stageDef) + && phase == stage.phase + && Objects.equals(startTime, stage.startTime); + } + + @Override + public int hashCode() + { + return Objects.hash(stageNumber, stageDef, phase, workerCount, partitionCount, startTime, duration); + } + + @Override + public String toString() + { + return "Stage{" + + "stageNumber=" + stageNumber + + ", stageDef=" + stageDef + + ", phase=" + phase + + ", workerCount=" + workerCount + + ", partitionCount=" + partitionCount + + ", startTime=" + startTime + + ", duration=" + duration + + '}'; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java new file mode 100644 index 00000000000..d2849d8b2ca --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java @@ -0,0 +1,135 @@ +/* + * 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.druid.msq.indexing.report; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.Collections; +import java.util.Objects; + +public class MSQStatusReport +{ + private final TaskState status; + + @Nullable + private final MSQErrorReport errorReport; + + private final Collection warningReports; + + @Nullable + private final DateTime startTime; + + private final long durationMs; + + + @JsonCreator + public MSQStatusReport( + @JsonProperty("status") TaskState status, + @JsonProperty("errorReport") @Nullable MSQErrorReport errorReport, + @JsonProperty("warnings") Collection warningReports, + @JsonProperty("startTime") @Nullable DateTime startTime, + @JsonProperty("durationMs") long durationMs + ) + { + this.status = Preconditions.checkNotNull(status, "status"); + this.errorReport = errorReport; + this.warningReports = warningReports != null ? warningReports : Collections.emptyList(); + this.startTime = startTime; + this.durationMs = durationMs; + } + + @JsonProperty + public TaskState getStatus() + { + return status; + } + + @Nullable + @JsonProperty("errorReport") + @JsonInclude(JsonInclude.Include.NON_NULL) + public MSQErrorReport getErrorReport() + { + return errorReport; + } + + @JsonProperty("warnings") + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public Collection getWarningReports() + { + return warningReports; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public DateTime getStartTime() + { + return startTime; + } + + @JsonProperty + public long getDurationMs() + { + return durationMs; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MSQStatusReport that = (MSQStatusReport) o; + return durationMs == that.durationMs + && status == that.status + && Objects.equals(errorReport, that.errorReport) + && Objects.equals(warningReports, that.warningReports) + && Objects.equals(startTime, that.startTime); + } + + @Override + public int hashCode() + { + return Objects.hash(status, errorReport, warningReports, startTime, durationMs); + } + + @Override + public String toString() + { + return "MSQStatusReport{" + + "status=" + status + + ", errorReport=" + errorReport + + ", warningReports=" + warningReports + + ", startTime=" + startTime + + ", durationMs=" + durationMs + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReport.java new file mode 100644 index 00000000000..0fbe6fec587 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReport.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.druid.msq.indexing.report; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.indexing.common.TaskReport; + +@JsonTypeName(MSQTaskReport.REPORT_KEY) +public class MSQTaskReport implements TaskReport +{ + public static final String REPORT_KEY = "multiStageQuery"; + + private final String taskId; + private final MSQTaskReportPayload payload; + + @JsonCreator + public MSQTaskReport( + @JsonProperty("taskId") final String taskId, + @JsonProperty("payload") final MSQTaskReportPayload payload + ) + { + this.taskId = taskId; + this.payload = payload; + } + + @Override + @JsonProperty + public String getTaskId() + { + return taskId; + } + + @Override + public String getReportKey() + { + return REPORT_KEY; + } + + @Override + @JsonProperty + public MSQTaskReportPayload getPayload() + { + return payload; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReportPayload.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReportPayload.java new file mode 100644 index 00000000000..111cb5aa83a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQTaskReportPayload.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.druid.msq.indexing.report; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.msq.counters.CounterSnapshotsTree; + +import javax.annotation.Nullable; + +public class MSQTaskReportPayload +{ + private final MSQStatusReport status; + + @Nullable + private final MSQStagesReport stages; + + @Nullable + private final CounterSnapshotsTree counters; + + @Nullable + private final MSQResultsReport results; + + @JsonCreator + public MSQTaskReportPayload( + @JsonProperty("status") MSQStatusReport status, + @JsonProperty("stages") @Nullable MSQStagesReport stages, + @JsonProperty("counters") @Nullable CounterSnapshotsTree counters, + @JsonProperty("results") @Nullable MSQResultsReport results + ) + { + this.status = status; + this.stages = stages; + this.counters = counters; + this.results = results; + } + + @JsonProperty + public MSQStatusReport getStatus() + { + return status; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public MSQStagesReport getStages() + { + return stages; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public CounterSnapshotsTree getCounters() + { + return counters; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public MSQResultsReport getResults() + { + return results; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.java new file mode 100644 index 00000000000..bbed49a5b35 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.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.druid.msq.input; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * Slice of an {@link InputSpec} assigned to a particular worker. + * + * On the controller, these are produced using {@link InputSpecSlicer}. On workers, these are read + * using {@link InputSliceReader}. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface InputSlice +{ + /** + * Returns the number of files contained within this split. This is the same number that would be added to + * {@link org.apache.druid.msq.counters.CounterTracker} on full iteration through {@link InputSliceReader#attach}. + * + * May be zero for some kinds of slices, even if they contain data, if the input is not file-based. + */ + int fileCount(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java new file mode 100644 index 00000000000..69b48d58554 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input; + +import org.apache.druid.msq.counters.CounterTracker; + +import java.util.function.Consumer; + +/** + * Reads {@link InputSlice} on workers. + */ +public interface InputSliceReader +{ + /** + * Returns the number of {@link ReadableInput} that would result from a call to {@link #attach}. + * + * @throws UnsupportedOperationException if this reader does not support this spec + */ + int numReadableInputs(InputSlice slice); + + /** + * Returns an iterable sequence of {@link ReadableInput} for an {@link InputSpec}, bundled with a + * {@link org.apache.druid.frame.read.FrameReader} if appropriate. + * + * @throws UnsupportedOperationException if this reader does not support this spec + */ + ReadableInputs attach( + int inputNumber, + InputSlice slice, + CounterTracker counters, + Consumer warningPublisher + ); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java new file mode 100644 index 00000000000..98e28b07542 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java @@ -0,0 +1,65 @@ +/* + * 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.druid.msq.input; + +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.input.stage.StageInputSlice; + +import java.util.ArrayList; +import java.util.List; + +public class InputSlices +{ + private InputSlices() + { + // No instantiation. + } + + public static ReadablePartitions allReadablePartitions(final List slices) + { + final List partitionsList = new ArrayList<>(); + + for (final InputSlice slice : slices) { + if (slice instanceof StageInputSlice) { + partitionsList.add(((StageInputSlice) slice).getPartitions()); + } + } + + return ReadablePartitions.combine(partitionsList); + } + + public static int getNumNonBroadcastReadableInputs( + final List slices, + final InputSliceReader reader, + final IntSet broadcastInputs + ) + { + int numInputs = 0; + + for (int i = 0; i < slices.size(); i++) { + if (!broadcastInputs.contains(i)) { + numInputs += reader.numReadableInputs(slices.get(i)); + } + } + + return numInputs; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpec.java new file mode 100644 index 00000000000..1e07415c147 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpec.java @@ -0,0 +1,30 @@ +/* + * 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.druid.msq.input; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * Description of an input. Sliced into {@link InputSlice} by {@link InputSpecSlicer} and then assigned to workers. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface InputSpec +{ +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicer.java new file mode 100644 index 00000000000..de40d70343c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicer.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input; + +import java.util.List; + +/** + * Slices {@link InputSpec} into {@link InputSlice} on the controller. + */ +public interface InputSpecSlicer +{ + boolean canSliceDynamic(InputSpec inputSpec); + + /** + * Slice a spec into a given maximum number of slices. The returned list may contain fewer slices, but cannot + * contain more. + * + * This method creates as many slices as possible while staying at or under maxNumSlices. For example, if a spec + * contains 8 files, and maxNumSlices is 10, then 8 slices will be created. + */ + List sliceStatic(InputSpec inputSpec, int maxNumSlices); + + /** + * Slice a spec based on a particular maximum number of files and bytes per slice. + * + * This method creates as few slices as possible, while keeping each slice under the provided limits. + * + * If there is a conflict between maxNumSlices and maxFilesPerSlice or maxBytesPerSlice, then maxNumSlices wins. + * This means that for small values of maxNumSlices, slices may have more than maxFilesPerSlice files, or more + * than maxBytesPerSlice bytes. + */ + List sliceDynamic(InputSpec inputSpec, int maxNumSlices, int maxFilesPerSlice, long maxBytesPerSlice); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicerFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicerFactory.java new file mode 100644 index 00000000000..24b5cc1c525 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecSlicerFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.input.stage.StageInputSpecSlicer; + +/** + * Creates an {@link InputSpecSlicer} given a map of stage numbers to output partitions of that stage. + * + * In production, this is typically used to create a {@link MapInputSpecSlicer} that contains a + * {@link StageInputSpecSlicer}. The stage slicer needs the output partitions map in order to do its job, and + * these aren't always known until the stage has started. + */ +public interface InputSpecSlicerFactory +{ + InputSpecSlicer makeSlicer(Int2ObjectMap stagePartitionsMap); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java new file mode 100644 index 00000000000..78241257710 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input; + +import it.unimi.dsi.fastutil.ints.IntRBTreeSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.msq.input.stage.StageInputSpec; + +import java.util.List; + +/** + * Utility functions for working with {@link InputSpec}. + */ +public class InputSpecs +{ + private InputSpecs() + { + // No instantiation. + } + + public static IntSet getStageNumbers(final List specs) + { + final IntSet retVal = new IntRBTreeSet(); + + for (final InputSpec spec : specs) { + if (spec instanceof StageInputSpec) { + retVal.add(((StageInputSpec) spec).getStageNumber()); + } + } + + return retVal; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java new file mode 100644 index 00000000000..aa9abe75cd1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java @@ -0,0 +1,70 @@ +/* + * 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.druid.msq.input; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.counters.CounterTracker; + +import java.util.Map; +import java.util.function.Consumer; + +/** + * Reader that handles multiple types of slices. + */ +public class MapInputSliceReader implements InputSliceReader +{ + private final Map, InputSliceReader> readerMap; + + @Inject + public MapInputSliceReader(final Map, InputSliceReader> readerMap) + { + this.readerMap = ImmutableMap.copyOf(readerMap); + } + + @Override + public int numReadableInputs(InputSlice slice) + { + return getReader(slice.getClass()).numReadableInputs(slice); + } + + @Override + public ReadableInputs attach( + int inputNumber, + InputSlice slice, + CounterTracker counters, + Consumer warningPublisher + ) + { + return getReader(slice.getClass()).attach(inputNumber, slice, counters, warningPublisher); + } + + private InputSliceReader getReader(final Class clazz) + { + final InputSliceReader reader = readerMap.get(clazz); + + if (reader == null) { + throw new ISE("Cannot handle inputSpec of class [%s]", clazz.getName()); + } + + return reader; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSpecSlicer.java new file mode 100644 index 00000000000..f26a0459c89 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSpecSlicer.java @@ -0,0 +1,73 @@ +/* + * 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.druid.msq.input; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.ISE; + +import java.util.List; +import java.util.Map; + +/** + * Slicer that handles multiple types of specs. + */ +public class MapInputSpecSlicer implements InputSpecSlicer +{ + private final Map, InputSpecSlicer> splitterMap; + + public MapInputSpecSlicer(final Map, InputSpecSlicer> splitterMap) + { + this.splitterMap = ImmutableMap.copyOf(splitterMap); + } + + @Override + public boolean canSliceDynamic(InputSpec inputSpec) + { + return getSlicer(inputSpec.getClass()).canSliceDynamic(inputSpec); + } + + @Override + public List sliceStatic(InputSpec inputSpec, int maxNumSlices) + { + return getSlicer(inputSpec.getClass()).sliceStatic(inputSpec, maxNumSlices); + } + + @Override + public List sliceDynamic( + InputSpec inputSpec, + int maxNumSlices, + int maxFilesPerSlice, + long maxBytesPerSlice + ) + { + return getSlicer(inputSpec.getClass()).sliceDynamic(inputSpec, maxNumSlices, maxFilesPerSlice, maxBytesPerSlice); + } + + private InputSpecSlicer getSlicer(final Class clazz) + { + final InputSpecSlicer slicer = splitterMap.get(clazz); + + if (slicer == null) { + throw new ISE("Cannot handle inputSpec of class [%s]", clazz.getName()); + } + + return slicer; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSlice.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSlice.java new file mode 100644 index 00000000000..e0b2a701c7b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSlice.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.druid.msq.input; + +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * An input slice that represents nothing. + */ +@JsonTypeName("nil") +public class NilInputSlice implements InputSlice +{ + public static final NilInputSlice INSTANCE = new NilInputSlice(); + + private NilInputSlice() + { + // Singleton. + } + + @Override + public int fileCount() + { + return 0; + } + + @Override + public boolean equals(Object obj) + { + return obj != null && obj.getClass() == this.getClass(); + } + + @Override + public int hashCode() + { + return 0; + } + + @Override + public String toString() + { + return "NilInputSlice"; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.java new file mode 100644 index 00000000000..9276dbfca8f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.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.druid.msq.input; + +import org.apache.druid.msq.counters.CounterTracker; + +import java.util.Collections; +import java.util.function.Consumer; + +/** + * Reads slices of type {@link NilInputSlice}. + */ +public class NilInputSliceReader implements InputSliceReader +{ + public static final NilInputSliceReader INSTANCE = new NilInputSliceReader(); + + private NilInputSliceReader() + { + // Singleton. + } + + @Override + public int numReadableInputs(InputSlice slice) + { + return 0; + } + + @Override + public ReadableInputs attach( + final int inputNumber, + final InputSlice slice, + final CounterTracker counters, + final Consumer warningPublisher + ) + { + return ReadableInputs.segments(Collections.emptyList()); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java new file mode 100644 index 00000000000..58912a372c1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java @@ -0,0 +1,90 @@ +/* + * 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.druid.msq.input; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Collections; + +/** + * An {@link InputSource} that returns nothing (no rows). + */ +@JsonTypeName("nil") +public class NilInputSource implements InputSource +{ + private static final NilInputSource INSTANCE = new NilInputSource(); + + private NilInputSource() + { + // Singleton. + } + + @JsonCreator + public static NilInputSource instance() + { + return INSTANCE; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + public InputSourceReader reader( + final InputRowSchema inputRowSchema, + @Nullable final InputFormat inputFormat, + final File temporaryDirectory + ) + { + return new InputSourceReader() + { + @Override + public CloseableIterator read() + { + return CloseableIterators.wrap(Collections.emptyIterator(), () -> {}); + } + + @Override + public CloseableIterator sample() + { + return CloseableIterators.wrap(Collections.emptyIterator(), () -> {}); + } + }; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java new file mode 100644 index 00000000000..ada4b719153 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.kernel.StagePartition; + +import javax.annotation.Nullable; + +/** + * A single item of readable input. Generated by {@link InputSliceReader#attach} from an {@link InputSlice}. + * + * Each item is either readable as a {@link org.apache.druid.segment.Segment} or as a {@link ReadableFrameChannel}, + * but not both. Check {@link #hasSegment()} and {@link #hasChannel()} to see which one you have. + */ +public class ReadableInput +{ + @Nullable + private final SegmentWithDescriptor segment; + + @Nullable + private final ReadableFrameChannel inputChannel; + + @Nullable + private final FrameReader frameReader; + + @Nullable + private final StagePartition stagePartition; + + private ReadableInput( + @Nullable SegmentWithDescriptor segment, + @Nullable ReadableFrameChannel channel, + @Nullable FrameReader frameReader, + @Nullable StagePartition stagePartition + ) + { + this.segment = segment; + this.inputChannel = channel; + this.frameReader = frameReader; + this.stagePartition = stagePartition; + + if ((segment == null) == (channel == null)) { + throw new ISE("Provide either 'segment' or 'channel'"); + } + } + + public static ReadableInput segment(final SegmentWithDescriptor segment) + { + return new ReadableInput(segment, null, null, null); + } + + public static ReadableInput channel( + final ReadableFrameChannel inputChannel, + final FrameReader frameReader, + final StagePartition stagePartition + ) + { + return new ReadableInput(null, inputChannel, frameReader, stagePartition); + } + + public boolean hasSegment() + { + return segment != null; + } + + public boolean hasChannel() + { + return inputChannel != null; + } + + public SegmentWithDescriptor getSegment() + { + return Preconditions.checkNotNull(segment, "segment"); + } + + public ReadableFrameChannel getChannel() + { + return Preconditions.checkNotNull(inputChannel, "channel"); + } + + public FrameReader getChannelFrameReader() + { + return Preconditions.checkNotNull(frameReader, "frameReader"); + } + + @Nullable + public StagePartition getStagePartition() + { + return stagePartition; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInputs.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInputs.java new file mode 100644 index 00000000000..449b2b42956 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInputs.java @@ -0,0 +1,92 @@ +/* + * 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.druid.msq.input; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.java.util.common.ISE; + +import javax.annotation.Nullable; +import java.util.Iterator; + +/** + * Iterable sequence of {@link ReadableInput}. Returned by {@link InputSliceReader#attach}. + * + * Inputs in the sequence are either all segments, or all channels. Segments and channels are not mixed. + */ +public class ReadableInputs implements Iterable +{ + private final Iterable iterable; + + @Nullable + private final FrameReader frameReader; + + private ReadableInputs(Iterable iterable, @Nullable FrameReader frameReader) + { + this.iterable = Preconditions.checkNotNull(iterable, "iterable"); + this.frameReader = frameReader; + } + + /** + * Create a sequence of channel-based {@link ReadableInput}. + */ + public static ReadableInputs channels(final Iterable iterable, FrameReader frameReader) + { + return new ReadableInputs(iterable, Preconditions.checkNotNull(frameReader, "frameReader")); + } + + /** + * Create a sequence of segment-based {@link ReadableInput}. + */ + public static ReadableInputs segments(final Iterable iterable) + { + return new ReadableInputs(iterable, null); + } + + /** + * Returns the {@link ReadableInput} as an Iterator. + */ + @Override + public Iterator iterator() + { + return iterable.iterator(); + } + + /** + * Return the frame reader for channel-based inputs. Throws {@link IllegalStateException} if this instance represents + * segments rather than channels. + */ + public FrameReader frameReader() + { + if (frameReader == null) { + throw new ISE("No frame reader; check hasChannels() first"); + } + + return frameReader; + } + + /** + * Whether this instance represents channels. If false, the instance represents segments. + */ + public boolean isChannelBased() + { + return frameReader != null; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/SlicerUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/SlicerUtils.java new file mode 100644 index 00000000000..41891074832 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/SlicerUtils.java @@ -0,0 +1,129 @@ +/* + * 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.druid.msq.input; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.PriorityQueue; +import java.util.function.ToLongFunction; + +public class SlicerUtils +{ + + /** + * Creates "numSlices" lists from "iterator", trying to keep each one as evenly-sized as possible. Some lists may + * be empty. + * + * Items are assigned round-robin. + */ + public static List> makeSlices(final Iterator iterator, final int numSlices) + { + final List> slicesList = new ArrayList<>(numSlices); + + while (slicesList.size() < numSlices) { + slicesList.add(new ArrayList<>()); + } + + int i = 0; + while (iterator.hasNext()) { + final T obj = iterator.next(); + slicesList.get(i % numSlices).add(obj); + i++; + } + + return slicesList; + } + + /** + * Creates "numSlices" lists from "iterator", trying to keep each one as evenly-weighted as possible. Some lists may + * be empty. + * + * Each item is assigned to the split list that has the lowest weight at the time that item is encountered, which + * leads to pseudo-round-robin assignment. + */ + public static List> makeSlices( + final Iterator iterator, + final ToLongFunction weightFunction, + final int numSlices + ) + { + final List> slicesList = new ArrayList<>(numSlices); + final PriorityQueue> pq = new PriorityQueue<>( + numSlices, + + // Break ties with position, so earlier slices fill first. + Comparator., Long>comparing(ListWithWeight::getWeight) + .thenComparing(ListWithWeight::getPosition) + ); + + while (slicesList.size() < numSlices) { + final ArrayList list = new ArrayList<>(); + pq.add(new ListWithWeight<>(list, pq.size(), 0)); + slicesList.add(list); + } + + while (iterator.hasNext()) { + final T obj = iterator.next(); + final long itemWeight = weightFunction.applyAsLong(obj); + final ListWithWeight listWithWeight = pq.remove(); + listWithWeight.getList().add(obj); + pq.add( + new ListWithWeight<>( + listWithWeight.getList(), + listWithWeight.getPosition(), + listWithWeight.getWeight() + itemWeight + ) + ); + } + + return slicesList; + } + + private static class ListWithWeight + { + private final List list; + private final int position; + private final long totalSize; + + public ListWithWeight(List list, int position, long totalSize) + { + this.list = list; + this.position = position; + this.totalSize = totalSize; + } + + public List getList() + { + return list; + } + + public int getPosition() + { + return position; + } + + public long getWeight() + { + return totalSize; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSlice.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSlice.java new file mode 100644 index 00000000000..0252ed78c8d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSlice.java @@ -0,0 +1,111 @@ +/* + * 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.druid.msq.input.external; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.segment.column.RowSignature; + +import java.util.List; +import java.util.Objects; + +/** + * Input slice representing external data. + * + * Corresponds to {@link org.apache.druid.sql.calcite.external.ExternalDataSource}. + */ +@JsonTypeName("external") +public class ExternalInputSlice implements InputSlice +{ + private final List inputSources; + private final InputFormat inputFormat; + private final RowSignature signature; + + @JsonCreator + public ExternalInputSlice( + @JsonProperty("inputSources") List inputSources, + @JsonProperty("inputFormat") InputFormat inputFormat, + @JsonProperty("signature") RowSignature signature + ) + { + this.inputSources = inputSources; + this.inputFormat = inputFormat; + this.signature = signature; + } + + @JsonProperty + public List getInputSources() + { + return inputSources; + } + + @JsonProperty + public InputFormat getInputFormat() + { + return inputFormat; + } + + @JsonProperty + public RowSignature getSignature() + { + return signature; + } + + @Override + public int fileCount() + { + return inputSources.size(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExternalInputSlice that = (ExternalInputSlice) o; + return Objects.equals(inputSources, that.inputSources) + && Objects.equals(inputFormat, that.inputFormat) + && Objects.equals(signature, that.signature); + } + + @Override + public int hashCode() + { + return Objects.hash(inputSources, inputFormat, signature); + } + + @Override + public String toString() + { + return "ExternalInputSlice{" + + "inputSources=" + inputSources + + ", inputFormat=" + inputFormat + + ", signature=" + signature + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java new file mode 100644 index 00000000000..2c51f74c7c8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -0,0 +1,243 @@ +/* + * 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.druid.msq.input.external; + +import com.google.common.collect.Iterators; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.guava.BaseSequence; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.counters.WarningCounters; +import org.apache.druid.msq.indexing.CountableInputSourceReader; +import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.NilInputSource; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.querykit.LazyResourceHolder; +import org.apache.druid.msq.util.DimensionSchemaUtils; +import org.apache.druid.segment.RowAdapters; +import org.apache.druid.segment.RowBasedSegment; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.timeline.SegmentId; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * Reads {@link ExternalInputSlice} using {@link RowBasedSegment} backed by {@link InputSource#reader}. + */ +public class ExternalInputSliceReader implements InputSliceReader +{ + private final File temporaryDirectory; + + public ExternalInputSliceReader(final File temporaryDirectory) + { + this.temporaryDirectory = temporaryDirectory; + } + + @Override + public int numReadableInputs(InputSlice slice) + { + final ExternalInputSlice externalInputSlice = (ExternalInputSlice) slice; + return externalInputSlice.getInputSources().size(); + } + + @Override + public ReadableInputs attach( + final int inputNumber, + final InputSlice slice, + final CounterTracker counters, + final Consumer warningPublisher + ) + { + final ExternalInputSlice externalInputSlice = (ExternalInputSlice) slice; + + return ReadableInputs.segments( + () -> Iterators.transform( + inputSourceSegmentIterator( + externalInputSlice.getInputSources(), + externalInputSlice.getInputFormat(), + externalInputSlice.getSignature(), + temporaryDirectory, + counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()), + counters.warnings(), + warningPublisher + ), + ReadableInput::segment + ) + ); + } + + private static Iterator inputSourceSegmentIterator( + final List inputSources, + final InputFormat inputFormat, + final RowSignature signature, + final File temporaryDirectory, + final ChannelCounters channelCounters, + final WarningCounters warningCounters, + final Consumer warningPublisher + ) + { + final InputRowSchema schema = new InputRowSchema( + new TimestampSpec("__dummy__", "auto", DateTimes.utc(0)), + new DimensionsSpec( + signature.getColumnNames().stream().map( + column -> + DimensionSchemaUtils.createDimensionSchema( + column, + signature.getColumnType(column).orElse(null) + ) + ).collect(Collectors.toList()) + ), + ColumnsFilter.all() + ); + + if (!temporaryDirectory.exists() && !temporaryDirectory.mkdir()) { + throw new ISE("Cannot create temporary directory at [%s]", temporaryDirectory); + } + return Iterators.transform( + inputSources.iterator(), + inputSource -> { + final InputSourceReader reader; + final boolean incrementCounters = isFileBasedInputSource(inputSource); + + if (incrementCounters) { + reader = new CountableInputSourceReader( + inputSource.reader(schema, inputFormat, temporaryDirectory), + channelCounters + ); + } else { + reader = inputSource.reader(schema, inputFormat, temporaryDirectory); + } + + final SegmentId segmentId = SegmentId.dummy("dummy"); + final RowBasedSegment segment = new RowBasedSegment<>( + segmentId, + new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public CloseableIterator make() + { + try { + CloseableIterator baseIterator = reader.read(); + return new CloseableIterator() + { + private InputRow next = null; + + @Override + public void close() throws IOException + { + baseIterator.close(); + } + + @Override + public boolean hasNext() + { + while (true) { + try { + while (next == null && baseIterator.hasNext()) { + next = baseIterator.next(); + } + break; + } + catch (ParseException e) { + warningCounters.incrementWarningCount(CannotParseExternalDataFault.CODE); + warningPublisher.accept(e); + } + } + return next != null; + } + + @Override + public InputRow next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final InputRow row = next; + next = null; + return row; + } + }; + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void cleanup(CloseableIterator iterFromMake) + { + try { + iterFromMake.close(); + // We increment the file count whenever the caller calls clean up. So we can double count here + // if the callers are not careful. + // This logic only works because we are using FilePerSplitHintSpec. Each input source only + // has one file. + if (incrementCounters) { + channelCounters.incrementFileCount(); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } + ), + RowAdapters.standardRow(), + signature + ); + + return new SegmentWithDescriptor( + new LazyResourceHolder<>(() -> Pair.of(segment, () -> {})), + segmentId.toDescriptor() + ); + } + ); + } + + static boolean isFileBasedInputSource(final InputSource inputSource) + { + return !(inputSource instanceof NilInputSource) && !(inputSource instanceof InlineInputSource); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpec.java new file mode 100644 index 00000000000..a35c76b9ecf --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpec.java @@ -0,0 +1,106 @@ +/* + * 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.druid.msq.input.external; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.segment.column.RowSignature; + +import java.util.Objects; + +@JsonTypeName("external") +public class ExternalInputSpec implements InputSpec +{ + private final InputSource inputSource; + private final InputFormat inputFormat; + private final RowSignature signature; + + @JsonCreator + public ExternalInputSpec( + @JsonProperty("inputSource") InputSource inputSource, + @JsonProperty("inputFormat") InputFormat inputFormat, + @JsonProperty("signature") RowSignature signature + ) + { + this.inputSource = Preconditions.checkNotNull(inputSource, "inputSource"); + this.inputFormat = inputFormat; + this.signature = Preconditions.checkNotNull(signature, "signature"); + + if (inputSource.needsFormat()) { + Preconditions.checkNotNull(inputFormat, "inputFormat"); + } + } + + @JsonProperty + public InputSource getInputSource() + { + return inputSource; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public InputFormat getInputFormat() + { + return inputFormat; + } + + @JsonProperty + public RowSignature getSignature() + { + return signature; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExternalInputSpec that = (ExternalInputSpec) o; + return Objects.equals(inputSource, that.inputSource) + && Objects.equals(inputFormat, that.inputFormat) + && Objects.equals(signature, that.signature); + } + + @Override + public int hashCode() + { + return Objects.hash(inputSource, inputFormat, signature); + } + + @Override + public String toString() + { + return "ExternalInputSpec{" + + "inputSources=" + inputSource + + ", inputFormat=" + inputFormat + + ", signature=" + signature + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java new file mode 100644 index 00000000000..8745ea49a72 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java @@ -0,0 +1,166 @@ +/* + * 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.druid.msq.input.external; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Iterators; +import org.apache.druid.data.input.InputFileAttribute; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.MaxSizeSplitHintSpec; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.input.SlicerUtils; +import org.apache.druid.segment.column.RowSignature; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Slices {@link ExternalInputSpec} into {@link ExternalInputSlice} or {@link NilInputSlice}. + */ +public class ExternalInputSpecSlicer implements InputSpecSlicer +{ + @Override + public boolean canSliceDynamic(InputSpec inputSpec) + { + return ((ExternalInputSpec) inputSpec).getInputSource().isSplittable(); + } + + @Override + public List sliceStatic(InputSpec inputSpec, int maxNumSlices) + { + final ExternalInputSpec externalInputSpec = (ExternalInputSpec) inputSpec; + final InputSource inputSource = externalInputSpec.getInputSource(); + final InputFormat inputFormat = externalInputSpec.getInputFormat(); + final RowSignature signature = externalInputSpec.getSignature(); + + // Worker number -> input source for that worker. + final List> workerInputSourcess; + + // Figure out input splits for each worker. + if (inputSource.isSplittable()) { + //noinspection unchecked + final SplittableInputSource splittableInputSource = (SplittableInputSource) inputSource; + + try { + workerInputSourcess = SlicerUtils.makeSlices( + splittableInputSource.createSplits(inputFormat, FilePerSplitHintSpec.INSTANCE) + .map(splittableInputSource::withSplit) + .iterator(), + maxNumSlices + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } else { + workerInputSourcess = Collections.singletonList(Collections.singletonList(inputSource)); + } + + // Sanity check. It is a bug in this method if this exception is ever thrown. + if (workerInputSourcess.size() > maxNumSlices) { + throw new ISE("Generated too many slices [%d > %d]", workerInputSourcess.size(), maxNumSlices); + } + + return IntStream.range(0, maxNumSlices) + .mapToObj( + workerNumber -> { + final List workerInputSources; + + if (workerNumber < workerInputSourcess.size()) { + workerInputSources = workerInputSourcess.get(workerNumber); + } else { + workerInputSources = Collections.emptyList(); + } + + if (workerInputSources.isEmpty()) { + return NilInputSlice.INSTANCE; + } else { + return new ExternalInputSlice(workerInputSources, inputFormat, signature); + } + } + ) + .collect(Collectors.toList()); + } + + @Override + public List sliceDynamic( + InputSpec inputSpec, + int maxNumSlices, + int maxFilesPerSlice, + long maxBytesPerSlice + ) + { + final ExternalInputSpec externalInputSpec = (ExternalInputSpec) inputSpec; + + if (!externalInputSpec.getInputSource().isSplittable()) { + return sliceStatic(inputSpec, 1); + } + + final SplittableInputSource inputSource = (SplittableInputSource) externalInputSpec.getInputSource(); + final MaxSizeSplitHintSpec maxSizeSplitHintSpec = new MaxSizeSplitHintSpec( + new HumanReadableBytes(maxBytesPerSlice), + maxFilesPerSlice + ); + + final long numSlices; + + try { + numSlices = inputSource.createSplits(externalInputSpec.getInputFormat(), maxSizeSplitHintSpec).count(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + return sliceStatic(inputSpec, (int) Math.min(numSlices, maxNumSlices)); + } + + @VisibleForTesting + static class FilePerSplitHintSpec implements SplitHintSpec + { + static FilePerSplitHintSpec INSTANCE = new FilePerSplitHintSpec(); + + private FilePerSplitHintSpec() + { + // Singleton. + } + + @Override + public Iterator> split( + final Iterator inputIterator, + final Function inputAttributeExtractor + ) + { + return Iterators.transform(inputIterator, Collections::singletonList); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java new file mode 100644 index 00000000000..c02bcc44d86 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java @@ -0,0 +1,126 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Iterators; +import it.unimi.dsi.fastutil.ints.Int2IntAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2IntSortedMap; +import org.apache.druid.msq.input.SlicerUtils; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Set of partitions that have been collected onto specific workers: each partition is on exactly one worker. + */ +public class CollectedReadablePartitions implements ReadablePartitions +{ + private final int stageNumber; + private final Int2IntSortedMap partitionToWorkerMap; + + CollectedReadablePartitions(final int stageNumber, final Int2IntSortedMap partitionToWorkerMap) + { + this.stageNumber = stageNumber; + this.partitionToWorkerMap = partitionToWorkerMap; + } + + @JsonCreator + private CollectedReadablePartitions( + @JsonProperty("stageNumber") final int stageNumber, + @JsonProperty("partitionToWorkerMap") final Map partitionToWorkerMap + ) + { + this(stageNumber, new Int2IntAVLTreeMap(partitionToWorkerMap)); + } + + @Override + public Iterator iterator() + { + return Iterators.transform( + partitionToWorkerMap.int2IntEntrySet().iterator(), + entry -> ReadablePartition.collected(stageNumber, entry.getIntValue(), entry.getIntKey()) + ); + } + + @Override + public List split(int maxNumSplits) + { + return SlicerUtils.makeSlices(partitionToWorkerMap.int2IntEntrySet().iterator(), maxNumSplits) + .stream() + .map( + entries -> { + final Int2IntSortedMap map = new Int2IntAVLTreeMap(); + + for (final Int2IntMap.Entry entry : entries) { + map.put(entry.getIntKey(), entry.getIntValue()); + } + + return new CollectedReadablePartitions(stageNumber, map); + } + ) + .collect(Collectors.toList()); + } + + @JsonProperty + int getStageNumber() + { + return stageNumber; + } + + @JsonProperty + Int2IntSortedMap getPartitionToWorkerMap() + { + return partitionToWorkerMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CollectedReadablePartitions that = (CollectedReadablePartitions) o; + return stageNumber == that.stageNumber && Objects.equals(partitionToWorkerMap, that.partitionToWorkerMap); + } + + @Override + public int hashCode() + { + return Objects.hash(stageNumber, partitionToWorkerMap); + } + + @Override + public String toString() + { + return "CollectedReadablePartitions{" + + "stageNumber=" + stageNumber + + ", partitionToWorkerMap=" + partitionToWorkerMap + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java new file mode 100644 index 00000000000..b56a574ed37 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java @@ -0,0 +1,111 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Iterables; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Combines a set of {@link ReadablePartitions} into a single instance. + */ +public class CombinedReadablePartitions implements ReadablePartitions +{ + private final List readablePartitions; + + @JsonCreator + public CombinedReadablePartitions(@JsonProperty("children") final List readablePartitions) + { + this.readablePartitions = readablePartitions; + } + + @Override + public List split(final int maxNumSplits) + { + // Split each item of "readablePartitions", then combine all the 0s, all the 1s, all the 2s, etc. + final List> splits = + readablePartitions.stream().map(rp -> rp.split(maxNumSplits)).collect(Collectors.toList()); + + final List retVal = new ArrayList<>(); + + for (int i = 0; i < maxNumSplits; i++) { + final List combo = new ArrayList<>(); + + for (int j = 0; j < readablePartitions.size(); j++) { + if (splits.get(j).size() > i) { + combo.add(splits.get(j).get(i)); + } + } + + if (combo.size() == 1) { + retVal.add(Iterables.getOnlyElement(combo)); + } else { + retVal.add(new CombinedReadablePartitions(combo)); + } + } + + return retVal; + } + + @Override + public Iterator iterator() + { + return Iterables.concat(readablePartitions).iterator(); + } + + @JsonProperty("children") + List getReadablePartitions() + { + return readablePartitions; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CombinedReadablePartitions that = (CombinedReadablePartitions) o; + return Objects.equals(readablePartitions, that.readablePartitions); + } + + @Override + public int hashCode() + { + return Objects.hash(readablePartitions); + } + + @Override + public String toString() + { + return "CombinedReadablePartitions{" + + "readablePartitions=" + readablePartitions + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/InputChannels.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/InputChannels.java new file mode 100644 index 00000000000..dfe5147d087 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/InputChannels.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.druid.msq.input.stage; + +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.msq.kernel.StagePartition; + +import java.io.IOException; + +/** + * Provides a way to open channels to read the outputs of prior stages. Used by {@link StageInputSliceReader}. + */ +public interface InputChannels +{ + /** + * Open a channel to the given output partition of the given stage. + */ + ReadableFrameChannel openChannel(StagePartition stagePartition) throws IOException; + + /** + * Frame reader for output of the given stage. + */ + FrameReader frameReader(int stageNumber); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartition.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartition.java new file mode 100644 index 00000000000..99098d1d4cb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartition.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.druid.msq.input.stage; + +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import it.unimi.dsi.fastutil.ints.IntSortedSet; +import org.apache.druid.java.util.common.IAE; + +import java.util.Objects; + +/** + * Represents an output partition associated with a particular stage. Each partition is readable from one or + * more workers. + */ +public class ReadablePartition +{ + private final int stageNumber; + private final IntSortedSet workerNumbers; + private final int partitionNumber; + + private ReadablePartition(final int stageNumber, final IntSortedSet workerNumbers, final int partitionNumber) + { + this.stageNumber = stageNumber; + this.workerNumbers = workerNumbers; + this.partitionNumber = partitionNumber; + + if (workerNumbers.isEmpty()) { + throw new IAE("Cannot have empty worker set"); + } + } + + /** + * Returns an output partition that is striped across {@code numWorkers} workers. + */ + public static ReadablePartition striped(final int stageNumber, final int numWorkers, final int partitionNumber) + { + final IntAVLTreeSet workerNumbers = new IntAVLTreeSet(); + for (int i = 0; i < numWorkers; i++) { + workerNumbers.add(i); + } + + return new ReadablePartition(stageNumber, workerNumbers, partitionNumber); + } + + /** + * Returns an output partition that has been collected onto a single worker. + */ + public static ReadablePartition collected(final int stageNumber, final int workerNumber, final int partitionNumber) + { + final IntAVLTreeSet workerNumbers = new IntAVLTreeSet(); + workerNumbers.add(workerNumber); + + return new ReadablePartition(stageNumber, workerNumbers, partitionNumber); + } + + public int getStageNumber() + { + return stageNumber; + } + + public IntSortedSet getWorkerNumbers() + { + return workerNumbers; + } + + public int getPartitionNumber() + { + return partitionNumber; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReadablePartition that = (ReadablePartition) o; + return stageNumber == that.stageNumber + && partitionNumber == that.partitionNumber + && Objects.equals(workerNumbers, that.workerNumbers); + } + + @Override + public int hashCode() + { + return Objects.hash(stageNumber, workerNumbers, partitionNumber); + } + + @Override + public String toString() + { + return "ReadablePartition{" + + "stageNumber=" + stageNumber + + ", workerNumbers=" + workerNumbers + + ", partitionNumber=" + partitionNumber + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.java new file mode 100644 index 00000000000..a71535fbcfc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.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.druid.msq.input.stage; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import it.unimi.dsi.fastutil.ints.Int2IntAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2IntSortedMap; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * A set of {@link ReadablePartition} representing outputs in terms of stage-partitions. Used by {@link StageInputSlice} + * to represent inputs to a particular worker in a particular stage. + * + * Each implementation of this interface represents a different way that partitions can be distributed. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "collected", value = CollectedReadablePartitions.class), + @JsonSubTypes.Type(name = "striped", value = StripedReadablePartitions.class), + @JsonSubTypes.Type(name = "combined", value = CombinedReadablePartitions.class) +}) +public interface ReadablePartitions extends Iterable +{ + /** + * Splits the current instance into a list of disjoint subsets, up to {@code maxNumSplits}. + */ + List split(int maxNumSplits); + + /** + * Empty set of partitions. + */ + static ReadablePartitions empty() + { + return new CombinedReadablePartitions(Collections.emptyList()); + } + + /** + * Combines various sets of partitions into a single set. + */ + static CombinedReadablePartitions combine(List readablePartitions) + { + return new CombinedReadablePartitions(readablePartitions); + } + + /** + * Returns a set of {@code numPartitions} partitions striped across {@code numWorkers} workers: each worker contains + * a "stripe" of each partition. + */ + static StripedReadablePartitions striped( + final int stageNumber, + final int numWorkers, + final int numPartitions + ) + { + final IntAVLTreeSet partitionNumbers = new IntAVLTreeSet(); + for (int i = 0; i < numPartitions; i++) { + partitionNumbers.add(i); + } + + return new StripedReadablePartitions(stageNumber, numWorkers, partitionNumbers); + } + + /** + * Returns a set of partitions that have been collected onto specific workers: each partition is on exactly + * one worker. + */ + static CollectedReadablePartitions collected( + final int stageNumber, + final Map partitionToWorkerMap + ) + { + if (partitionToWorkerMap instanceof Int2IntSortedMap) { + return new CollectedReadablePartitions(stageNumber, (Int2IntSortedMap) partitionToWorkerMap); + } else { + return new CollectedReadablePartitions(stageNumber, new Int2IntAVLTreeMap(partitionToWorkerMap)); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSlice.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSlice.java new file mode 100644 index 00000000000..eaf47a5df0d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSlice.java @@ -0,0 +1,96 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.input.InputSlice; + +import java.util.Objects; + +/** + * Input slice representing some partitions of a stage. + * + * Sliced from {@link StageInputSpec} by {@link StageInputSpecSlicer}. + */ +@JsonTypeName("stage") +public class StageInputSlice implements InputSlice +{ + private final int stage; + private final ReadablePartitions partitions; + + @JsonCreator + public StageInputSlice( + @JsonProperty("stage") int stageNumber, + @JsonProperty("partitions") ReadablePartitions partitions + ) + { + this.stage = stageNumber; + this.partitions = Preconditions.checkNotNull(partitions, "partitions"); + } + + @JsonProperty("stage") + public int getStageNumber() + { + return stage; + } + + @JsonProperty("partitions") + public ReadablePartitions getPartitions() + { + return partitions; + } + + @Override + public int fileCount() + { + return 0; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StageInputSlice that = (StageInputSlice) o; + return stage == that.stage && Objects.equals(partitions, that.partitions); + } + + @Override + public int hashCode() + { + return Objects.hash(stage, partitions); + } + + @Override + public String toString() + { + return "StageInputSpec{" + + "stage=" + stage + + ", partitions=" + partitions + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java new file mode 100644 index 00000000000..782952dae50 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java @@ -0,0 +1,96 @@ +/* + * 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.druid.msq.input.stage; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.indexing.CountingReadableFrameChannel; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.StagePartition; + +import java.io.IOException; +import java.util.function.Consumer; + +/** + * Reads slices of type {@link StageInputSlice}. + */ +public class StageInputSliceReader implements InputSliceReader +{ + private final String queryId; + private final InputChannels inputChannels; + + public StageInputSliceReader(String queryId, InputChannels inputChannels) + { + this.queryId = queryId; + this.inputChannels = inputChannels; + } + + @Override + public int numReadableInputs(final InputSlice slice) + { + final StageInputSlice stageInputSlice = (StageInputSlice) slice; + return Iterables.size(stageInputSlice.getPartitions()); + } + + @Override + public ReadableInputs attach( + final int inputNumber, + final InputSlice slice, + final CounterTracker counters, + final Consumer warningPublisher + ) + { + final StageInputSlice stageInputSlice = (StageInputSlice) slice; + final StageId stageId = new StageId(queryId, stageInputSlice.getStageNumber()); + final FrameReader frameReader = inputChannels.frameReader(stageInputSlice.getStageNumber()); + + return ReadableInputs.channels( + () -> Iterators.transform( + stageInputSlice.getPartitions().iterator(), + partition -> { + final StagePartition stagePartition = new StagePartition(stageId, partition.getPartitionNumber()); + + try { + return ReadableInput.channel( + new CountingReadableFrameChannel( + inputChannels.openChannel(stagePartition), + counters.channel(CounterNames.inputChannel(inputNumber)), + stagePartition.getPartitionNumber() + ), + frameReader, + stagePartition + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ), + frameReader + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpec.java new file mode 100644 index 00000000000..189ead4e73e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpec.java @@ -0,0 +1,75 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.msq.input.InputSpec; + +import java.util.Objects; + +/** + * Input spec representing another stage in the same query. + */ +@JsonTypeName("stage") +public class StageInputSpec implements InputSpec +{ + private final int stage; + + @JsonCreator + public StageInputSpec(@JsonProperty("stage") int stageNumber) + { + this.stage = stageNumber; + } + + @JsonProperty("stage") + public int getStageNumber() + { + return stage; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StageInputSpec that = (StageInputSpec) o; + return stage == that.stage; + } + + @Override + public int hashCode() + { + return Objects.hash(stage); + } + + @Override + public String toString() + { + return "StageInputSpec{" + + "stage=" + stage + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpecSlicer.java new file mode 100644 index 00000000000..ad41b4234e8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSpecSlicer.java @@ -0,0 +1,82 @@ +/* + * 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.druid.msq.input.stage; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; + +import java.util.ArrayList; +import java.util.List; + +/** + * Slices {@link StageInputSpec} into {@link StageInputSlice}. + */ +public class StageInputSpecSlicer implements InputSpecSlicer +{ + // Stage number -> partitions for that stage + private final Int2ObjectMap stagePartitionsMap; + + public StageInputSpecSlicer(final Int2ObjectMap stagePartitionsMap) + { + this.stagePartitionsMap = stagePartitionsMap; + } + + @Override + public boolean canSliceDynamic(InputSpec inputSpec) + { + return false; + } + + @Override + public List sliceStatic(InputSpec inputSpec, int maxNumSlices) + { + final StageInputSpec stageInputSpec = (StageInputSpec) inputSpec; + + final ReadablePartitions stagePartitions = stagePartitionsMap.get(stageInputSpec.getStageNumber()); + + if (stagePartitions == null) { + throw new ISE("Stage [%d] not available", stageInputSpec.getStageNumber()); + } + + // Decide how many workers to use, and assign inputs. + final List workerPartitions = stagePartitions.split(maxNumSlices); + final List retVal = new ArrayList<>(); + + for (final ReadablePartitions partitions : workerPartitions) { + retVal.add(new StageInputSlice(stageInputSpec.getStageNumber(), partitions)); + } + + return retVal; + } + + @Override + public List sliceDynamic( + InputSpec inputSpec, + int maxNumSlices, + int maxFilesPerSlice, + long maxBytesPerSlice + ) + { + throw new UnsupportedOperationException("Cannot sliceDynamic."); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java new file mode 100644 index 00000000000..24edcc1b6d9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java @@ -0,0 +1,132 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Iterators; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import it.unimi.dsi.fastutil.ints.IntSortedSet; +import org.apache.druid.msq.input.SlicerUtils; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +/** + * Set of partitions striped across {@code numWorkers} workers: each worker contains a "stripe" of each partition. + */ +public class StripedReadablePartitions implements ReadablePartitions +{ + private final int stageNumber; + private final int numWorkers; + private final IntSortedSet partitionNumbers; + + StripedReadablePartitions(final int stageNumber, final int numWorkers, final IntSortedSet partitionNumbers) + { + this.stageNumber = stageNumber; + this.numWorkers = numWorkers; + this.partitionNumbers = partitionNumbers; + } + + @JsonCreator + private StripedReadablePartitions( + @JsonProperty("stageNumber") final int stageNumber, + @JsonProperty("numWorkers") final int numWorkers, + @JsonProperty("partitionNumbers") final Set partitionNumbers + ) + { + this(stageNumber, numWorkers, new IntAVLTreeSet(partitionNumbers)); + } + + @Override + public Iterator iterator() + { + return Iterators.transform( + partitionNumbers.iterator(), + partitionNumber -> ReadablePartition.striped(stageNumber, numWorkers, partitionNumber) + ); + } + + @Override + public List split(final int maxNumSplits) + { + final List retVal = new ArrayList<>(); + + for (List entries : SlicerUtils.makeSlices(partitionNumbers.iterator(), maxNumSplits)) { + if (!entries.isEmpty()) { + retVal.add(new StripedReadablePartitions(stageNumber, numWorkers, new IntAVLTreeSet(entries))); + } + } + + return retVal; + } + + @JsonProperty + int getStageNumber() + { + return stageNumber; + } + + @JsonProperty + int getNumWorkers() + { + return numWorkers; + } + + @JsonProperty + IntSortedSet getPartitionNumbers() + { + return partitionNumbers; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StripedReadablePartitions that = (StripedReadablePartitions) o; + return stageNumber == that.stageNumber + && numWorkers == that.numWorkers + && Objects.equals(partitionNumbers, that.partitionNumbers); + } + + @Override + public int hashCode() + { + return Objects.hash(stageNumber, numWorkers, partitionNumbers); + } + + @Override + public String toString() + { + return "StripedReadablePartitions{" + + "stageNumber=" + stageNumber + + ", numWorkers=" + numWorkers + + ", partitionNumbers=" + partitionNumbers + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java new file mode 100644 index 00000000000..3ca48ef9cbd --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input.table; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.query.SegmentDescriptor; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment. + * (SegmentDescriptor only provides the clipped interval.) + * + * To keep the serialized form lightweight, the full interval is only serialized if it is different from the + * clipped interval. + * + * It is possible to deserialize this class as {@link SegmentDescriptor}. However, going the other direction is + * not a good idea, because the {@link #fullInterval} will not end up being set correctly. + */ +public class RichSegmentDescriptor extends SegmentDescriptor +{ + @Nullable + private final Interval fullInterval; + + public RichSegmentDescriptor( + final Interval fullInterval, + final Interval interval, + final String version, + final int partitionNumber + ) + { + super(interval, version, partitionNumber); + this.fullInterval = interval.equals(Preconditions.checkNotNull(fullInterval, "fullInterval")) ? null : fullInterval; + } + + @JsonCreator + static RichSegmentDescriptor fromJson( + @JsonProperty("fi") @Nullable final Interval fullInterval, + @JsonProperty("itvl") final Interval interval, + @JsonProperty("ver") final String version, + @JsonProperty("part") final int partitionNumber + ) + { + return new RichSegmentDescriptor( + fullInterval != null ? fullInterval : interval, + interval, + version, + partitionNumber + ); + } + + public Interval getFullInterval() + { + return fullInterval == null ? getInterval() : fullInterval; + } + + @JsonProperty("fi") + @JsonInclude(JsonInclude.Include.NON_NULL) + @Nullable + public Interval getFullIntervalForJson() + { + return fullInterval; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + RichSegmentDescriptor that = (RichSegmentDescriptor) o; + return Objects.equals(fullInterval, that.fullInterval); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), fullInterval); + } + + @Override + public String toString() + { + return "RichSegmentDescriptor{" + + "fullInterval=" + (fullInterval == null ? getInterval() : fullInterval) + + ", interval=" + getInterval() + + ", version='" + getVersion() + '\'' + + ", partitionNumber=" + getPartitionNumber() + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java new file mode 100644 index 00000000000..1a94efa1c5a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -0,0 +1,78 @@ +/* + * 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.druid.msq.input.table; + +import com.google.common.base.Preconditions; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.Segment; + +import java.io.Closeable; +import java.util.Objects; + +public class SegmentWithDescriptor implements Closeable +{ + private final ResourceHolder segmentHolder; + private final SegmentDescriptor descriptor; + + public SegmentWithDescriptor( + final ResourceHolder segmentHolder, + final SegmentDescriptor descriptor + ) + { + this.segmentHolder = Preconditions.checkNotNull(segmentHolder, "segment"); + this.descriptor = Preconditions.checkNotNull(descriptor, "descriptor"); + } + + public Segment getOrLoadSegment() + { + return segmentHolder.get(); + } + + @Override + public void close() + { + segmentHolder.close(); + } + + public SegmentDescriptor getDescriptor() + { + return descriptor; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentWithDescriptor that = (SegmentWithDescriptor) o; + return Objects.equals(segmentHolder, that.segmentHolder) && Objects.equals(descriptor, that.descriptor); + } + + @Override + public int hashCode() + { + return Objects.hash(segmentHolder, descriptor); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java new file mode 100644 index 00000000000..dadaf0254da --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java @@ -0,0 +1,103 @@ +/* + * 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.druid.msq.input.table; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.msq.input.InputSlice; + +import java.util.List; +import java.util.Objects; + +/** + * Input slice representing a set of segments to read. + * + * Sliced from {@link TableInputSpec} by {@link TableInputSpecSlicer}. + * + * Similar to {@link org.apache.druid.query.spec.MultipleSpecificSegmentSpec} from native queries. + * + * These use {@link RichSegmentDescriptor}, not {@link org.apache.druid.timeline.DataSegment}, to minimize overhead + * in scenarios where the target server already has the segment cached. If the segment isn't cached, the target + * server does need to fetch the full {@link org.apache.druid.timeline.DataSegment} object, so it can get the + * {@link org.apache.druid.segment.loading.LoadSpec} and fetch the segment from deep storage. + */ +@JsonTypeName("segments") +public class SegmentsInputSlice implements InputSlice +{ + private final String dataSource; + private final List descriptors; + + @JsonCreator + public SegmentsInputSlice( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List descriptors + ) + { + this.dataSource = dataSource; + this.descriptors = descriptors; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty("segments") + public List getDescriptors() + { + return descriptors; + } + + @Override + public int fileCount() + { + return descriptors.size(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentsInputSlice that = (SegmentsInputSlice) o; + return Objects.equals(dataSource, that.dataSource) && Objects.equals(descriptors, that.descriptors); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, descriptors); + } + + @Override + public String toString() + { + return "SegmentsInputSlice{" + + "dataSource='" + dataSource + '\'' + + ", descriptors=" + descriptors + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java new file mode 100644 index 00000000000..29236d7b492 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.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.druid.msq.input.table; + +import com.google.common.collect.Iterators; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.segment.Segment; +import org.apache.druid.timeline.SegmentId; + +import java.util.Iterator; +import java.util.List; +import java.util.function.Consumer; + +/** + * Reads slices of type {@link SegmentsInputSlice}. + */ +public class SegmentsInputSliceReader implements InputSliceReader +{ + private final DataSegmentProvider dataSegmentProvider; + + public SegmentsInputSliceReader(final DataSegmentProvider dataSegmentProvider) + { + this.dataSegmentProvider = dataSegmentProvider; + } + + @Override + public int numReadableInputs(InputSlice slice) + { + final SegmentsInputSlice segmentsInputSlice = (SegmentsInputSlice) slice; + return segmentsInputSlice.getDescriptors().size(); + } + + @Override + public ReadableInputs attach( + final int inputNumber, + final InputSlice slice, + final CounterTracker counters, + final Consumer warningPublisher + ) + { + final SegmentsInputSlice segmentsInputSlice = (SegmentsInputSlice) slice; + + return ReadableInputs.segments( + () -> Iterators.transform( + dataSegmentIterator( + segmentsInputSlice.getDataSource(), + segmentsInputSlice.getDescriptors(), + counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()) + ), + ReadableInput::segment + ) + ); + } + + private Iterator dataSegmentIterator( + final String dataSource, + final List descriptors, + final ChannelCounters channelCounters + ) + { + return descriptors.stream().map( + descriptor -> { + final SegmentId segmentId = SegmentId.of( + dataSource, + descriptor.getFullInterval(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ); + + final ResourceHolder segmentHolder = dataSegmentProvider.fetchSegment(segmentId, channelCounters); + return new SegmentWithDescriptor(segmentHolder, descriptor); + } + ).iterator(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java new file mode 100644 index 00000000000..b3337f4307c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java @@ -0,0 +1,130 @@ +/* + * 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.druid.msq.input.table; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.query.filter.DimFilter; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +/** + * Input spec representing a Druid table. + */ +@JsonTypeName("table") +public class TableInputSpec implements InputSpec +{ + private final String dataSource; + private final List intervals; + + @Nullable + private final DimFilter filter; + + /** + * Create a table input spec. + * + * @param dataSource datasource to read + * @param intervals intervals to filter, or null if no time filtering is desired. Interval filtering is strict, + * meaning that when this spec is sliced and read, the returned {@link SegmentWithDescriptor} + * from {@link ReadableInput#getSegment()} are clipped to these intervals. + * @param filter other filters to use for pruning, or null if no pruning is desired. Pruning filters are + * *not strict*, which means that processors must re-apply them when processing the returned + * {@link SegmentWithDescriptor} from {@link ReadableInput#getSegment()}. This matches how + * Broker-based pruning works for native queries. + */ + @JsonCreator + public TableInputSpec( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("intervals") @Nullable List intervals, + @JsonProperty("filter") @Nullable DimFilter filter + ) + { + this.dataSource = dataSource; + this.intervals = intervals == null ? Intervals.ONLY_ETERNITY : intervals; + this.filter = filter; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public List getIntervals() + { + return intervals; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + @Nullable + private List getIntervalsForSerialization() + { + return intervals.equals(Intervals.ONLY_ETERNITY) ? null : intervals; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + @Nullable + public DimFilter getFilter() + { + return filter; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableInputSpec that = (TableInputSpec) o; + return Objects.equals(dataSource, that.dataSource) + && Objects.equals(intervals, that.intervals) + && Objects.equals(filter, that.filter); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, intervals, filter); + } + + @Override + public String toString() + { + return "TableInputSpec{" + + "dataSource='" + dataSource + '\'' + + ", intervals=" + intervals + + ", filter=" + filter + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java new file mode 100644 index 00000000000..7a54daca4fc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -0,0 +1,196 @@ +/* + * 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.druid.msq.input.table; + +import com.google.common.base.Preconditions; +import com.google.common.math.LongMath; +import com.google.common.primitives.Ints; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.input.SlicerUtils; +import org.apache.druid.msq.querykit.DataSegmentTimelineView; +import org.apache.druid.query.filter.DimFilterUtils; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; +import org.joda.time.Interval; + +import java.math.RoundingMode; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.stream.StreamSupport; + +/** + * Slices {@link TableInputSpec} into {@link SegmentsInputSlice}. + */ +public class TableInputSpecSlicer implements InputSpecSlicer +{ + private final DataSegmentTimelineView timelineView; + + public TableInputSpecSlicer(DataSegmentTimelineView timelineView) + { + this.timelineView = timelineView; + } + + @Override + public boolean canSliceDynamic(InputSpec inputSpec) + { + return true; + } + + @Override + public List sliceStatic(InputSpec inputSpec, int maxNumSlices) + { + final TableInputSpec tableInputSpec = (TableInputSpec) inputSpec; + final Set prunedSegmentSet = getPrunedSegmentSet(tableInputSpec); + return makeSlices(tableInputSpec, prunedSegmentSet, maxNumSlices); + } + + @Override + public List sliceDynamic( + InputSpec inputSpec, + int maxNumSlices, + int maxFilesPerSlice, + long maxBytesPerSlice + ) + { + final TableInputSpec tableInputSpec = (TableInputSpec) inputSpec; + final Set prunedSegmentSet = getPrunedSegmentSet(tableInputSpec); + + if (prunedSegmentSet.isEmpty()) { + return Collections.emptyList(); + } + + int totalFiles = 0; + long totalBytes = 0; + + for (DataSegmentWithInterval segmentWithInterval : prunedSegmentSet) { + totalFiles++; + totalBytes += segmentWithInterval.getSegment().getSize(); + } + + final int numSlices = + Math.min( + maxNumSlices, + Math.max( + Ints.checkedCast(LongMath.divide(totalFiles, maxFilesPerSlice, RoundingMode.CEILING)), + Ints.checkedCast(LongMath.divide(totalBytes, maxBytesPerSlice, RoundingMode.CEILING)) + ) + ); + + return makeSlices(tableInputSpec, prunedSegmentSet, numSlices); + } + + private Set getPrunedSegmentSet(final TableInputSpec tableInputSpec) + { + final TimelineLookup timeline = + timelineView.getTimeline(tableInputSpec.getDataSource(), tableInputSpec.getIntervals()).orElse(null); + + if (timeline == null) { + return Collections.emptySet(); + } else { + final Iterator dataSegmentIterator = + tableInputSpec.getIntervals().stream() + .flatMap(interval -> timeline.lookup(interval).stream()) + .flatMap( + holder -> + StreamSupport.stream(holder.getObject().spliterator(), false) + .map( + chunk -> + new DataSegmentWithInterval( + chunk.getObject(), + holder.getInterval() + ) + ) + ).iterator(); + + return DimFilterUtils.filterShards( + tableInputSpec.getFilter(), + () -> dataSegmentIterator, + segment -> segment.getSegment().getShardSpec() + ); + } + } + + private List makeSlices( + final TableInputSpec tableInputSpec, + final Set prunedSegmentSet, + final int maxNumSlices + ) + { + if (prunedSegmentSet.isEmpty()) { + return Collections.emptyList(); + } + + final List> assignments = SlicerUtils.makeSlices( + prunedSegmentSet.iterator(), + segment -> segment.getSegment().getSize(), + maxNumSlices + ); + + final List retVal = new ArrayList<>(); + + for (final List dataSegmentWithIntervals : assignments) { + final List descriptors = new ArrayList<>(); + for (final DataSegmentWithInterval dataSegmentWithInterval : dataSegmentWithIntervals) { + descriptors.add(dataSegmentWithInterval.toRichSegmentDescriptor()); + } + + if (descriptors.isEmpty()) { + retVal.add(NilInputSlice.INSTANCE); + } else { + retVal.add(new SegmentsInputSlice(tableInputSpec.getDataSource(), descriptors)); + } + } + + return retVal; + } + + private static class DataSegmentWithInterval + { + private final DataSegment segment; + private final Interval interval; + + public DataSegmentWithInterval(DataSegment segment, Interval interval) + { + this.segment = Preconditions.checkNotNull(segment, "segment"); + this.interval = Preconditions.checkNotNull(interval, "interval"); + } + + public DataSegment getSegment() + { + return segment; + } + + public RichSegmentDescriptor toRichSegmentDescriptor() + { + return new RichSegmentDescriptor( + segment.getInterval(), + interval, + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ExtraInfoHolder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ExtraInfoHolder.java new file mode 100644 index 00000000000..cd9bb5e80aa --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ExtraInfoHolder.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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Holds extra info that will be passed as the "extra" parameter to + * {@link FrameProcessorFactory#makeProcessors}. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public abstract class ExtraInfoHolder +{ + public static final String INFO_KEY = "info"; + + @Nullable + private final ExtraInfoType extra; + + public ExtraInfoHolder(@Nullable final ExtraInfoType extra) + { + this.extra = extra; + } + + @JsonProperty(INFO_KEY) + @JsonInclude(JsonInclude.Include.NON_NULL) + @Nullable + public ExtraInfoType getExtraInfo() + { + return extra; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExtraInfoHolder that = (ExtraInfoHolder) o; + return Objects.equals(extra, that.extra); + } + + @Override + public int hashCode() + { + return Objects.hash(extra); + } + + @Override + public String toString() + { + return "ExtraInfoHolder{" + + "extra=" + extra + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java new file mode 100644 index 00000000000..d0668507254 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.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.druid.msq.kernel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.loading.DataSegmentPusher; + +import java.io.File; + +/** + * Provides services and objects for the functioning of the frame processors + */ +public interface FrameContext +{ + JoinableFactory joinableFactory(); + + GroupByStrategySelector groupByStrategySelector(); + + RowIngestionMeters rowIngestionMeters(); + + DataSegmentProvider dataSegmentProvider(); + + File tempDir(); + + ObjectMapper jsonMapper(); + + IndexIO indexIO(); + + File persistDir(); + + DataSegmentPusher segmentPusher(); + + IndexMergerV9 indexMerger(); + + WorkerMemoryParameters memoryParameters(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.java new file mode 100644 index 00000000000..4b76d8d281f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; +import java.util.function.Consumer; + +/** + * Property of {@link StageDefinition} that describes its computation logic. + * + * Workers call {@link #makeProcessors} to generate the processors that perform computations within that worker's + * {@link org.apache.druid.frame.processor.FrameProcessorExecutor}. Additionally, provides methods for accumulating + * the results of the processors: {@link #newAccumulatedResult()}, {@link #accumulateResult}, and + * {@link #mergeAccumulatedResult}. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface FrameProcessorFactory, T, R> +{ + /** + * Create processors for a particular worker in a particular stage. The processors will be run on a thread pool, + * with at most "maxOutstandingProcessors" number of processors outstanding at once. + * + * The Sequence returned by {@link ProcessorsAndChannels#processors()} is passed directly to + * {@link org.apache.druid.frame.processor.FrameProcessorExecutor#runAllFully}. + * + * @param stageDefinition stage definition + * @param workerNumber current worker number; some factories use this to determine what work to do + * @param inputSlices input slices for this worker, indexed by input number (one for each + * {@link StageDefinition#getInputSpecs()}) + * @param inputSliceReader reader for the input slices + * @param extra any extra, out-of-band information associated with this particular worker; some + * factories use this to determine what work to do + * @param outputChannelFactory factory for generating output channels. + * @param frameContext Context which provides services needed by frame processors + * @param maxOutstandingProcessors maximum number of processors that will be active at once + * @param counters allows creation of custom processor counters + * @param warningPublisher publisher for warnings encountered during execution + * + * @return a processor sequence, which may be computed lazily; and a list of output channels. + */ + ProcessorsAndChannels makeProcessors( + StageDefinition stageDefinition, + int workerNumber, + List inputSlices, + InputSliceReader inputSliceReader, + @Nullable ExtraInfoType extra, + OutputChannelFactory outputChannelFactory, + FrameContext frameContext, + int maxOutstandingProcessors, + CounterTracker counters, + Consumer warningPublisher + ) throws IOException; + + TypeReference getAccumulatedResultTypeReference(); + + /** + * Produces a "blank slate" result. + */ + R newAccumulatedResult(); + + /** + * Accumulates an additional result. May modify the left-hand side {@code accumulated}. Does not modify the + * right-hand side {@code current}. + */ + R accumulateResult(R accumulated, T current); + + /** + * Merges two accumulated results. May modify the left-hand side {@code accumulated}. Does not modify the right-hand + * side {@code current}. + */ + R mergeAccumulatedResult(R accumulated, R otherAccumulated); + + /** + * Produces an {@link ExtraInfoHolder} wrapper that allows serialization of {@code ExtraInfoType}. + */ + @SuppressWarnings("rawtypes") + ExtraInfoHolder makeExtraInfoHolder(@Nullable ExtraInfoType extra); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/MaxCountShuffleSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/MaxCountShuffleSpec.java new file mode 100644 index 00000000000..f10d3b4ea86 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/MaxCountShuffleSpec.java @@ -0,0 +1,137 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Shuffle spec that generates up to a certain number of output partitions. Commonly used for shuffles between stages. + */ +public class MaxCountShuffleSpec implements ShuffleSpec +{ + private final ClusterBy clusterBy; + private final int partitions; + private final boolean aggregate; + + @JsonCreator + public MaxCountShuffleSpec( + @JsonProperty("clusterBy") final ClusterBy clusterBy, + @JsonProperty("partitions") final int partitions, + @JsonProperty("aggregate") final boolean aggregate + ) + { + this.clusterBy = Preconditions.checkNotNull(clusterBy, "clusterBy"); + this.partitions = partitions; + this.aggregate = aggregate; + + if (partitions < 1) { + throw new IAE("Partition count must be at least 1"); + } + } + + @Override + @JsonProperty("aggregate") + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean doesAggregateByClusterKey() + { + return aggregate; + } + + @Override + public boolean needsStatistics() + { + return partitions > 1 || clusterBy.getBucketByCount() > 0; + } + + @Override + public Either generatePartitions( + @Nullable final ClusterByStatisticsCollector collector, + final int maxNumPartitions + ) + { + if (!needsStatistics()) { + return Either.value(ClusterByPartitions.oneUniversalPartition()); + } else if (partitions > maxNumPartitions) { + return Either.error((long) partitions); + } else { + final ClusterByPartitions generatedPartitions = collector.generatePartitionsWithMaxCount(partitions); + if (generatedPartitions.size() <= maxNumPartitions) { + return Either.value(generatedPartitions); + } else { + return Either.error((long) generatedPartitions.size()); + } + } + } + + @Override + @JsonProperty + public ClusterBy getClusterBy() + { + return clusterBy; + } + + @JsonProperty + int getPartitions() + { + return partitions; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MaxCountShuffleSpec that = (MaxCountShuffleSpec) o; + return partitions == that.partitions + && aggregate == that.aggregate + && Objects.equals(clusterBy, that.clusterBy); + } + + @Override + public int hashCode() + { + return Objects.hash(clusterBy, partitions, aggregate); + } + + @Override + public String toString() + { + return "MaxCountShuffleSpec{" + + "clusterBy=" + clusterBy + + ", partitions=" + partitions + + ", aggregate=" + aggregate + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/NilExtraInfoHolder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/NilExtraInfoHolder.java new file mode 100644 index 00000000000..e8cd896c64f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/NilExtraInfoHolder.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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; + +@JsonTypeName("nil") +public class NilExtraInfoHolder extends ExtraInfoHolder +{ + private static final NilExtraInfoHolder INSTANCE = new NilExtraInfoHolder(); + + private NilExtraInfoHolder() + { + super(null); + } + + @JsonCreator + public static NilExtraInfoHolder instance() + { + return INSTANCE; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java new file mode 100644 index 00000000000..89d882d588a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.kernel; + +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.java.util.common.guava.Sequence; + +/** + * Returned from {@link FrameProcessorFactory#makeProcessors}. + * + * Includes a processor sequence and a list of output channels. + */ +public class ProcessorsAndChannels, T> +{ + private final Sequence workers; + private final OutputChannels outputChannels; + + public ProcessorsAndChannels( + final Sequence workers, + final OutputChannels outputChannels + ) + { + this.workers = workers; + this.outputChannels = outputChannels; + } + + public Sequence processors() + { + return workers; + } + + public OutputChannels getOutputChannels() + { + return outputChannels; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java new file mode 100644 index 00000000000..553e119131d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java @@ -0,0 +1,169 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * Multi-stage query definition. + * + * Query definitions are a directed acyclic graph (DAG) of stages, represented by {@link StageDefinition}. + * + * One stage is the final stage, which could also be called the root stage, and which produces the output for + * the query. Queries have a single final stage: in other words, they do not have multiple outputs. + */ +public class QueryDefinition +{ + private final Map stageDefinitions; + private final StageId finalStage; + + private QueryDefinition( + final Map stageDefinitions, + final StageId finalStage + ) + { + this.stageDefinitions = stageDefinitions; + this.finalStage = finalStage; + } + + @JsonCreator + static QueryDefinition create(@JsonProperty("stages") final List stageDefinitions) + { + final Map stageMap = new HashMap<>(); + final Set nonFinalStages = new HashSet<>(); + final IntSet stageNumbers = new IntOpenHashSet(); + + for (final StageDefinition stage : stageDefinitions) { + if (!stageNumbers.add(stage.getStageNumber())) { + throw new ISE("Cannot accept duplicate stage numbers"); + } + + stageMap.put(stage.getId(), stage); + + for (int stageNumber : stage.getInputStageNumbers()) { + nonFinalStages.add(new StageId(stage.getId().getQueryId(), stageNumber)); + } + } + + for (final StageId nonFinalStageId : nonFinalStages) { + if (!stageMap.containsKey(nonFinalStageId)) { + throw new ISE("Stage [%s] is missing a definition", nonFinalStageId); + } + } + + final int finalStageCandidates = stageMap.size() - nonFinalStages.size(); + + if (finalStageCandidates == 1) { + return new QueryDefinition( + stageMap, + Iterables.getOnlyElement(Sets.difference(stageMap.keySet(), nonFinalStages)) + ); + } else { + throw new IAE("Must have a single final stage, but found [%d] candidates", finalStageCandidates); + } + } + + public static QueryDefinitionBuilder builder() + { + return new QueryDefinitionBuilder(); + } + + public static QueryDefinitionBuilder builder(final QueryDefinition queryDef) + { + return new QueryDefinitionBuilder().addAll(queryDef); + } + + public String getQueryId() + { + return finalStage.getQueryId(); + } + + public StageDefinition getFinalStageDefinition() + { + return getStageDefinition(finalStage); + } + + @JsonProperty("stages") + public List getStageDefinitions() + { + return ImmutableList.copyOf(stageDefinitions.values()); + } + + public StageDefinition getStageDefinition(final int stageNumber) + { + return getStageDefinition(new StageId(getQueryId(), stageNumber)); + } + + public StageDefinition getStageDefinition(final StageId stageId) + { + return Preconditions.checkNotNull(stageDefinitions.get(stageId), "No stageId [%s]", stageId); + } + + /** + * Returns a number that is higher than all current stage numbers. + */ + public int getNextStageNumber() + { + return stageDefinitions.values().stream().mapToInt(StageDefinition::getStageNumber).max().orElse(-1) + 1; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + QueryDefinition that = (QueryDefinition) o; + return Objects.equals(stageDefinitions, that.stageDefinitions) && Objects.equals(finalStage, that.finalStage); + } + + @Override + public int hashCode() + { + return Objects.hash(stageDefinitions, finalStage); + } + + @Override + public String toString() + { + return "QueryDefinition{" + + "stageDefinitions=" + stageDefinitions + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinitionBuilder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinitionBuilder.java new file mode 100644 index 00000000000..166369cc3b2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinitionBuilder.java @@ -0,0 +1,87 @@ +/* + * 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.druid.msq.kernel; + +import com.google.common.base.Preconditions; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; + +/** + * Builder for {@link QueryDefinition}. + */ +public class QueryDefinitionBuilder +{ + private String queryId = UUID.randomUUID().toString(); + private final List stageBuilders = new ArrayList<>(); + + /** + * Package-private: callers should use {@link QueryDefinition#builder()}. + */ + QueryDefinitionBuilder() + { + } + + public QueryDefinitionBuilder queryId(final String queryId) + { + this.queryId = Preconditions.checkNotNull(queryId, "queryId"); + return this; + } + + public QueryDefinitionBuilder add(final StageDefinitionBuilder stageBuilder) + { + stageBuilders.add(stageBuilder); + return this; + } + + public QueryDefinitionBuilder addAll(final QueryDefinition queryDef) + { + for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { + add(StageDefinition.builder(stageDef)); + } + return this; + } + + public QueryDefinitionBuilder addAll(final QueryDefinitionBuilder queryDefBuilder) + { + for (final StageDefinitionBuilder stageDefBuilder : queryDefBuilder.stageBuilders) { + add(stageDefBuilder); + } + return this; + } + + /** + * Returns a number that is higher than all current stage numbers. + */ + public int getNextStageNumber() + { + return stageBuilders.stream().mapToInt(StageDefinitionBuilder::getStageNumber).max().orElse(-1) + 1; + } + + public QueryDefinition build() + { + final List stageDefinitions = + stageBuilders.stream().map(builder -> builder.build(queryId)).collect(Collectors.toList()); + + return QueryDefinition.create(stageDefinitions); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ShuffleSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ShuffleSpec.java new file mode 100644 index 00000000000..319e85850c3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ShuffleSpec.java @@ -0,0 +1,72 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; + +import javax.annotation.Nullable; + +/** + * Describes how outputs of a stage are shuffled. Property of {@link StageDefinition}. + * + * When the output of a stage is shuffled, it is globally sorted and partitioned according to the {@link ClusterBy}. + * Hash-based (non-sorting) shuffle is not currently implemented. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "maxCount", value = MaxCountShuffleSpec.class), + @JsonSubTypes.Type(name = "targetSize", value = TargetSizeShuffleSpec.class) +}) +public interface ShuffleSpec +{ + /** + * Clustering key that will determine how data are partitioned during the shuffle. + */ + ClusterBy getClusterBy(); + + /** + * Whether this stage aggregates by the clustering key or not. + */ + boolean doesAggregateByClusterKey(); + + /** + * Whether {@link #generatePartitions} needs a nonnull collector. + */ + boolean needsStatistics(); + + /** + * Generates a set of partitions based on the provided statistics. + * + * @param collector must be nonnull if {@link #needsStatistics()} is true; may be null otherwise + * @param maxNumPartitions maximum number of partitions to generate + * + * @return either the partition assignment, or (as an error) a number of partitions, greater than maxNumPartitions, + * that would be expected to be created + */ + Either generatePartitions( + @Nullable ClusterByStatisticsCollector collector, + int maxNumPartitions + ); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java new file mode 100644 index 00000000000..21fc56bd345 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java @@ -0,0 +1,353 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.base.Suppliers; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import it.unimi.dsi.fastutil.ints.IntSets; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecs; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollectorImpl; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; + +/** + * Definition of a stage in a multi-stage {@link QueryDefinition}. + * + * Each stage has a list of {@link InputSpec} describing its inputs. The position of each spec within the list is + * its "input number". Some inputs are broadcast to all workers (see {@link #getBroadcastInputNumbers()}). Other, + * non-broadcast inputs are split up across workers. + * + * The number of workers in a stage is at most {@link #getMaxWorkerCount()}. It may be less, depending on the + * {@link WorkerAssignmentStrategy} in play and depending on the number of distinct inputs available. (For example: + * if there is only one input file, then there can be only one worker.) + * + * Each stage has a {@link FrameProcessorFactory} describing the work it does. Output frames written by these + * processors have the signature given by {@link #getSignature()}. + * + * Each stage has a {@link ShuffleSpec} describing the shuffle that occurs as part of the stage. The shuffle spec is + * optional: if none is provided, then the {@link FrameProcessorFactory} directly writes to output partitions. If a + * shuffle spec is provided, then the {@link FrameProcessorFactory} is expected to sort each output frame individually + * according to {@link ShuffleSpec#getClusterBy()}. The execution system handles the rest, including sorting data across + * frames and producing the appropriate output partitions. + * + * The rarely-used parameter {@link #getShuffleCheckHasMultipleValues()} controls whether the execution system + * checks, while shuffling, if the key used for shuffling has any multi-value fields. When this is true, the method + * {@link ClusterByStatisticsCollector#hasMultipleValues} is enabled on collectors + * {@link #createResultKeyStatisticsCollector()}. Its primary purpose is to allow ingestion jobs to detect whether the + * secondary partitioning (CLUSTERED BY) key is multivalued or not. + */ +public class StageDefinition +{ + private static final int PARTITION_STATS_MAX_KEYS = 2 << 15; // Avoid immediate downsample of single-bucket collectors + private static final int PARTITION_STATS_MAX_BUCKETS = 5_000; // Limit for TooManyBuckets + private static final int MAX_PARTITIONS = 25_000; // Limit for TooManyPartitions + + // If adding any fields here, add them to builder(StageDefinition) below too. + private final StageId id; + private final List inputSpecs; + private final IntSet broadcastInputNumbers; + @SuppressWarnings("rawtypes") + private final FrameProcessorFactory processorFactory; + private final RowSignature signature; + private final int maxWorkerCount; + private final boolean shuffleCheckHasMultipleValues; + + @Nullable + private final ShuffleSpec shuffleSpec; + + // Set here to encourage sharing, rather than re-creation. + private final Supplier frameReader; + + @JsonCreator + StageDefinition( + @JsonProperty("id") final StageId id, + @JsonProperty("input") final List inputSpecs, + @JsonProperty("broadcast") final Set broadcastInputNumbers, + @SuppressWarnings("rawtypes") @JsonProperty("processor") final FrameProcessorFactory processorFactory, + @JsonProperty("signature") final RowSignature signature, + @Nullable @JsonProperty("shuffleSpec") final ShuffleSpec shuffleSpec, + @JsonProperty("maxWorkerCount") final int maxWorkerCount, + @JsonProperty("shuffleCheckHasMultipleValues") final boolean shuffleCheckHasMultipleValues + ) + { + this.id = Preconditions.checkNotNull(id, "id"); + this.inputSpecs = Preconditions.checkNotNull(inputSpecs, "inputSpecs"); + + if (broadcastInputNumbers == null) { + this.broadcastInputNumbers = IntSets.emptySet(); + } else if (broadcastInputNumbers instanceof IntSet) { + this.broadcastInputNumbers = (IntSet) broadcastInputNumbers; + } else { + this.broadcastInputNumbers = new IntAVLTreeSet(broadcastInputNumbers); + } + + this.processorFactory = Preconditions.checkNotNull(processorFactory, "processorFactory"); + this.signature = Preconditions.checkNotNull(signature, "signature"); + this.shuffleSpec = shuffleSpec; + this.maxWorkerCount = maxWorkerCount; + this.shuffleCheckHasMultipleValues = shuffleCheckHasMultipleValues; + this.frameReader = Suppliers.memoize(() -> FrameReader.create(signature))::get; + + if (shuffleSpec != null && shuffleSpec.needsStatistics() && shuffleSpec.getClusterBy().getColumns().isEmpty()) { + throw new IAE("Cannot shuffle with spec [%s] and nil clusterBy", shuffleSpec); + } + + for (final String columnName : signature.getColumnNames()) { + if (!signature.getColumnType(columnName).isPresent()) { + throw new ISE("Missing type for column [%s]", columnName); + } + } + + for (final int broadcastInputNumber : this.broadcastInputNumbers) { + if (broadcastInputNumber < 0 || broadcastInputNumber >= inputSpecs.size()) { + throw new ISE("Broadcast input number out of range [%s]", broadcastInputNumber); + } + } + } + + public static StageDefinitionBuilder builder(final int stageNumber) + { + return new StageDefinitionBuilder(stageNumber); + } + + public static StageDefinitionBuilder builder(final StageDefinition stageDef) + { + return new StageDefinitionBuilder(stageDef.getStageNumber()) + .inputs(stageDef.getInputSpecs()) + .broadcastInputs(stageDef.getBroadcastInputNumbers()) + .processorFactory(stageDef.getProcessorFactory()) + .signature(stageDef.getSignature()) + .shuffleSpec(stageDef.getShuffleSpec().orElse(null)) + .maxWorkerCount(stageDef.getMaxWorkerCount()) + .shuffleCheckHasMultipleValues(stageDef.getShuffleCheckHasMultipleValues()); + } + + /** + * Returns a unique stage identifier. + */ + @JsonProperty + public StageId getId() + { + return id; + } + + /** + * Returns input specs for this stage. Positions in this spec list are called "input numbers". + */ + @JsonProperty("input") + public List getInputSpecs() + { + return inputSpecs; + } + + public IntSet getInputStageNumbers() + { + return InputSpecs.getStageNumbers(inputSpecs); + } + + @JsonProperty("broadcast") + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public IntSet getBroadcastInputNumbers() + { + return broadcastInputNumbers; + } + + @JsonProperty("processor") + @SuppressWarnings("rawtypes") + public FrameProcessorFactory getProcessorFactory() + { + return processorFactory; + } + + @JsonProperty + public RowSignature getSignature() + { + return signature; + } + + public boolean doesShuffle() + { + return shuffleSpec != null; + } + + public boolean doesSortDuringShuffle() + { + if (shuffleSpec == null) { + return false; + } else { + return !shuffleSpec.getClusterBy().getColumns().isEmpty() || shuffleSpec.needsStatistics(); + } + } + + public Optional getShuffleSpec() + { + return Optional.ofNullable(shuffleSpec); + } + + /** + * Returns the {@link ClusterBy} of the {@link ShuffleSpec} if set, otherwise {@link ClusterBy#none()}. + */ + public ClusterBy getClusterBy() + { + return shuffleSpec != null ? shuffleSpec.getClusterBy() : ClusterBy.none(); + } + + @Nullable + @JsonProperty("shuffleSpec") + @JsonInclude(JsonInclude.Include.NON_NULL) + ShuffleSpec getShuffleSpecForSerialization() + { + return shuffleSpec; + } + + @JsonProperty + public int getMaxWorkerCount() + { + return maxWorkerCount; + } + + @JsonProperty("shuffleCheckHasMultipleValues") + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + boolean getShuffleCheckHasMultipleValues() + { + return shuffleCheckHasMultipleValues; + } + + public int getMaxPartitionCount() + { + // Pretends to be an instance method, but really returns a constant. Maybe one day this will be defined per stage. + return MAX_PARTITIONS; + } + + public int getStageNumber() + { + return id.getStageNumber(); + } + + public boolean mustGatherResultKeyStatistics() + { + return shuffleSpec != null && shuffleSpec.needsStatistics(); + } + + public Either generatePartitionsForShuffle( + @Nullable ClusterByStatisticsCollector collector + ) + { + if (shuffleSpec == null) { + throw new ISE("No shuffle"); + } else if (mustGatherResultKeyStatistics() && collector == null) { + throw new ISE("Statistics required, but not gathered"); + } else if (!mustGatherResultKeyStatistics() && collector != null) { + throw new ISE("Statistics gathered, but not required"); + } else { + return shuffleSpec.generatePartitions(collector, MAX_PARTITIONS); + } + } + + public ClusterByStatisticsCollector createResultKeyStatisticsCollector() + { + if (!mustGatherResultKeyStatistics()) { + throw new ISE("No statistics needed"); + } + + return ClusterByStatisticsCollectorImpl.create( + shuffleSpec.getClusterBy(), + signature, + PARTITION_STATS_MAX_KEYS, + PARTITION_STATS_MAX_BUCKETS, + shuffleSpec.doesAggregateByClusterKey(), + shuffleCheckHasMultipleValues + ); + } + + public FrameReader getFrameReader() + { + return frameReader.get(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StageDefinition that = (StageDefinition) o; + return maxWorkerCount == that.maxWorkerCount + && shuffleCheckHasMultipleValues == that.shuffleCheckHasMultipleValues + && Objects.equals(id, that.id) + && Objects.equals(inputSpecs, that.inputSpecs) + && Objects.equals(broadcastInputNumbers, that.broadcastInputNumbers) + && Objects.equals(processorFactory, that.processorFactory) + && Objects.equals(signature, that.signature) + && Objects.equals(shuffleSpec, that.shuffleSpec); + } + + @Override + public int hashCode() + { + return Objects.hash( + id, + inputSpecs, + broadcastInputNumbers, + processorFactory, + signature, + maxWorkerCount, + shuffleCheckHasMultipleValues, + shuffleSpec + ); + } + + @Override + public String toString() + { + return "StageDefinition{" + + "id=" + id + + ", inputSpecs=" + inputSpecs + + (!broadcastInputNumbers.isEmpty() ? ", broadcastInputStages=" + broadcastInputNumbers : "") + + ", processorFactory=" + processorFactory + + ", signature=" + signature + + ", maxWorkerCount=" + maxWorkerCount + + ", shuffleSpec=" + shuffleSpec + + (shuffleCheckHasMultipleValues ? ", shuffleCheckHasMultipleValues=" + shuffleCheckHasMultipleValues : "") + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinitionBuilder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinitionBuilder.java new file mode 100644 index 00000000000..e1364eeb16d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinitionBuilder.java @@ -0,0 +1,126 @@ +/* + * 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.druid.msq.kernel; + +import it.unimi.dsi.fastutil.ints.IntRBTreeSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.segment.column.RowSignature; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Builder for {@link StageDefinition}. See class-level javadoc for that class for a description of the parameters. + */ +public class StageDefinitionBuilder +{ + private final int stageNumber; + private final List inputSpecs = new ArrayList<>(); + private final IntSet broadcastInputNumbers = new IntRBTreeSet(); + @SuppressWarnings("rawtypes") + private FrameProcessorFactory processorFactory; + private RowSignature signature = RowSignature.empty(); + private int maxWorkerCount = 1; + private ShuffleSpec shuffleSpec = null; + private boolean shuffleCheckHasMultipleValues = false; + + /** + * Package-private: callers should prefer {@link StageDefinition#builder(int)} rather than this constructor. + */ + StageDefinitionBuilder(final int stageNumber) + { + this.stageNumber = stageNumber; + } + + public StageDefinitionBuilder inputs(final List inputSpecs) + { + this.inputSpecs.clear(); + this.inputSpecs.addAll(inputSpecs); + return this; + } + + public StageDefinitionBuilder inputs(final InputSpec... inputSpecs) + { + return inputs(Arrays.asList(inputSpecs)); + } + + public StageDefinitionBuilder broadcastInputs(final IntSet broadcastInputNumbers) + { + this.broadcastInputNumbers.clear(); + + for (int broadcastInputNumber : broadcastInputNumbers) { + this.broadcastInputNumbers.add(broadcastInputNumber); + } + + return this; + } + + @SuppressWarnings("rawtypes") + public StageDefinitionBuilder processorFactory(final FrameProcessorFactory processorFactory) + { + this.processorFactory = processorFactory; + return this; + } + + public StageDefinitionBuilder signature(final RowSignature signature) + { + this.signature = signature; + return this; + } + + public StageDefinitionBuilder maxWorkerCount(final int maxWorkerCount) + { + this.maxWorkerCount = maxWorkerCount; + return this; + } + + public StageDefinitionBuilder shuffleCheckHasMultipleValues(final boolean shuffleCheckHasMultipleValues) + { + this.shuffleCheckHasMultipleValues = shuffleCheckHasMultipleValues; + return this; + } + + public StageDefinitionBuilder shuffleSpec(final ShuffleSpec shuffleSpec) + { + this.shuffleSpec = shuffleSpec; + return this; + } + + int getStageNumber() + { + return stageNumber; + } + + public StageDefinition build(final String queryId) + { + return new StageDefinition( + new StageId(queryId, stageNumber), + inputSpecs, + broadcastInputNumbers, + processorFactory, + signature, + shuffleSpec, + maxWorkerCount, + shuffleCheckHasMultipleValues + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageId.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageId.java new file mode 100644 index 00000000000..35c8dc43665 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageId.java @@ -0,0 +1,115 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Strings; +import org.apache.druid.common.guava.GuavaUtils; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; + +import java.util.Comparator; +import java.util.Objects; + +/** + * Globally unique stage identifier: query ID plus stage number. + */ +public class StageId implements Comparable +{ + private static final Comparator COMPARATOR = + Comparator.comparing(StageId::getQueryId) + .thenComparing(StageId::getStageNumber); + + private final String queryId; + private final int stageNumber; + + public StageId(final String queryId, final int stageNumber) + { + if (Strings.isNullOrEmpty(queryId)) { + throw new IAE("Null or empty queryId"); + } + + if (stageNumber < 0) { + throw new IAE("Invalid stageNumber [%s]", stageNumber); + } + + this.queryId = queryId; + this.stageNumber = stageNumber; + } + + @JsonCreator + public static StageId fromString(final String s) + { + final int lastUnderscore = s.lastIndexOf('_'); + + if (lastUnderscore > 0 && lastUnderscore < s.length() - 1) { + final Long stageNumber = GuavaUtils.tryParseLong(s.substring(lastUnderscore + 1)); + + if (stageNumber != null && stageNumber >= 0 && stageNumber <= Integer.MAX_VALUE) { + return new StageId(s.substring(0, lastUnderscore), stageNumber.intValue()); + } + } + + throw new IAE("Not a valid stage id: [%s]", s); + } + + public String getQueryId() + { + return queryId; + } + + public int getStageNumber() + { + return stageNumber; + } + + @Override + public int compareTo(StageId that) + { + return COMPARATOR.compare(this, that); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StageId stageId = (StageId) o; + return stageNumber == stageId.stageNumber && Objects.equals(queryId, stageId.queryId); + } + + @Override + public int hashCode() + { + return Objects.hash(queryId, stageNumber); + } + + @Override + @JsonValue + public String toString() + { + return StringUtils.format("%s_%s", queryId, stageNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StagePartition.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StagePartition.java new file mode 100644 index 00000000000..5a8584bc012 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StagePartition.java @@ -0,0 +1,86 @@ +/* + * 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.druid.msq.kernel; + +import org.apache.druid.java.util.common.StringUtils; + +import java.util.Comparator; +import java.util.Objects; + +/** + * Represents a particular output partition of a particular stage. + */ +public class StagePartition implements Comparable +{ + // StagePartition is Comparable because it is used as a sorted map key in InputChannels. + private static final Comparator COMPARATOR = + Comparator.comparing(StagePartition::getStageId) + .thenComparing(StagePartition::getPartitionNumber); + + private final StageId stageId; + private final int partitionNumber; + + public StagePartition(StageId stageId, int partitionNumber) + { + this.stageId = stageId; + this.partitionNumber = partitionNumber; + } + + public StageId getStageId() + { + return stageId; + } + + public int getPartitionNumber() + { + return partitionNumber; + } + + @Override + public int compareTo(final StagePartition that) + { + return COMPARATOR.compare(this, that); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StagePartition that = (StagePartition) o; + return partitionNumber == that.partitionNumber && Objects.equals(stageId, that.stageId); + } + + @Override + public int hashCode() + { + return Objects.hash(stageId, partitionNumber); + } + + @Override + public String toString() + { + return StringUtils.format("%s_%s", stageId, partitionNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/TargetSizeShuffleSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/TargetSizeShuffleSpec.java new file mode 100644 index 00000000000..49f4d718683 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/TargetSizeShuffleSpec.java @@ -0,0 +1,132 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Shuffle spec that generates a variable number of partitions, attempting to keep the number of rows in each partition + * to a particular {@link #targetSize}. Commonly used when generating segments, which we want to have a certain number + * of rows per segment. + */ +public class TargetSizeShuffleSpec implements ShuffleSpec +{ + private final ClusterBy clusterBy; + private final long targetSize; + private final boolean aggregate; + + @JsonCreator + public TargetSizeShuffleSpec( + @JsonProperty("clusterBy") final ClusterBy clusterBy, + @JsonProperty("targetSize") final long targetSize, + @JsonProperty("aggregate") final boolean aggregate + ) + { + this.clusterBy = Preconditions.checkNotNull(clusterBy, "clusterBy"); + this.targetSize = targetSize; + this.aggregate = aggregate; + } + + @Override + @JsonProperty("aggregate") + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean doesAggregateByClusterKey() + { + return aggregate; + } + + @Override + public boolean needsStatistics() + { + return true; + } + + @Override + public Either generatePartitions( + @Nullable final ClusterByStatisticsCollector collector, + final int maxNumPartitions + ) + { + final long expectedPartitions = collector.estimatedTotalWeight() / targetSize; + + if (expectedPartitions > maxNumPartitions) { + return Either.error(expectedPartitions); + } else { + final ClusterByPartitions generatedPartitions = collector.generatePartitionsWithTargetWeight(targetSize); + if (generatedPartitions.size() <= maxNumPartitions) { + return Either.value(generatedPartitions); + } else { + return Either.error((long) generatedPartitions.size()); + } + } + } + + @Override + @JsonProperty + public ClusterBy getClusterBy() + { + return clusterBy; + } + + @JsonProperty + long getTargetSize() + { + return targetSize; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TargetSizeShuffleSpec that = (TargetSizeShuffleSpec) o; + return targetSize == that.targetSize && aggregate == that.aggregate && Objects.equals(clusterBy, that.clusterBy); + } + + @Override + public int hashCode() + { + return Objects.hash(clusterBy, targetSize, aggregate); + } + + @Override + public String toString() + { + return "TargetSizeShuffleSpec{" + + "clusterBy=" + clusterBy + + ", targetSize=" + targetSize + + ", aggregate=" + aggregate + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java new file mode 100644 index 00000000000..b9a3024048b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java @@ -0,0 +1,143 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.input.InputSlice; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +/** + * Represents the work done by a single worker in a single stage. + * + * A list of {@link InputSlice} provides the inputs that the worker reads. These are eventually passed to the + * {@link FrameProcessorFactory#makeProcessors} method for the processor associated with the {@link StageDefinition}. + * + * The entire {@link QueryDefinition} is included, even for other stages, to enable the worker to correctly read + * from the outputs of prior stages. + */ +public class WorkOrder +{ + private final QueryDefinition queryDefinition; + private final int stageNumber; + private final int workerNumber; + private final List workerInputs; + private final ExtraInfoHolder extraInfoHolder; + + @JsonCreator + @SuppressWarnings("rawtypes") + public WorkOrder( + @JsonProperty("query") final QueryDefinition queryDefinition, + @JsonProperty("stage") final int stageNumber, + @JsonProperty("worker") final int workerNumber, + @JsonProperty("input") final List workerInputs, + @JsonProperty("extra") @Nullable final ExtraInfoHolder extraInfoHolder + ) + { + this.queryDefinition = Preconditions.checkNotNull(queryDefinition, "queryDefinition"); + this.stageNumber = stageNumber; + this.workerNumber = workerNumber; + this.workerInputs = Preconditions.checkNotNull(workerInputs, "workerInputs"); + this.extraInfoHolder = extraInfoHolder; + } + + @JsonProperty("query") + public QueryDefinition getQueryDefinition() + { + return queryDefinition; + } + + @JsonProperty("stage") + public int getStageNumber() + { + return stageNumber; + } + + @JsonProperty("worker") + public int getWorkerNumber() + { + return workerNumber; + } + + @JsonProperty("input") + public List getInputs() + { + return workerInputs; + } + + @Nullable + @JsonProperty("extra") + @JsonInclude(JsonInclude.Include.NON_NULL) + ExtraInfoHolder getExtraInfoHolder() + { + return extraInfoHolder; + } + + @Nullable + public Object getExtraInfo() + { + return extraInfoHolder != null ? extraInfoHolder.getExtraInfo() : null; + } + + public StageDefinition getStageDefinition() + { + return queryDefinition.getStageDefinition(stageNumber); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkOrder workOrder = (WorkOrder) o; + return stageNumber == workOrder.stageNumber + && workerNumber == workOrder.workerNumber + && Objects.equals(queryDefinition, workOrder.queryDefinition) + && Objects.equals(workerInputs, workOrder.workerInputs) + && Objects.equals(extraInfoHolder, workOrder.extraInfoHolder); + } + + @Override + public int hashCode() + { + return Objects.hash(queryDefinition, stageNumber, workerInputs, workerNumber, extraInfoHolder); + } + + @Override + public String toString() + { + return "WorkOrder{" + + "queryDefinition=" + queryDefinition + + ", stageNumber=" + stageNumber + + ", workerNumber=" + workerNumber + + ", workerInputs=" + workerInputs + + ", extraInfoHolder=" + extraInfoHolder + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkerAssignmentStrategy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkerAssignmentStrategy.java new file mode 100644 index 00000000000..0e947c77a50 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkerAssignmentStrategy.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.kernel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.exec.Limits; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; + +import java.util.List; +import java.util.OptionalInt; + +/** + * Strategy for assigning input slices to tasks. Influences how {@link InputSpecSlicer} is used. + */ +public enum WorkerAssignmentStrategy +{ + /** + * Use the highest possible number of tasks, while staying within {@link StageDefinition#getMaxWorkerCount()}. + * + * Implemented using {@link InputSpecSlicer#sliceStatic}. + */ + MAX { + @Override + public List assign( + StageDefinition stageDef, + InputSpec inputSpec, + Int2IntMap stageWorkerCountMap, + InputSpecSlicer slicer + ) + { + return slicer.sliceStatic(inputSpec, stageDef.getMaxWorkerCount()); + } + }, + + /** + * Use the lowest possible number of tasks, while keeping each task's workload under + * {@link Limits#MAX_INPUT_FILES_PER_WORKER} files and {@link Limits#MAX_INPUT_BYTES_PER_WORKER} bytes. + * + * Implemented using {@link InputSpecSlicer#sliceDynamic} whenever possible. + */ + AUTO { + @Override + public List assign( + final StageDefinition stageDef, + final InputSpec inputSpec, + final Int2IntMap stageWorkerCountMap, + final InputSpecSlicer slicer + ) + { + if (slicer.canSliceDynamic(inputSpec)) { + return slicer.sliceDynamic( + inputSpec, + stageDef.getMaxWorkerCount(), + Limits.MAX_INPUT_FILES_PER_WORKER, + Limits.MAX_INPUT_BYTES_PER_WORKER + ); + } else { + // In auto mode, if we can't slice inputs dynamically, we instead carry forwards the number of workers from + // the prior stages (or use 1 worker if there are no input stages). + + // To handle cases where the input stage is limited to 1 worker because it is reading 1 giant file, I think it + // would be better to base the number of workers on the number of rows read by the prior stage, which would + // allow later stages to fan out when appropriate. However, we're not currently tracking this information + // in a way that is accessible to the assignment strategy. + + final IntSet inputStages = stageDef.getInputStageNumbers(); + final OptionalInt maxInputStageWorkerCount = inputStages.intStream().map(stageWorkerCountMap).max(); + final int workerCount = maxInputStageWorkerCount.orElse(1); + return slicer.sliceStatic(inputSpec, workerCount); + } + } + }; + + @JsonCreator + public static WorkerAssignmentStrategy fromString(final String name) + { + if (name == null) { + throw new NullPointerException("Null worker assignment strategy"); + } + return valueOf(StringUtils.toUpperCase(name)); + } + + @Override + @JsonValue + public String toString() + { + return StringUtils.toLowerCase(name()); + } + + public abstract List assign( + StageDefinition stageDef, + InputSpec inputSpec, + Int2IntMap stageWorkerCountMap, + InputSpecSlicer slicer + ); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java new file mode 100644 index 00000000000..9c02beea88d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -0,0 +1,546 @@ +/* + * 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.druid.msq.kernel.controller; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.ints.Int2IntAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.QueryValidator; +import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.input.InputSpecSlicerFactory; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.kernel.ExtraInfoHolder; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Kernel for the controller of a multi-stage query. + * + * Instances of this class are state machines for query execution. Kernels do not do any RPC or deal with any data. + * This separation of decision-making from the "real world" allows the decision-making to live in one, + * easy-to-follow place. + * + * @see org.apache.druid.msq.kernel.worker.WorkerStageKernel state machine on the worker side + */ +public class ControllerQueryKernel +{ + private final QueryDefinition queryDef; + + /** + * Stage ID -> tracker for that stage. An extension of the state of this kernel. + */ + private final Map stageTracker = new HashMap<>(); + + /** + * Stage ID -> stages that flow *into* that stage. Computed by {@link #computeStageInflowMap}. + */ + private final ImmutableMap> inflowMap; + + /** + * Stage ID -> stages that *depend on* that stage. Computed by {@link #computeStageOutflowMap}. + */ + private final ImmutableMap> outflowMap; + + /** + * Maintains a running map of (stageId -> pending inflow stages) which need to be completed to provision the stage + * corresponding to the stageId. After initializing, if the value of the entry becomes an empty set, it is removed + * from the map, and the removed entry is added to {@link #readyToRunStages}. + */ + private final Map> pendingInflowMap; + + /** + * Maintains a running count of (stageId -> outflow stages pending on its results). After initializing, if + * the value of the entry becomes an empty set, it is removed from the map and the removed entry is added to + * {@link #effectivelyFinishedStages}. + */ + private final Map> pendingOutflowMap; + + /** + * Tracks those stages which can be initialized safely. + */ + private final Set readyToRunStages = new HashSet<>(); + + /** + * Tracks the stageIds which can be finished. Once returned by {@link #getEffectivelyFinishedStageIds()}, it gets + * cleared and not tracked anymore in this Set. + */ + private final Set effectivelyFinishedStages = new HashSet<>(); + + public ControllerQueryKernel(final QueryDefinition queryDef) + { + this.queryDef = queryDef; + this.inflowMap = ImmutableMap.copyOf(computeStageInflowMap(queryDef)); + this.outflowMap = ImmutableMap.copyOf(computeStageOutflowMap(queryDef)); + + // pendingInflowMap and pendingOutflowMap are wholly separate from inflowMap, so we can edit the Sets. + this.pendingInflowMap = computeStageInflowMap(queryDef); + this.pendingOutflowMap = computeStageOutflowMap(queryDef); + + initializeReadyToRunStages(); + } + + /** + * Creates new kernels, if they can be initialized, and returns the tracked kernels which are in NEW phase + */ + public List createAndGetNewStageIds( + final InputSpecSlicerFactory slicerFactory, + final WorkerAssignmentStrategy assignmentStrategy + ) + { + final Int2IntMap stageWorkerCountMap = new Int2IntAVLTreeMap(); + final Int2ObjectMap stagePartitionsMap = new Int2ObjectAVLTreeMap<>(); + + for (final ControllerStageTracker stageKernel : stageTracker.values()) { + final int stageNumber = stageKernel.getStageDefinition().getStageNumber(); + stageWorkerCountMap.put(stageNumber, stageKernel.getWorkerInputs().workerCount()); + + if (stageKernel.hasResultPartitions()) { + stagePartitionsMap.put(stageNumber, stageKernel.getResultPartitions()); + } + } + + createNewKernels(stageWorkerCountMap, slicerFactory.makeSlicer(stagePartitionsMap), assignmentStrategy); + return stageTracker.values() + .stream() + .filter(controllerStageTracker -> controllerStageTracker.getPhase() == ControllerStagePhase.NEW) + .map(stageKernel -> stageKernel.getStageDefinition().getId()) + .collect(Collectors.toList()); + } + + /** + * @return Stage kernels in this query kernel which can be safely cleaned up and marked as FINISHED. This returns the + * kernel corresponding to a particular stage only once, to reduce the number of stages to iterate through. + * It is expectant of the caller to eventually mark the stage as {@link ControllerStagePhase#FINISHED} after fetching + * the stage kernel + */ + public List getEffectivelyFinishedStageIds() + { + return ImmutableList.copyOf(effectivelyFinishedStages); + } + + /** + * Returns all the kernels which have been initialized and are being tracked + */ + public List getActiveStages() + { + return ImmutableList.copyOf(stageTracker.keySet()); + } + + /** + * Returns a stage's kernel corresponding to a particular stage number + */ + public StageId getStageId(final int stageNumber) + { + return new StageId(queryDef.getQueryId(), stageNumber); + } + + /** + * Returns true if query needs no further processing, i.e. if final stage is successful or if any of the stages have + * been failed + */ + public boolean isDone() + { + return Optional.ofNullable(stageTracker.get(queryDef.getFinalStageDefinition().getId())) + .filter(tracker -> ControllerStagePhase.isSuccessfulTerminalPhase(tracker.getPhase())) + .isPresent() + || stageTracker.values().stream().anyMatch(tracker -> tracker.getPhase() == ControllerStagePhase.FAILED); + } + + /** + * Marks all the successful terminal stages to completion, so that the queryKernel shows a canonical view of + * phases of the stages once it completes + */ + public void markSuccessfulTerminalStagesAsFinished() + { + for (final StageId stageId : getActiveStages()) { + ControllerStagePhase phase = getStagePhase(stageId); + // While the following conditional is redundant currently, it makes logical sense to mark all the "successful + // terminal phases" to FINISHED at the end, hence the if clause. Inside the conditional, depending on the + // terminal phase it resides in, we synthetically mark it to completion (and therefore we need to check which + // stage it is precisely in) + if (ControllerStagePhase.isSuccessfulTerminalPhase(phase)) { + if (phase == ControllerStagePhase.RESULTS_READY) { + finishStage(stageId, false); + } + } + } + } + + /** + * Returns true if all the stages comprising the query definition have been sucessful in producing their results + */ + public boolean isSuccess() + { + return stageTracker.size() == queryDef.getStageDefinitions().size() + && stageTracker.values() + .stream() + .allMatch(tracker -> ControllerStagePhase.isSuccessfulTerminalPhase(tracker.getPhase())); + } + + /** + * Creates a list of work orders, corresponding to each worker, for a particular stageNumber + */ + public Int2ObjectMap createWorkOrders( + final int stageNumber, + @Nullable final Int2ObjectMap extraInfos + ) + { + final Int2ObjectMap retVal = new Int2ObjectAVLTreeMap<>(); + final ControllerStageTracker stageKernel = getStageKernelOrThrow(getStageId(stageNumber)); + + final WorkerInputs workerInputs = stageKernel.getWorkerInputs(); + for (int workerNumber : workerInputs.workers()) { + final Object extraInfo = extraInfos != null ? extraInfos.get(workerNumber) : null; + + //noinspection unchecked + final ExtraInfoHolder extraInfoHolder = + stageKernel.getStageDefinition().getProcessorFactory().makeExtraInfoHolder(extraInfo); + + final WorkOrder workOrder = new WorkOrder( + queryDef, + stageNumber, + workerNumber, + workerInputs.inputsForWorker(workerNumber), + extraInfoHolder + ); + + QueryValidator.validateWorkOrder(workOrder); + retVal.put(workerNumber, workOrder); + } + + return retVal; + } + + private void createNewKernels( + final Int2IntMap stageWorkerCountMap, + final InputSpecSlicer slicer, + final WorkerAssignmentStrategy assignmentStrategy + ) + { + for (final StageId nextStage : readyToRunStages) { + // Create a tracker. + final StageDefinition stageDef = queryDef.getStageDefinition(nextStage); + final ControllerStageTracker stageKernel = ControllerStageTracker.create( + stageDef, + stageWorkerCountMap, + slicer, + assignmentStrategy + ); + stageTracker.put(nextStage, stageKernel); + } + + readyToRunStages.clear(); + } + + /** + * Called by the constructor. Initializes {@link #readyToRunStages} and removes any ready-to-run stages from + * the {@link #pendingInflowMap}. + */ + private void initializeReadyToRunStages() + { + final Iterator>> pendingInflowIterator = pendingInflowMap.entrySet().iterator(); + + while (pendingInflowIterator.hasNext()) { + Map.Entry> stageToInflowStages = pendingInflowIterator.next(); + if (stageToInflowStages.getValue().size() == 0) { + readyToRunStages.add(stageToInflowStages.getKey()); + pendingInflowIterator.remove(); + } + } + } + + // Following section contains the methods which delegate to appropriate stage kernel + + /** + * Delegates call to {@link ControllerStageTracker#getStageDefinition()} + */ + public StageDefinition getStageDefinition(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getStageDefinition(); + } + + /** + * Delegates call to {@link ControllerStageTracker#getPhase()} + */ + public ControllerStagePhase getStagePhase(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getPhase(); + } + + /** + * Delegates call to {@link ControllerStageTracker#hasResultPartitions()} + */ + public boolean doesStageHaveResultPartitions(final StageId stageId) + { + return getStageKernelOrThrow(stageId).hasResultPartitions(); + } + + /** + * Delegates call to {@link ControllerStageTracker#getResultPartitions()} + */ + public ReadablePartitions getResultPartitionsForStage(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getResultPartitions(); + } + + /** + * Delegates call to {@link ControllerStageTracker#getResultPartitionBoundaries()} + */ + public ClusterByPartitions getResultPartitionBoundariesForStage(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getResultPartitionBoundaries(); + } + + /** + * Delegates call to {@link ControllerStageTracker#collectorEncounteredAnyMultiValueField()} + */ + public boolean hasStageCollectorEncounteredAnyMultiValueField(final StageId stageId) + { + return getStageKernelOrThrow(stageId).collectorEncounteredAnyMultiValueField(); + } + + /** + * Delegates call to {@link ControllerStageTracker#getResultObject()} + */ + public Object getResultObjectForStage(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getResultObject(); + } + + /** + * Checks if the stage can be started, delegates call to {@link ControllerStageTracker#start()} for internal phase + * transition and registers the transition in this queryKernel + */ + public void startStage(final StageId stageId) + { + final ControllerStageTracker stageKernel = getStageKernelOrThrow(stageId); + if (stageKernel.getPhase() != ControllerStagePhase.NEW) { + throw new ISE("Cannot start the stage: [%s]", stageId); + } + stageKernel.start(); + transitionStageKernel(stageId, ControllerStagePhase.READING_INPUT); + } + + /** + * Checks if the stage can be finished, delegates call to {@link ControllerStageTracker#finish()} for internal phase + * transition and registers the transition in this query kernel + * + * If the method is called with strict = true, we confirm if the stage can be marked as finished or else + * throw illegal argument exception + */ + public void finishStage(final StageId stageId, final boolean strict) + { + if (strict && !effectivelyFinishedStages.contains(stageId)) { + throw new IAE("Cannot mark the stage: [%s] finished", stageId); + } + getStageKernelOrThrow(stageId).finish(); + effectivelyFinishedStages.remove(stageId); + transitionStageKernel(stageId, ControllerStagePhase.FINISHED); + } + + /** + * Delegates call to {@link ControllerStageTracker#getWorkerInputs()} + */ + public WorkerInputs getWorkerInputsForStage(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getWorkerInputs(); + } + + /** + * Delegates call to {@link ControllerStageTracker#addResultKeyStatisticsForWorker(int, ClusterByStatisticsSnapshot)}. + * If calling this causes transition for the stage kernel, then this gets registered in this query kernel + */ + public void addResultKeyStatisticsForStageAndWorker( + final StageId stageId, + final int workerNumber, + final ClusterByStatisticsSnapshot snapshot + ) + { + ControllerStagePhase newPhase = getStageKernelOrThrow(stageId).addResultKeyStatisticsForWorker( + workerNumber, + snapshot + ); + + // If the phase is POST_READING or FAILED, that implies the kernel has transitioned. We need to account for that + switch (newPhase) { + case POST_READING: + case FAILED: + transitionStageKernel(stageId, newPhase); + break; + } + } + + /** + * Delegates call to {@link ControllerStageTracker#setResultsCompleteForWorker(int, Object)}. If calling this causes + * transition for the stage kernel, then this gets registered in this query kernel + */ + public void setResultsCompleteForStageAndWorker( + final StageId stageId, + final int workerNumber, + final Object resultObject + ) + { + if (getStageKernelOrThrow(stageId).setResultsCompleteForWorker(workerNumber, resultObject)) { + transitionStageKernel(stageId, ControllerStagePhase.RESULTS_READY); + } + } + + /** + * Delegates call to {@link ControllerStageTracker#getFailureReason()} + */ + public MSQFault getFailureReasonForStage(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getFailureReason(); + } + + /** + * Delegates call to {@link ControllerStageTracker#fail()} and registers this transition to FAILED in this query kernel + */ + public void failStage(final StageId stageId) + { + getStageKernelOrThrow(stageId).fail(); + transitionStageKernel(stageId, ControllerStagePhase.FAILED); + } + + /** + * Fetches and returns the stage kernel corresponding to the provided stage id, else throws {@link IAE} + */ + private ControllerStageTracker getStageKernelOrThrow(StageId stageId) + { + ControllerStageTracker stageKernel = stageTracker.get(stageId); + if (stageKernel == null) { + throw new IAE("Cannot find kernel corresponding to stage [%s] in query [%s]", stageId, queryDef.getQueryId()); + } + return stageKernel; + } + + /** + * Whenever a stage kernel changes it phase, the change must be "registered" by calling this method with the stageId + * and the new phase + */ + public void transitionStageKernel(StageId stageId, ControllerStagePhase newPhase) + { + Preconditions.checkArgument( + stageTracker.containsKey(stageId), + "Attempting to modify an unknown stageKernel" + ); + + if (newPhase == ControllerStagePhase.RESULTS_READY) { + // Once the stage has produced its results, we remove it from all the stages depending on this stage (for its + // output). + for (StageId dependentStageId : outflowMap.get(stageId)) { + if (!pendingInflowMap.containsKey(dependentStageId)) { + continue; + } + pendingInflowMap.get(dependentStageId).remove(stageId); + // Check the dependent stage. If it has no dependencies left, it can be marked as to be initialized + if (pendingInflowMap.get(dependentStageId).size() == 0) { + readyToRunStages.add(dependentStageId); + pendingInflowMap.remove(dependentStageId); + } + } + } + + if (ControllerStagePhase.isPostReadingPhase(newPhase)) { + // Once the stage has consumed all the data/input from its dependent stages, we remove it from all the stages + // whose input it was dependent on + for (StageId inputStage : inflowMap.get(stageId)) { + if (!pendingOutflowMap.containsKey(inputStage)) { + continue; + } + pendingOutflowMap.get(inputStage).remove(stageId); + // If no more stage is dependent on the "inputStage's" results, it can be safely transitioned to FINISHED + if (pendingOutflowMap.get(inputStage).size() == 0) { + effectivelyFinishedStages.add(inputStage); + pendingOutflowMap.remove(inputStage); + } + } + } + } + + @VisibleForTesting + ControllerStageTracker getControllerStageKernel(int stageNumber) + { + return stageTracker.get(new StageId(queryDef.getQueryId(), stageNumber)); + } + + /** + * Returns a mapping of stage -> stages that flow *into* that stage. + */ + private static Map> computeStageInflowMap(final QueryDefinition queryDefinition) + { + final Map> retVal = new HashMap<>(); + + for (final StageDefinition stageDef : queryDefinition.getStageDefinitions()) { + final StageId stageId = stageDef.getId(); + retVal.computeIfAbsent(stageId, ignored -> new HashSet<>()); + + for (final int inputStageNumber : queryDefinition.getStageDefinition(stageId).getInputStageNumbers()) { + final StageId inputStageId = new StageId(queryDefinition.getQueryId(), inputStageNumber); + retVal.computeIfAbsent(stageId, ignored -> new HashSet<>()).add(inputStageId); + } + } + + return retVal; + } + + /** + * Returns a mapping of stage -> stages that depend on that stage. + */ + private static Map> computeStageOutflowMap(final QueryDefinition queryDefinition) + { + final Map> retVal = new HashMap<>(); + + for (final StageDefinition stageDef : queryDefinition.getStageDefinitions()) { + final StageId stageId = stageDef.getId(); + retVal.computeIfAbsent(stageId, ignored -> new HashSet<>()); + + for (final int inputStageNumber : queryDefinition.getStageDefinition(stageId).getInputStageNumbers()) { + final StageId inputStageId = new StageId(queryDefinition.getQueryId(), inputStageNumber); + retVal.computeIfAbsent(inputStageId, ignored -> new HashSet<>()).add(stageId); + } + } + + return retVal; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java new file mode 100644 index 00000000000..6686292c1d8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.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.druid.msq.kernel.controller; + +import com.google.common.collect.ImmutableSet; + +import java.util.Set; + +/** + * Phases that a stage can be in, as far as the controller is concerned. + * + * Used by {@link ControllerStageTracker}. + */ +public enum ControllerStagePhase +{ + // Not doing anything yet. Just recently initialized. + NEW { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return false; + } + }, + + // Reading and mapping inputs (using "stateless" operators like filters, transforms which operate on individual records). + READING_INPUT { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == NEW; + } + }, + + // Post the inputs have been read and mapped to frames, in the `POST_READING` stage, we pre-shuffle and determing the partition boundaries. + // This step for a stage spits out the statistics of the data as a whole (and not just the individual records). This + // phase is not required in non-pre shuffle contexts. + POST_READING { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == READING_INPUT; + } + }, + + // Done doing work and all results have been generated. + RESULTS_READY { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == READING_INPUT || priorPhase == POST_READING; + } + }, + + // The worker outputs for this stage might have been cleaned up in the workers, and they cannot be used by + // any other phase. "Metadata" for the stage such as counters are still available however + FINISHED { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == RESULTS_READY; + } + }, + + // Something went wrong. + FAILED { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return true; + } + }; + + public abstract boolean canTransitionFrom(ControllerStagePhase priorPhase); + + private static final Set TERMINAL_PHASES = ImmutableSet.of( + RESULTS_READY, + FINISHED + ); + + /** + * @return true if the phase indicates that the stage has completed its work and produced results successfully + */ + public static boolean isSuccessfulTerminalPhase(final ControllerStagePhase phase) + { + return TERMINAL_PHASES.contains(phase); + } + + private static final Set POST_READING_PHASES = ImmutableSet.of( + POST_READING, + RESULTS_READY, + FINISHED + ); + + /** + * @return true if the phase indicates that the stage has consumed its inputs from the previous stages successfully + */ + public static boolean isPostReadingPhase(final ControllerStagePhase phase) + { + return POST_READING_PHASES.contains(phase); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java new file mode 100644 index 00000000000..1b32deb5341 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -0,0 +1,412 @@ +/* + * 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.druid.msq.kernel.controller; + +import it.unimi.dsi.fastutil.ints.Int2IntAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2IntSortedMap; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * Controller-side state machine for each stage. Used by {@link ControllerQueryKernel} to form the overall state + * machine for an entire query. + * + * Package-private: stage trackers are an internal implementation detail of {@link ControllerQueryKernel}, not meant + * for separate use. + */ +class ControllerStageTracker +{ + private final StageDefinition stageDef; + private final int workerCount; + + private final WorkerInputs workerInputs; + private final IntSet workersWithResultKeyStatistics = new IntAVLTreeSet(); + private final IntSet workersWithResultsComplete = new IntAVLTreeSet(); + + private ControllerStagePhase phase = ControllerStagePhase.NEW; + + @Nullable + private final ClusterByStatisticsCollector resultKeyStatisticsCollector; + + // Result partitions and where they can be read from. + @Nullable + private ReadablePartitions resultPartitions; + + // Boundaries for the result partitions. Only set if this stage is shuffling. + @Nullable + private ClusterByPartitions resultPartitionBoundaries; + + @Nullable + private Object resultObject; + + @Nullable // Set if phase is FAILED + private MSQFault failureReason; + + private ControllerStageTracker( + final StageDefinition stageDef, + final WorkerInputs workerInputs + ) + { + this.stageDef = stageDef; + this.workerCount = workerInputs.workerCount(); + this.workerInputs = workerInputs; + + if (stageDef.mustGatherResultKeyStatistics()) { + this.resultKeyStatisticsCollector = stageDef.createResultKeyStatisticsCollector(); + } else { + this.resultKeyStatisticsCollector = null; + generateResultPartitionsAndBoundaries(); + } + } + + /** + * Given a stage definition and number of workers to available per stage, this method creates a stage tracker. + * This method determines the actual number of workers to use (which in turn depends on the input slices and + * the assignment strategy) + */ + static ControllerStageTracker create( + final StageDefinition stageDef, + final Int2IntMap stageWorkerCountMap, + final InputSpecSlicer slicer, + final WorkerAssignmentStrategy assignmentStrategy + ) + { + final WorkerInputs workerInputs = WorkerInputs.create(stageDef, stageWorkerCountMap, slicer, assignmentStrategy); + return new ControllerStageTracker(stageDef, workerInputs); + } + + /** + * StageDefinition associated with the stage represented by this tracker + */ + StageDefinition getStageDefinition() + { + return stageDef; + } + + /** + * The phase this stage tracker is in. + */ + ControllerStagePhase getPhase() + { + return phase; + } + + /** + * Whether partitions for the results of this stage have been set. + */ + boolean hasResultPartitions() + { + return resultPartitions != null; + } + + /** + * Partitions for the results of the stage associated with this tracker. + */ + ReadablePartitions getResultPartitions() + { + if (resultPartitions == null) { + throw new ISE("Result partition information is not ready yet"); + } else { + return resultPartitions; + } + } + + /** + * @return Partition boundaries for the results of this stage + */ + ClusterByPartitions getResultPartitionBoundaries() + { + if (!getStageDefinition().doesShuffle()) { + throw new ISE("Result partition information is not relevant to this stage because it does not shuffle"); + } else if (resultPartitionBoundaries == null) { + throw new ISE("Result partition information is not ready yet"); + } else { + return resultPartitionBoundaries; + } + } + + /** + * Whether the result key statistics collector for this stage has encountered any multi-valued input at + * any key position. + * + * This method exists because {@link org.apache.druid.timeline.partition.DimensionRangeShardSpec} does not + * support partitioning on multi-valued strings, so we need to know if any multi-valued strings exist in order + * to decide whether we can use this kind of shard spec. + */ + boolean collectorEncounteredAnyMultiValueField() + { + if (resultKeyStatisticsCollector == null) { + throw new ISE("Stage does not gather result key statistics"); + } else if (resultPartitions == null) { + throw new ISE("Result key statistics are not ready"); + } else { + for (int i = 0; i < resultKeyStatisticsCollector.getClusterBy().getColumns().size(); i++) { + if (resultKeyStatisticsCollector.hasMultipleValues(i)) { + return true; + } + } + + return false; + } + } + + /** + * @return Result object associated with this stage + */ + Object getResultObject() + { + if (phase == ControllerStagePhase.FINISHED) { + throw new ISE("Result object has been cleaned up prematurely"); + } else if (phase != ControllerStagePhase.RESULTS_READY) { + throw new ISE("Result object is not ready yet"); + } else if (resultObject == null) { + throw new NullPointerException("resultObject was unexpectedly null"); + } else { + return resultObject; + } + } + + /** + * Marks that the stage is no longer NEW and has started reading inputs (and doing work) + */ + void start() + { + transitionTo(ControllerStagePhase.READING_INPUT); + } + + /** + * Marks that the stage is finished and its results must not be used as they could have cleaned up. + */ + void finish() + { + transitionTo(ControllerStagePhase.FINISHED); + } + + /** + * Inputs to each worker for this particular stage. + */ + WorkerInputs getWorkerInputs() + { + return workerInputs; + } + + /** + * Adds result key statistics for a particular worker number. If statistics have already been added for this worker, + * then this call ignores the new ones and does nothing. + * + * @param workerNumber the worker + * @param snapshot worker statistics + */ + ControllerStagePhase addResultKeyStatisticsForWorker( + final int workerNumber, + final ClusterByStatisticsSnapshot snapshot + ) + { + if (resultKeyStatisticsCollector == null) { + throw new ISE("Stage does not gather result key statistics"); + } + + if (workerNumber < 0 || workerNumber >= workerCount) { + throw new IAE("Invalid workerNumber [%s]", workerNumber); + } + + if (phase != ControllerStagePhase.READING_INPUT) { + throw new ISE("Cannot add result key statistics from stage [%s]", phase); + } + + try { + if (workersWithResultKeyStatistics.add(workerNumber)) { + resultKeyStatisticsCollector.addAll(snapshot); + + if (workersWithResultKeyStatistics.size() == workerCount) { + generateResultPartitionsAndBoundaries(); + + // Phase can become FAILED after generateResultPartitionsAndBoundaries, if there were too many partitions. + if (phase != ControllerStagePhase.FAILED) { + transitionTo(ControllerStagePhase.POST_READING); + } + } + } + } + catch (Exception e) { + // If this op fails, we're in an inconsistent state and must cancel the stage. + fail(); + throw e; + } + return getPhase(); + } + + /** + * Accepts and sets the results that each worker produces for this particular stage + * + * @return true if the results for this stage have been gathered from all the workers, else false + */ + @SuppressWarnings("unchecked") + boolean setResultsCompleteForWorker(final int workerNumber, final Object resultObject) + { + if (workerNumber < 0 || workerNumber >= workerCount) { + throw new IAE("Invalid workerNumber [%s]", workerNumber); + } + + if (resultObject == null) { + throw new NullPointerException("resultObject must not be null"); + } + + // This is unidirectional flow of data. While this works in the current state of MSQ where partial fault tolerance + // is implemented and a query flows in one direction only, rolling back of workers' state and query kernel's + // phase should be allowed to fully support fault tolerance in cases such as: + // 1. Rolling back worker's state in case it fails (and then retries) + // 2. Rolling back query kernel's phase in case the results are lost (and needs workers to retry the computation) + if (workersWithResultsComplete.add(workerNumber)) { + if (this.resultObject == null) { + this.resultObject = resultObject; + } else { + //noinspection unchecked + this.resultObject = getStageDefinition().getProcessorFactory() + .mergeAccumulatedResult(this.resultObject, resultObject); + } + } + + if (workersWithResultsComplete.size() == workerCount) { + transitionTo(ControllerStagePhase.RESULTS_READY); + return true; + } + return false; + } + + /** + * Reason for failure of this stage. + */ + MSQFault getFailureReason() + { + if (phase != ControllerStagePhase.FAILED) { + throw new ISE("No failure"); + } + + return failureReason; + } + + /** + * Marks the stage as failed for no particular reason. + */ + void fail() + { + failForReason(UnknownFault.forMessage(null)); + } + + /** + * Sets {@link #resultPartitions} (always) and {@link #resultPartitionBoundaries}. + * + * If {@link StageDefinition#mustGatherResultKeyStatistics()} is true, this method cannot be called until after + * statistics have been provided to {@link #addResultKeyStatisticsForWorker} for all workers. + */ + private void generateResultPartitionsAndBoundaries() + { + if (resultPartitions != null) { + throw new ISE("Result partitions have already been generated"); + } + + final int stageNumber = stageDef.getStageNumber(); + + if (stageDef.doesShuffle()) { + if (stageDef.mustGatherResultKeyStatistics() && workersWithResultKeyStatistics.size() != workerCount) { + throw new ISE("Cannot generate result partitions without all worker statistics"); + } + + final Either maybeResultPartitionBoundaries = + stageDef.generatePartitionsForShuffle(resultKeyStatisticsCollector); + + if (maybeResultPartitionBoundaries.isError()) { + failForReason(new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); + return; + } + + resultPartitionBoundaries = maybeResultPartitionBoundaries.valueOrThrow(); + resultPartitions = ReadablePartitions.striped( + stageNumber, + workerCount, + resultPartitionBoundaries.size() + ); + } else { + // No reshuffling: retain partitioning from nonbroadcast inputs. + final Int2IntSortedMap partitionToWorkerMap = new Int2IntAVLTreeMap(); + for (int workerNumber : workerInputs.workers()) { + final List slices = workerInputs.inputsForWorker(workerNumber); + for (int inputNumber = 0; inputNumber < slices.size(); inputNumber++) { + final InputSlice slice = slices.get(inputNumber); + + if (slice instanceof StageInputSlice && !stageDef.getBroadcastInputNumbers().contains(inputNumber)) { + final StageInputSlice stageInputSlice = (StageInputSlice) slice; + for (final ReadablePartition partition : stageInputSlice.getPartitions()) { + partitionToWorkerMap.put(partition.getPartitionNumber(), workerNumber); + } + } + } + } + + resultPartitions = ReadablePartitions.collected(stageNumber, partitionToWorkerMap); + } + } + + /** + * Marks the stage as failed and sets the reason for the same. + * + * @param fault reason why this stage has failed + */ + private void failForReason(final MSQFault fault) + { + transitionTo(ControllerStagePhase.FAILED); + + this.failureReason = fault; + + if (resultKeyStatisticsCollector != null) { + resultKeyStatisticsCollector.clear(); + } + } + + void transitionTo(final ControllerStagePhase newPhase) + { + if (newPhase.canTransitionFrom(phase)) { + phase = newPhase; + } else { + throw new IAE("Cannot transition from [%s] to [%s]", phase, newPhase); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/WorkerInputs.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/WorkerInputs.java new file mode 100644 index 00000000000..57e10f784de --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/WorkerInputs.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.kernel.controller; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntSet; +import it.unimi.dsi.fastutil.objects.ObjectIterator; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.IntStream; + +/** + * Represents assignment of {@link InputSlice} to workers. + */ +public class WorkerInputs +{ + // Worker number -> input number -> input slice. + private final Int2ObjectMap> assignmentsMap; + + private WorkerInputs(final Int2ObjectMap> assignmentsMap) + { + this.assignmentsMap = assignmentsMap; + } + + /** + * Create worker assignments for a stage. + */ + public static WorkerInputs create( + final StageDefinition stageDef, + final Int2IntMap stageWorkerCountMap, + final InputSpecSlicer slicer, + final WorkerAssignmentStrategy assignmentStrategy + ) + { + // Split each inputSpec and assign to workers. This list maps worker number -> input number -> input slice. + final Int2ObjectMap> assignmentsMap = new Int2ObjectAVLTreeMap<>(); + final int numInputs = stageDef.getInputSpecs().size(); + + if (numInputs == 0) { + // No inputs: run a single worker. (It might generate some data out of nowhere.) + assignmentsMap.put(0, Collections.singletonList(NilInputSlice.INSTANCE)); + return new WorkerInputs(assignmentsMap); + } + + // Assign input slices to workers. + for (int inputNumber = 0; inputNumber < numInputs; inputNumber++) { + final InputSpec inputSpec = stageDef.getInputSpecs().get(inputNumber); + + if (stageDef.getBroadcastInputNumbers().contains(inputNumber)) { + // Broadcast case: send everything everywhere. + final List broadcastSlices = slicer.sliceStatic(inputSpec, 1); + final InputSlice broadcastSlice = broadcastSlices.isEmpty() + ? NilInputSlice.INSTANCE + : Iterables.getOnlyElement(broadcastSlices); + + for (int workerNumber = 0; workerNumber < stageDef.getMaxWorkerCount(); workerNumber++) { + assignmentsMap.computeIfAbsent( + workerNumber, + ignored -> Arrays.asList(new InputSlice[numInputs]) + ).set(inputNumber, broadcastSlice); + } + } else { + // Non-broadcast case: split slices across workers. + final List slices = assignmentStrategy.assign(stageDef, inputSpec, stageWorkerCountMap, slicer); + + // Flip the slices, so it's worker number -> slices for that worker. + for (int workerNumber = 0; workerNumber < slices.size(); workerNumber++) { + assignmentsMap.computeIfAbsent( + workerNumber, + ignored -> Arrays.asList(new InputSlice[numInputs]) + ).set(inputNumber, slices.get(workerNumber)); + } + } + } + + final ObjectIterator>> assignmentsIterator = + assignmentsMap.int2ObjectEntrySet().iterator(); + + boolean first = true; + while (assignmentsIterator.hasNext()) { + final Int2ObjectMap.Entry> entry = assignmentsIterator.next(); + final List slices = entry.getValue(); + + // Replace all null slices with nil slices: this way, logic later on doesn't have to deal with nulls. + for (int inputNumber = 0; inputNumber < numInputs; inputNumber++) { + if (slices.get(inputNumber) == null) { + slices.set(inputNumber, NilInputSlice.INSTANCE); + } + } + + // Eliminate workers that have no non-nil, non-broadcast inputs. (Except the first one, because if all input + // is nil, *some* worker has to do *something*.) + final boolean hasNonNilNonBroadcastInput = + IntStream.range(0, numInputs) + .anyMatch(i -> + !slices.get(i).equals(NilInputSlice.INSTANCE) // Non-nil + && !stageDef.getBroadcastInputNumbers().contains(i) // Non-broadcast + ); + + if (!first && !hasNonNilNonBroadcastInput) { + assignmentsIterator.remove(); + } + + first = false; + } + + return new WorkerInputs(assignmentsMap); + } + + public List inputsForWorker(final int workerNumber) + { + return Preconditions.checkNotNull(assignmentsMap.get(workerNumber), "worker [%s]", workerNumber); + } + + public IntSet workers() + { + return assignmentsMap.keySet(); + } + + public int workerCount() + { + return assignmentsMap.size(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkerInputs that = (WorkerInputs) o; + return Objects.equals(assignmentsMap, that.assignmentsMap); + } + + @Override + public int hashCode() + { + return Objects.hash(assignmentsMap); + } + + @Override + public String toString() + { + return assignmentsMap.toString(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java new file mode 100644 index 00000000000..b0ed8e5c19d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java @@ -0,0 +1,217 @@ +/* + * 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.druid.msq.kernel.worker; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.Set; + +/** + * Kernel for a worker in a multi-stage query. + * + * Instances of this class are state machines for worker execution. Kernels do not do any RPC or deal with any data. + * This separation of decision-making from the "real world" allows the decision-making to live in one, + * easy-to-follow place. + * + * @see org.apache.druid.msq.kernel.controller.ControllerQueryKernel state machine on the controller side + */ +public class WorkerStageKernel +{ + private final WorkOrder workOrder; + + private WorkerStagePhase phase = WorkerStagePhase.NEW; + + @Nullable + private ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot; + + @Nullable + private ClusterByPartitions resultPartitionBoundaries; + + @Nullable + private Object resultObject; + + @Nullable + private Throwable exceptionFromFail; + + private final Set> postedResultsComplete = new HashSet<>(); + + private WorkerStageKernel(final WorkOrder workOrder) + { + this.workOrder = workOrder; + + if (workOrder.getStageDefinition().doesShuffle() + && !workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { + // Use valueOrThrow instead of a nicer error collection mechanism, because we really don't expect the + // MAX_PARTITIONS to be exceeded here. It would involve having a shuffleSpec that was statically configured + // to use a huge number of partitions. + resultPartitionBoundaries = workOrder.getStageDefinition().generatePartitionsForShuffle(null).valueOrThrow(); + } + } + + public static WorkerStageKernel create(final WorkOrder workOrder) + { + return new WorkerStageKernel(workOrder); + } + + public WorkerStagePhase getPhase() + { + return phase; + } + + public WorkOrder getWorkOrder() + { + return workOrder; + } + + public StageDefinition getStageDefinition() + { + return workOrder.getStageDefinition(); + } + + public void startReading() + { + transitionTo(WorkerStagePhase.READING_INPUT); + } + + public void startPreshuffleWaitingForResultPartitionBoundaries() + { + assertPreshuffleStatisticsNeeded(); + transitionTo(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); + } + + public void startPreshuffleWritingOutput() + { + assertPreshuffleStatisticsNeeded(); + transitionTo(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT); + } + + public void setResultKeyStatisticsSnapshot(final ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot) + { + assertPreshuffleStatisticsNeeded(); + this.resultKeyStatisticsSnapshot = resultKeyStatisticsSnapshot; + } + + public void setResultPartitionBoundaries(final ClusterByPartitions resultPartitionBoundaries) + { + assertPreshuffleStatisticsNeeded(); + this.resultPartitionBoundaries = resultPartitionBoundaries; + } + + public boolean hasResultKeyStatisticsSnapshot() + { + return resultKeyStatisticsSnapshot != null; + } + + public boolean hasResultPartitionBoundaries() + { + return resultPartitionBoundaries != null; + } + + public ClusterByStatisticsSnapshot getResultKeyStatisticsSnapshot() + { + return Preconditions.checkNotNull(resultKeyStatisticsSnapshot, "resultKeyStatisticsSnapshot"); + } + + public ClusterByPartitions getResultPartitionBoundaries() + { + return Preconditions.checkNotNull(resultPartitionBoundaries, "resultPartitionBoundaries"); + } + + @Nullable + public Object getResultObject() + { + if (phase == WorkerStagePhase.RESULTS_READY || phase == WorkerStagePhase.FINISHED) { + return resultObject; + } else { + throw new ISE("Results are not ready yet"); + } + } + + public Throwable getException() + { + if (phase == WorkerStagePhase.FAILED) { + return exceptionFromFail; + } else { + throw new ISE("Stage has not failed"); + } + } + + public void setResultsComplete(Object resultObject) + { + if (resultObject == null) { + throw new NullPointerException("resultObject must not be null"); + } + + transitionTo(WorkerStagePhase.RESULTS_READY); + this.resultObject = resultObject; + } + + public void setStageFinished() + { + transitionTo(WorkerStagePhase.FINISHED); + } + + public void fail(Throwable t) + { + Preconditions.checkNotNull(t, "t"); + + transitionTo(WorkerStagePhase.FAILED); + resultKeyStatisticsSnapshot = null; + resultPartitionBoundaries = null; + + if (exceptionFromFail == null) { + exceptionFromFail = t; + } + } + + public boolean addPostedResultsComplete(Pair stageIdAndWorkerNumber) + { + return postedResultsComplete.add(stageIdAndWorkerNumber); + } + + private void assertPreshuffleStatisticsNeeded() + { + if (!workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { + throw new ISE( + "Result partitioning is not necessary for stage [%s]", + workOrder.getStageDefinition().getId() + ); + } + } + + private void transitionTo(final WorkerStagePhase newPhase) + { + if (newPhase.canTransitionFrom(phase)) { + phase = newPhase; + } else { + throw new IAE("Cannot transition from [%s] to [%s]", phase, newPhase); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java new file mode 100644 index 00000000000..f54aa52349e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java @@ -0,0 +1,87 @@ +/* + * 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.druid.msq.kernel.worker; + +/** + * Phases that a stage can be in, as far as the worker is concerned. + * + * Used by {@link WorkerStageKernel}. + */ +public enum WorkerStagePhase +{ + NEW { + @Override + public boolean canTransitionFrom(final WorkerStagePhase priorPhase) + { + return false; + } + }, + + READING_INPUT { + @Override + public boolean canTransitionFrom(final WorkerStagePhase priorPhase) + { + return priorPhase == NEW; + } + }, + + PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES { + @Override + public boolean canTransitionFrom(final WorkerStagePhase priorPhase) + { + return priorPhase == READING_INPUT; + } + }, + + PRESHUFFLE_WRITING_OUTPUT { + @Override + public boolean canTransitionFrom(final WorkerStagePhase priorPhase) + { + return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES; + } + }, + + RESULTS_READY { + @Override + public boolean canTransitionFrom(final WorkerStagePhase priorPhase) + { + return priorPhase == READING_INPUT || priorPhase == PRESHUFFLE_WRITING_OUTPUT; + } + }, + + FINISHED { + @Override + public boolean canTransitionFrom(final WorkerStagePhase priorPhase) + { + return priorPhase == RESULTS_READY; + } + }, + + // Something went wrong. + FAILED { + @Override + public boolean canTransitionFrom(final WorkerStagePhase priorPhase) + { + return true; + } + }; + + public abstract boolean canTransitionFrom(WorkerStagePhase priorPhase); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java new file mode 100644 index 00000000000..576abd16315 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java @@ -0,0 +1,72 @@ +/* + * 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.druid.msq.querykit; + +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.kernel.ExtraInfoHolder; +import org.apache.druid.msq.kernel.FrameProcessorFactory; +import org.apache.druid.msq.kernel.NilExtraInfoHolder; + +import javax.annotation.Nullable; + +/** + * Basic abstract {@link FrameProcessorFactory} that yields workers that do not require extra info and that + * always return Longs. This base class isn't used for every worker factory, but it is used for many of them. + */ +public abstract class BaseFrameProcessorFactory + implements FrameProcessorFactory, Long, Long> +{ + @Override + public TypeReference getAccumulatedResultTypeReference() + { + return new TypeReference() {}; + } + + @Override + public Long newAccumulatedResult() + { + return 0L; + } + + @Nullable + @Override + public Long accumulateResult(Long accumulated, Long current) + { + return accumulated + current; + } + + @Override + public Long mergeAccumulatedResult(Long accumulated, Long otherAccumulated) + { + return accumulated + otherAccumulated; + } + + @Override + public ExtraInfoHolder makeExtraInfoHolder(@Nullable Object extra) + { + if (extra != null) { + throw new ISE("Expected null 'extra'"); + } + + return NilExtraInfoHolder.instance(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java new file mode 100644 index 00000000000..01bcacfd2e8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -0,0 +1,216 @@ +/* + * 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.druid.msq.querykit; + +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.Query; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.join.JoinableFactoryWrapper; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; + +public abstract class BaseLeafFrameProcessor implements FrameProcessor +{ + private final Query query; + private final ReadableInput baseInput; + private final List inputChannels; + private final ResourceHolder outputChannel; + private final ResourceHolder allocator; + private final BroadcastJoinHelper broadcastJoinHelper; + + private Function segmentMapFn; + + protected BaseLeafFrameProcessor( + final Query query, + final ReadableInput baseInput, + final Int2ObjectMap sideChannels, + final JoinableFactoryWrapper joinableFactory, + final ResourceHolder outputChannel, + final ResourceHolder allocator, + final long memoryReservedForBroadcastJoin + ) + { + this.query = query; + this.baseInput = baseInput; + this.outputChannel = outputChannel; + this.allocator = allocator; + + final Pair, BroadcastJoinHelper> inputChannelsAndBroadcastJoinHelper = + makeInputChannelsAndBroadcastJoinHelper( + query.getDataSource(), + baseInput, + sideChannels, + joinableFactory, + memoryReservedForBroadcastJoin + ); + + this.inputChannels = inputChannelsAndBroadcastJoinHelper.lhs; + this.broadcastJoinHelper = inputChannelsAndBroadcastJoinHelper.rhs; + } + + @Override + public List inputChannels() + { + return inputChannels; + } + + @Override + public List outputChannels() + { + return Collections.singletonList(outputChannel.get()); + } + + @Override + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + { + if (!initializeSegmentMapFn(readableInputs)) { + return ReturnOrAwait.awaitAll(broadcastJoinHelper.getSideChannelNumbers()); + } else if (readableInputs.size() != inputChannels.size()) { + return ReturnOrAwait.awaitAll(inputChannels.size()); + } else if (baseInput.hasSegment()) { + return runWithSegment(baseInput.getSegment()); + } else { + assert baseInput.hasChannel(); + return runWithInputChannel(baseInput.getChannel(), baseInput.getChannelFrameReader()); + } + } + + @Override + public void cleanup() throws IOException + { + // Don't close the output channel, because multiple workers write to the same channel. + // The channel should be closed by the caller. + FrameProcessors.closeAll(inputChannels(), Collections.emptyList(), outputChannel, allocator); + } + + protected MemoryAllocator getAllocator() + { + return allocator.get(); + } + + protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; + + protected abstract ReturnOrAwait runWithInputChannel( + ReadableFrameChannel inputChannel, + FrameReader inputFrameReader + ) throws IOException; + + /** + * Helper intended to be used by subclasses. Applies {@link #segmentMapFn}, which applies broadcast joins + * if applicable to this query. + */ + protected SegmentReference mapSegment(final Segment segment) + { + return segmentMapFn.apply(ReferenceCountingSegment.wrapRootGenerationSegment(segment)); + } + + private boolean initializeSegmentMapFn(final IntSet readableInputs) + { + if (segmentMapFn != null) { + return true; + } else if (broadcastJoinHelper == null) { + segmentMapFn = Function.identity(); + return true; + } else { + final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); + + if (retVal) { + segmentMapFn = broadcastJoinHelper.makeSegmentMapFn(query); + } + + return retVal; + } + } + + /** + * Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels. + */ + private static Pair, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper( + final DataSource dataSource, + final ReadableInput baseInput, + final Int2ObjectMap sideChannels, + final JoinableFactoryWrapper joinableFactory, + final long memoryReservedForBroadcastJoin + ) + { + if (!(dataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) { + throw new ISE("Did not expect side channels for dataSource [%s]", dataSource); + } + + final List inputChannels = new ArrayList<>(); + final BroadcastJoinHelper broadcastJoinHelper; + + if (baseInput.hasChannel()) { + inputChannels.add(baseInput.getChannel()); + } + + if (dataSource instanceof JoinDataSource) { + final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap(); + final List channelReaders = new ArrayList<>(); + + if (baseInput.hasChannel()) { + // BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader. + channelReaders.add(null); + } + + for (Int2ObjectMap.Entry sideChannelEntry : sideChannels.int2ObjectEntrySet()) { + final int inputNumber = sideChannelEntry.getIntKey(); + inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size()); + inputChannels.add(sideChannelEntry.getValue().getChannel()); + channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader()); + } + + broadcastJoinHelper = new BroadcastJoinHelper( + inputNumberToProcessorChannelMap, + inputChannels, + channelReaders, + joinableFactory, + memoryReservedForBroadcastJoin + ); + } else { + broadcastJoinHelper = null; + } + + return Pair.of(inputChannels, broadcastJoinHelper); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java new file mode 100644 index 00000000000..c844a11a15a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java @@ -0,0 +1,307 @@ +/* + * 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.druid.msq.querykit; + +import com.google.common.collect.Iterators; +import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.ReadableConcatFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.InputSlices; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.input.external.ExternalInputSlice; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +public abstract class BaseLeafFrameProcessorFactory extends BaseFrameProcessorFactory +{ + private static final Logger log = new Logger(BaseLeafFrameProcessorFactory.class); + + @Override + public ProcessorsAndChannels, Long> makeProcessors( + StageDefinition stageDefinition, + int workerNumber, + List inputSlices, + InputSliceReader inputSliceReader, + @Nullable Object extra, + OutputChannelFactory outputChannelFactory, + FrameContext frameContext, + int maxOutstandingProcessors, + CounterTracker counters, + Consumer warningPublisher + ) throws IOException + { + // BaseLeafFrameProcessorFactory is used for native Druid queries, where the following input cases can happen: + // 1) Union datasources: N nonbroadcast inputs, which are are treated as one big input + // 2) Join datasources: one nonbroadcast input, N broadcast inputs + // 3) All other datasources: single input + + final int totalProcessors = InputSlices.getNumNonBroadcastReadableInputs( + inputSlices, + inputSliceReader, + stageDefinition.getBroadcastInputNumbers() + ); + + if (totalProcessors == 0) { + return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); + } + + final int outstandingProcessors; + + if (hasParquet(inputSlices)) { + // This is a workaround for memory use in ParquetFileReader, which loads up an entire row group into memory as + // part of its normal operation. Row groups can be quite large (like, 1GB large) so this is a major source of + // unaccounted-for memory use during ingestion and query of external data. We are trying to prevent memory + // overload by running only a single processor at once. + outstandingProcessors = 1; + } else { + outstandingProcessors = Math.min(totalProcessors, maxOutstandingProcessors); + } + + final AtomicReference> allocatorQueueRef = + new AtomicReference<>(new ArrayDeque<>(outstandingProcessors)); + final AtomicReference> channelQueueRef = + new AtomicReference<>(new ArrayDeque<>(outstandingProcessors)); + final List outputChannels = new ArrayList<>(outstandingProcessors); + + for (int i = 0; i < outstandingProcessors; i++) { + final OutputChannel outputChannel = outputChannelFactory.openChannel(0 /* Partition number doesn't matter */); + outputChannels.add(outputChannel); + channelQueueRef.get().add(outputChannel.getWritableChannel()); + allocatorQueueRef.get().add(outputChannel.getFrameMemoryAllocator()); + } + + // Read all base inputs in separate processors, one per processor. + final Sequence processorBaseInputs = readBaseInputs( + stageDefinition, + inputSlices, + inputSliceReader, + counters, + warningPublisher + ); + + final Sequence> processors = processorBaseInputs.map( + processorBaseInput -> { + // For each processor, we are rebuilding the broadcast table again which is wasteful. This can be pushed + // up to the factory level + final Int2ObjectMap sideChannels = + readBroadcastInputs(stageDefinition, inputSlices, inputSliceReader, counters, warningPublisher); + + return makeProcessor( + processorBaseInput, + sideChannels, + makeLazyResourceHolder( + channelQueueRef, + channel -> { + try { + channel.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ), + makeLazyResourceHolder(allocatorQueueRef, ignored -> { + }), + stageDefinition.getSignature(), + stageDefinition.getClusterBy(), + frameContext + ); + } + ).withBaggage( + () -> { + final Queue channelQueue; + synchronized (channelQueueRef) { + // Set to null so any channels returned by outstanding workers are immediately closed. + channelQueue = channelQueueRef.getAndSet(null); + } + + WritableFrameChannel c; + while ((c = channelQueue.poll()) != null) { + try { + c.close(); + } + catch (Throwable e) { + log.warn(e, "Error encountered while closing channel for [%s]", this); + } + } + } + ); + + return new ProcessorsAndChannels<>(processors, OutputChannels.wrapReadOnly(outputChannels)); + } + + /** + * Read base inputs, where "base" is meant in the same sense as in + * {@link org.apache.druid.query.planning.DataSourceAnalysis}: the primary datasource that drives query processing. + */ + private static Sequence readBaseInputs( + final StageDefinition stageDef, + final List inputSlices, + final InputSliceReader inputSliceReader, + final CounterTracker counters, + final Consumer warningPublisher + ) + { + final List> sequences = new ArrayList<>(); + + for (int inputNumber = 0; inputNumber < inputSlices.size(); inputNumber++) { + if (!stageDef.getBroadcastInputNumbers().contains(inputNumber)) { + final int i = inputNumber; + final Sequence sequence = + Sequences.simple(inputSliceReader.attach(i, inputSlices.get(i), counters, warningPublisher)); + sequences.add(sequence); + } + } + + return Sequences.concat(sequences); + } + + /** + * Reads all broadcast inputs, which must be {@link StageInputSlice}. The execution framework supports broadcasting + * other types of inputs, but QueryKit does not use them at this time. + * + * Returns a map of input number -> channel containing all data for that input number. + */ + private static Int2ObjectMap readBroadcastInputs( + final StageDefinition stageDef, + final List inputSlices, + final InputSliceReader inputSliceReader, + final CounterTracker counterTracker, + final Consumer warningPublisher + ) + { + final Int2ObjectMap broadcastInputs = new Int2ObjectAVLTreeMap<>(); + + try { + for (int inputNumber = 0; inputNumber < inputSlices.size(); inputNumber++) { + if (stageDef.getBroadcastInputNumbers().contains(inputNumber)) { + // QueryKit only uses StageInputSlice at this time. + final StageInputSlice slice = (StageInputSlice) inputSlices.get(inputNumber); + final ReadableInputs readableInputs = + inputSliceReader.attach(inputNumber, slice, counterTracker, warningPublisher); + + if (!readableInputs.isChannelBased()) { + // QueryKit limitation: broadcast inputs must be channels. + throw new ISE("Broadcast inputs must be channels"); + } + + final ReadableFrameChannel channel = ReadableConcatFrameChannel.open( + Iterators.transform(readableInputs.iterator(), ReadableInput::getChannel) + ); + broadcastInputs.put(inputNumber, ReadableInput.channel(channel, readableInputs.frameReader(), null)); + } + } + + return broadcastInputs; + } + catch (Throwable e) { + // Close any already-opened channels. + try { + broadcastInputs.values().forEach(input -> input.getChannel().close()); + } + catch (Throwable e2) { + e.addSuppressed(e2); + } + + throw e; + } + } + + protected abstract FrameProcessor makeProcessor( + ReadableInput baseInput, + Int2ObjectMap sideChannels, + ResourceHolder outputChannelSupplier, + ResourceHolder allocatorSupplier, + RowSignature signature, + ClusterBy clusterBy, + FrameContext providerThingy + ); + + private static ResourceHolder makeLazyResourceHolder( + final AtomicReference> queueRef, + final Consumer backupCloser + ) + { + return new LazyResourceHolder<>( + () -> { + final T resource; + + synchronized (queueRef) { + resource = queueRef.get().poll(); + } + + return Pair.of( + resource, + () -> { + synchronized (queueRef) { + final Queue queue = queueRef.get(); + if (queue != null) { + queue.add(resource); + return; + } + } + + // Queue was null + backupCloser.accept(resource); + } + ); + } + ); + } + + private static boolean hasParquet(final List slices) + { + return slices.stream().anyMatch( + slice -> + slice instanceof ExternalInputSlice + && ((ExternalInputSlice) slice).getInputFormat().getClass().getName().contains("Parquet") + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java new file mode 100644 index 00000000000..07e73012457 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java @@ -0,0 +1,205 @@ +/* + * 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.druid.msq.querykit; + +import com.google.common.annotations.VisibleForTesting; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.IntIterator; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.join.JoinableFactoryWrapper; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class BroadcastJoinHelper +{ + private final Int2IntMap inputNumberToProcessorChannelMap; + private final List channels; + private final List channelReaders; + private final JoinableFactoryWrapper joinableFactory; + private final List> channelData; + private final IntSet sideChannelNumbers; + private final long memoryReservedForBroadcastJoin; + + private long memoryUsed = 0L; + + /** + * Create a new broadcast join helper. Currently this builds the tables in channelData. Using + * {@link org.apache.druid.query.groupby.epinephelinae.collection.MemoryOpenHashTable} should be more appropriate for + * this purpose + * + * @param inputNumberToProcessorChannelMap map of input slice number -> channel position in the "channels" list + * @param channels list of input channels + * @param channelReaders list of input channel readers; corresponds one-to-one with "channels" + * @param joinableFactory joinable factory for this server + * @param memoryReservedForBroadcastJoin total bytes of frames we are permitted to use; derived from + * {@link org.apache.druid.msq.exec.WorkerMemoryParameters#broadcastJoinMemory} + */ + public BroadcastJoinHelper( + final Int2IntMap inputNumberToProcessorChannelMap, + final List channels, + final List channelReaders, + final JoinableFactoryWrapper joinableFactory, + final long memoryReservedForBroadcastJoin + ) + { + this.inputNumberToProcessorChannelMap = inputNumberToProcessorChannelMap; + this.channels = channels; + this.channelReaders = channelReaders; + this.joinableFactory = joinableFactory; + this.channelData = new ArrayList<>(); + this.sideChannelNumbers = new IntOpenHashSet(); + this.sideChannelNumbers.addAll(inputNumberToProcessorChannelMap.values()); + this.memoryReservedForBroadcastJoin = memoryReservedForBroadcastJoin; + + for (int i = 0; i < channels.size(); i++) { + if (sideChannelNumbers.contains(i)) { + channelData.add(new ArrayList<>()); + sideChannelNumbers.add(i); + } else { + channelData.add(null); + } + } + } + + /** + * Reads up to one frame from each readable side channel, and uses them to incrementally build up joinable + * broadcast tables. + * + * @param readableInputs all readable input channel numbers, including non-side-channels + * + * @return whether side channels have been fully read + */ + public boolean buildBroadcastTablesIncrementally(final IntSet readableInputs) + { + final IntIterator inputChannelIterator = readableInputs.iterator(); + + while (inputChannelIterator.hasNext()) { + final int channelNumber = inputChannelIterator.nextInt(); + if (sideChannelNumbers.contains(channelNumber) && channels.get(channelNumber).canRead()) { + final Frame frame = channels.get(channelNumber).read(); + + memoryUsed += frame.numBytes(); + + if (memoryUsed > memoryReservedForBroadcastJoin) { + throw new MSQException(new BroadcastTablesTooLargeFault(memoryReservedForBroadcastJoin)); + } + + addFrame(channelNumber, frame); + } + } + + for (int channelNumber : sideChannelNumbers) { + if (!channels.get(channelNumber).isFinished()) { + return false; + } + } + + return true; + } + + public IntSet getSideChannelNumbers() + { + return sideChannelNumbers; + } + + public Function makeSegmentMapFn(final Query query) + { + final DataSource dataSourceWithInlinedChannelData = inlineChannelData(query.getDataSource()); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData); + + return joinableFactory.createSegmentMapFn( + analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), + analysis.getPreJoinableClauses(), + new AtomicLong(), + analysis.getBaseQuery().orElse(query) + ); + } + + @VisibleForTesting + DataSource inlineChannelData(final DataSource originalDataSource) + { + if (originalDataSource instanceof InputNumberDataSource) { + final int inputNumber = ((InputNumberDataSource) originalDataSource).getInputNumber(); + if (inputNumberToProcessorChannelMap.containsKey(inputNumber)) { + final int channelNumber = inputNumberToProcessorChannelMap.get(inputNumber); + + if (sideChannelNumbers.contains(channelNumber)) { + return InlineDataSource.fromIterable( + channelData.get(channelNumber), + channelReaders.get(channelNumber).signature() + ); + } else { + return originalDataSource; + } + } else { + return originalDataSource; + } + } else { + final List newChildren = new ArrayList<>(originalDataSource.getChildren().size()); + + for (final DataSource child : originalDataSource.getChildren()) { + newChildren.add(inlineChannelData(child)); + } + + return originalDataSource.withChildren(newChildren); + } + } + + private void addFrame(final int channelNumber, final Frame frame) + { + final List data = channelData.get(channelNumber); + final FrameReader frameReader = channelReaders.get(channelNumber); + final Cursor cursor = FrameProcessors.makeCursor(frame, frameReader); + + final List selectors = + frameReader.signature().getColumnNames().stream().map( + columnName -> + cursor.getColumnSelectorFactory().makeColumnValueSelector(columnName) + ).collect(Collectors.toList()); + + while (!cursor.isDone()) { + final Object[] row = new Object[selectors.size()]; + for (int i = 0; i < row.length; i++) { + row[i] = selectors.get(i).getObject(); + } + data.add(row); + cursor.advance(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java new file mode 100644 index 00000000000..b5d95a4401b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java @@ -0,0 +1,37 @@ +/* + * 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.druid.msq.querykit; + +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.segment.Segment; +import org.apache.druid.timeline.SegmentId; + +public interface DataSegmentProvider +{ + /** + * Fetches the segment corresponding to the provided segmentId from deep storage, + * segment fetched. + */ + ResourceHolder fetchSegment( + SegmentId segmentId, + ChannelCounters channelCounters + ); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentTimelineView.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentTimelineView.java new file mode 100644 index 00000000000..cc010a104c6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentTimelineView.java @@ -0,0 +1,49 @@ +/* + * 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.druid.msq.querykit; + +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Optional; + +public interface DataSegmentTimelineView +{ + /** + * Returns the timeline for a datasource, if it 'exists'. The analysis object passed in must represent a scan-based + * datasource of a single table. (i.e., {@link DataSourceAnalysis#getBaseTableDataSource()} must be present.) + * + * @param dataSource table data source name + * @param intervals relevant intervals. The returned timeline will *at least* include all segments that overlap + * these intervals. It may also include more. Empty means the timeline may not contain any + * segments at all. + * + * @return timeline, if it 'exists' + * + * @throws IllegalStateException if 'analysis' does not represent a scan-based datasource of a single table + */ + Optional> getTimeline( + String dataSource, + List intervals + ); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java new file mode 100644 index 00000000000..2bd59a7f9cd --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java @@ -0,0 +1,384 @@ +/* + * 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.druid.msq.querykit; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import it.unimi.dsi.fastutil.ints.IntSets; +import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.NilInputSource; +import org.apache.druid.msq.input.external.ExternalInputSpec; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.input.table.TableInputSpec; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.QueryDefinitionBuilder; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.planning.PreJoinableClause; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.external.ExternalDataSource; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Used by {@link QueryKit} implementations to produce {@link InputSpec} from native {@link DataSource}. + */ +public class DataSourcePlan +{ + private final DataSource newDataSource; + private final List inputSpecs; + private final IntSet broadcastInputs; + + @Nullable + private final QueryDefinitionBuilder subQueryDefBuilder; + + DataSourcePlan( + final DataSource newDataSource, + final List inputSpecs, + final IntSet broadcastInputs, + @Nullable final QueryDefinitionBuilder subQueryDefBuilder + ) + { + this.newDataSource = Preconditions.checkNotNull(newDataSource, "newDataSource"); + this.inputSpecs = Preconditions.checkNotNull(inputSpecs, "inputSpecs"); + this.broadcastInputs = Preconditions.checkNotNull(broadcastInputs, "broadcastInputs"); + this.subQueryDefBuilder = subQueryDefBuilder; + + for (int broadcastInput : broadcastInputs) { + if (broadcastInput < 0 || broadcastInput >= inputSpecs.size()) { + throw new IAE("Broadcast input number [%d] out of range [0, %d)", broadcastInput, inputSpecs.size()); + } + } + } + + @SuppressWarnings("rawtypes") + public static DataSourcePlan forDataSource( + final QueryKit queryKit, + final String queryId, + final DataSource dataSource, + final QuerySegmentSpec querySegmentSpec, + @Nullable DimFilter filter, + final int maxWorkerCount, + final int minStageNumber, + final boolean broadcast + ) + { + if (dataSource instanceof TableDataSource) { + return forTable((TableDataSource) dataSource, querySegmentSpecIntervals(querySegmentSpec), filter, broadcast); + } else if (dataSource instanceof ExternalDataSource) { + checkQuerySegmentSpecIsEternity(dataSource, querySegmentSpec); + return forExternal((ExternalDataSource) dataSource, broadcast); + } else if (dataSource instanceof InlineDataSource) { + checkQuerySegmentSpecIsEternity(dataSource, querySegmentSpec); + return forInline((InlineDataSource) dataSource, broadcast); + } else if (dataSource instanceof QueryDataSource) { + checkQuerySegmentSpecIsEternity(dataSource, querySegmentSpec); + return forQuery( + queryKit, + queryId, + (QueryDataSource) dataSource, + maxWorkerCount, + minStageNumber, + broadcast + ); + } else if (dataSource instanceof JoinDataSource) { + return forJoin( + queryKit, + queryId, + (JoinDataSource) dataSource, + querySegmentSpec, + maxWorkerCount, + minStageNumber, + broadcast + ); + } else { + throw new UOE("Cannot handle dataSource [%s]", dataSource); + } + } + + public DataSource getNewDataSource() + { + return newDataSource; + } + + public List getInputSpecs() + { + return inputSpecs; + } + + public IntSet getBroadcastInputs() + { + return broadcastInputs; + } + + public Optional getSubQueryDefBuilder() + { + return Optional.ofNullable(subQueryDefBuilder); + } + + /** + * Whether this datasource must be processed by a single worker. True if, and only if, all inputs are broadcast. + */ + public boolean isSingleWorker() + { + return broadcastInputs.size() == inputSpecs.size(); + } + + private static DataSourcePlan forTable( + final TableDataSource dataSource, + final List intervals, + @Nullable final DimFilter filter, + final boolean broadcast + ) + { + return new DataSourcePlan( + new InputNumberDataSource(0), + Collections.singletonList(new TableInputSpec(dataSource.getName(), intervals, filter)), + broadcast ? IntOpenHashSet.of(0) : IntSets.emptySet(), + null + ); + } + + private static DataSourcePlan forExternal( + final ExternalDataSource dataSource, + final boolean broadcast + ) + { + return new DataSourcePlan( + new InputNumberDataSource(0), + Collections.singletonList( + new ExternalInputSpec( + dataSource.getInputSource(), + dataSource.getInputFormat(), + dataSource.getSignature() + ) + ), + broadcast ? IntOpenHashSet.of(0) : IntSets.emptySet(), + null + ); + } + + private static DataSourcePlan forInline( + final InlineDataSource dataSource, + final boolean broadcast + ) + { + final ObjectMapper jsonMapper = new ObjectMapper(new JsonFactory()); + final RowSignature signature = dataSource.getRowSignature(); + final StringBuilder stringBuilder = new StringBuilder(); + + for (final Object[] rowArray : dataSource.getRows()) { + final Map m = new HashMap<>(); + + for (int i = 0; i < signature.size(); i++) { + m.put(signature.getColumnName(i), rowArray[i]); + } + + try { + stringBuilder.append(jsonMapper.writeValueAsString(m)).append('\n'); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + final String dataString = stringBuilder.toString(); + + return forExternal( + new ExternalDataSource( + dataString.isEmpty() ? NilInputSource.instance() : new InlineInputSource(dataString), + new JsonInputFormat(null, null, null), + signature + ), + broadcast + ); + } + + private static DataSourcePlan forQuery( + final QueryKit queryKit, + final String queryId, + final QueryDataSource dataSource, + final int maxWorkerCount, + final int minStageNumber, + final boolean broadcast + ) + { + final QueryDefinition subQueryDef = queryKit.makeQueryDefinition( + queryId, + dataSource.getQuery(), + queryKit, + ShuffleSpecFactories.subQueryWithMaxWorkerCount(maxWorkerCount), + maxWorkerCount, + minStageNumber + ); + + final int stageNumber = subQueryDef.getFinalStageDefinition().getStageNumber(); + + return new DataSourcePlan( + new InputNumberDataSource(0), + Collections.singletonList(new StageInputSpec(stageNumber)), + broadcast ? IntOpenHashSet.of(0) : IntSets.emptySet(), + QueryDefinition.builder(subQueryDef) + ); + } + + private static DataSourcePlan forJoin( + final QueryKit queryKit, + final String queryId, + final JoinDataSource dataSource, + final QuerySegmentSpec querySegmentSpec, + final int maxWorkerCount, + final int minStageNumber, + final boolean broadcast + ) + { + final QueryDefinitionBuilder subQueryDefBuilder = QueryDefinition.builder(); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSource); + + final DataSourcePlan basePlan = forDataSource( + queryKit, + queryId, + analysis.getBaseDataSource(), + querySegmentSpec, + null, // Don't push query filters down through a join: this needs some work to ensure pruning works properly. + maxWorkerCount, + Math.max(minStageNumber, subQueryDefBuilder.getNextStageNumber()), + broadcast + ); + + DataSource newDataSource = basePlan.getNewDataSource(); + final List inputSpecs = new ArrayList<>(basePlan.getInputSpecs()); + final IntSet broadcastInputs = new IntOpenHashSet(basePlan.getBroadcastInputs()); + basePlan.getSubQueryDefBuilder().ifPresent(subQueryDefBuilder::addAll); + + for (int i = 0; i < analysis.getPreJoinableClauses().size(); i++) { + final PreJoinableClause clause = analysis.getPreJoinableClauses().get(i); + final DataSourcePlan clausePlan = forDataSource( + queryKit, + queryId, + clause.getDataSource(), + new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY), + null, // Don't push query filters down through a join: this needs some work to ensure pruning works properly. + maxWorkerCount, + Math.max(minStageNumber, subQueryDefBuilder.getNextStageNumber()), + true // Always broadcast right-hand side of the join. + ); + + // Shift all input numbers in the clausePlan. + final int shift = inputSpecs.size(); + + newDataSource = JoinDataSource.create( + newDataSource, + shiftInputNumbers(clausePlan.getNewDataSource(), shift), + clause.getPrefix(), + clause.getCondition(), + clause.getJoinType(), + + // First JoinDataSource (i == 0) involves the base table, so we need to propagate the base table filter. + i == 0 ? analysis.getJoinBaseTableFilter().orElse(null) : null + ); + inputSpecs.addAll(clausePlan.getInputSpecs()); + clausePlan.getBroadcastInputs().intStream().forEach(n -> broadcastInputs.add(n + shift)); + clausePlan.getSubQueryDefBuilder().ifPresent(subQueryDefBuilder::addAll); + } + + return new DataSourcePlan(newDataSource, inputSpecs, broadcastInputs, subQueryDefBuilder); + } + + private static DataSource shiftInputNumbers(final DataSource dataSource, final int shift) + { + if (shift < 0) { + throw new IAE("Shift must be >= 0"); + } else if (shift == 0) { + return dataSource; + } else { + if (dataSource instanceof InputNumberDataSource) { + return new InputNumberDataSource(((InputNumberDataSource) dataSource).getInputNumber() + shift); + } else { + return dataSource.withChildren( + dataSource.getChildren() + .stream() + .map(child -> shiftInputNumbers(child, shift)) + .collect(Collectors.toList()) + ); + } + } + } + + private static List querySegmentSpecIntervals(final QuerySegmentSpec querySegmentSpec) + { + if (querySegmentSpec instanceof MultipleIntervalSegmentSpec) { + return querySegmentSpec.getIntervals(); + } else { + throw new UOE("Cannot handle querySegmentSpec type [%s]", querySegmentSpec.getClass().getName()); + } + } + + /** + * Verify that the provided {@link QuerySegmentSpec} is a {@link MultipleIntervalSegmentSpec} with + * interval {@link Intervals#ETERNITY}. If not, throw an {@link UnsupportedOperationException}. + * + * Anywhere this appears is a place that we do not support using the "intervals" parameter of a query + * (i.e., {@link org.apache.druid.query.BaseQuery#getQuerySegmentSpec()}) for time filtering. Ideally, + * we'd support this everywhere it appears, but we can get away without it for now. + */ + private static void checkQuerySegmentSpecIsEternity( + final DataSource dataSource, + final QuerySegmentSpec querySegmentSpec + ) + { + final boolean querySegmentSpecIsEternity = + querySegmentSpec instanceof MultipleIntervalSegmentSpec + && querySegmentSpec.getIntervals().equals(Intervals.ONLY_ETERNITY); + + if (!querySegmentSpecIsEternity) { + throw new UOE( + "Cannot filter datasource [%s] using [%s]", + dataSource.getClass().getName(), + ColumnHolder.TIME_COLUMN_NAME + ); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java new file mode 100644 index 00000000000..df3d9fc3b06 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -0,0 +1,116 @@ +/* + * 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.druid.msq.querykit; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.DataSource; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +@JsonTypeName("inputNumber") +public class InputNumberDataSource implements DataSource +{ + private final int inputNumber; + + @JsonCreator + public InputNumberDataSource(@JsonProperty("inputNumber") int inputNumber) + { + this.inputNumber = inputNumber; + } + + @Override + public Set getTableNames() + { + return Collections.emptySet(); + } + + @Override + public List getChildren() + { + return Collections.emptyList(); + } + + @Override + public DataSource withChildren(final List children) + { + if (!children.isEmpty()) { + throw new IAE("Cannot accept children"); + } + + return this; + } + + @Override + public boolean isCacheable(boolean isBroker) + { + return false; + } + + @Override + public boolean isGlobal() + { + return false; + } + + @Override + public boolean isConcrete() + { + return false; + } + + @JsonProperty + public int getInputNumber() + { + return inputNumber; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + InputNumberDataSource that = (InputNumberDataSource) o; + return inputNumber == that.inputNumber; + } + + @Override + public int hashCode() + { + return Objects.hash(inputNumber); + } + + @Override + public String toString() + { + return "InputNumberDataSource{" + + "inputNumber=" + inputNumber + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/LazyResourceHolder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/LazyResourceHolder.java new file mode 100644 index 00000000000..4ae03c5dcf7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/LazyResourceHolder.java @@ -0,0 +1,73 @@ +/* + * 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.druid.msq.querykit; + +import com.google.common.base.Preconditions; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.logger.Logger; + +import javax.annotation.concurrent.NotThreadSafe; +import java.io.Closeable; +import java.util.function.Supplier; + +@NotThreadSafe +public class LazyResourceHolder implements ResourceHolder +{ + private static final Logger log = new Logger(LazyResourceHolder.class); + + private final Supplier> supplier; + private T resource = null; + private Closeable closer = null; + + public LazyResourceHolder(final Supplier> supplier) + { + this.supplier = Preconditions.checkNotNull(supplier, "supplier"); + } + + @Override + public T get() + { + if (resource == null) { + final Pair supplied = supplier.get(); + resource = Preconditions.checkNotNull(supplied.lhs, "resource"); + closer = Preconditions.checkNotNull(supplied.rhs, "closer"); + } + + return resource; + } + + @Override + public void close() + { + if (resource != null) { + try { + closer.close(); + } + catch (Throwable e) { + log.noStackTrace().warn(e, "Exception encountered while closing resource: %s", resource); + } + finally { + resource = null; + closer = null; + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java new file mode 100644 index 00000000000..a795f649605 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java @@ -0,0 +1,68 @@ +/* + * 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.druid.msq.querykit; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.query.Query; + +import java.util.Map; + +/** + * Delegates to other {@link QueryKit} implementations based on the class of the {@link Query}. + */ +@SuppressWarnings("rawtypes") +public class MultiQueryKit implements QueryKit> +{ + private final Map, QueryKit> toolKitMap; + + public MultiQueryKit(final Map, QueryKit> toolKitMap) + { + this.toolKitMap = Preconditions.checkNotNull(toolKitMap, "toolKitMap"); + } + + @Override + public QueryDefinition makeQueryDefinition( + String queryId, + Query query, + QueryKit> toolKitForSubQueries, + ShuffleSpecFactory resultShuffleSpecFactory, + int maxWorkerCount, + int minStageNumber + ) + { + final QueryKit specificToolKit = toolKitMap.get(query.getClass()); + + if (specificToolKit != null) { + //noinspection unchecked + return specificToolKit.makeQueryDefinition( + queryId, + query, + this, + resultShuffleSpecFactory, + maxWorkerCount, + minStageNumber + ); + } else { + throw new ISE("Unsupported query class [%s]", query.getClass().getName()); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java new file mode 100644 index 00000000000..b259022bba5 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java @@ -0,0 +1,52 @@ +/* + * 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.druid.msq.querykit; + +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.query.Query; + +/** + * Kit for creating multi-stage query {@link QueryDefinition} out of Druid native {@link Query} objects. + */ +public interface QueryKit> +{ + /** + * Creates a {@link QueryDefinition} from a {@link Query}. + * + * @param queryId query ID of the resulting {@link QueryDefinition} + * @param query native query to translate + * @param toolKitForSubQueries kit that is used to translate native subqueries; i.e., + * {@link org.apache.druid.query.QueryDataSource}. Typically a {@link MultiQueryKit}. + * @param resultShuffleSpecFactory shuffle spec factory for the final output of this query. + * @param maxWorkerCount maximum number of workers: becomes + * {@link org.apache.druid.msq.kernel.StageDefinition#getMaxWorkerCount()} + * @param minStageNumber lowest stage number to use for any generated stages. Useful if the resulting + * {@link QueryDefinition} is going to be added to an existing + * {@link org.apache.druid.msq.kernel.QueryDefinitionBuilder}. + */ + QueryDefinition makeQueryDefinition( + String queryId, + QueryType query, + QueryKit> toolKitForSubQueries, + ShuffleSpecFactory resultShuffleSpecFactory, + int maxWorkerCount, + int minStageNumber + ); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java new file mode 100644 index 00000000000..2e5cd0c34c9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java @@ -0,0 +1,222 @@ +/* + * 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.druid.msq.querykit; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.calcite.sql.dialect.CalciteSqlDialect; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.indexing.error.ColumnNameRestrictedFault; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.query.Query; +import org.apache.druid.query.expression.TimestampFloorExprMacro; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; +import org.apache.druid.sql.calcite.planner.Calcites; +import org.joda.time.DateTimeZone; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Utility methods for QueryKit. + */ +public class QueryKitUtils +{ + /** + * Field in frames that stores the partition "boosting" value. Typically used as the last element of a partitioning + * key when generating segments. This is an incrementing number that helps split up otherwise too-large partitions. + */ + public static final String PARTITION_BOOST_COLUMN = "__boost"; + + /** + * Field in frames that stores the segment-granular timestamp. + */ + public static final String SEGMENT_GRANULARITY_COLUMN = "__bucket"; + + /** + * Enables QueryKit-generated processors to understand which output column will be mapped to + * {@link org.apache.druid.segment.column.ColumnHolder#TIME_COLUMN_NAME}. Necessary because {@link QueryKit} + * does not get direct access to {@link org.apache.druid.msq.indexing.ColumnMappings}. + */ + public static final String CTX_TIME_COLUMN_NAME = "__timeColumn"; + + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + + public static Granularity getSegmentGranularityFromContext(@Nullable final Map context) + { + final Object o = context == null ? null : context.get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY); + + if (o instanceof String) { + try { + return OBJECT_MAPPER.readValue((String) o, Granularity.class); + } + catch (JsonProcessingException e) { + throw new ISE("Invalid segment granularity [%s]", o); + } + } else if (o == null) { + return Granularities.ALL; + } else { + throw new ISE("Invalid segment granularity [%s]", o); + } + } + + /** + * Adds bucketing by {@link #SEGMENT_GRANULARITY_COLUMN} to a {@link ClusterBy} if needed. + */ + public static ClusterBy clusterByWithSegmentGranularity( + final ClusterBy clusterBy, + final Granularity segmentGranularity + ) + { + if (Granularities.ALL.equals(segmentGranularity)) { + return clusterBy; + } else { + final List newColumns = new ArrayList<>(clusterBy.getColumns().size() + 1); + newColumns.add(new SortColumn(QueryKitUtils.SEGMENT_GRANULARITY_COLUMN, false)); + newColumns.addAll(clusterBy.getColumns()); + return new ClusterBy(newColumns, 1); + } + } + + /** + * Verifies the {@link RowSignature} and throws an appropriate exception if it is invalid or uses restricted column + * names + */ + public static void verifyRowSignature(final RowSignature signature) + { + if (signature.contains(QueryKitUtils.PARTITION_BOOST_COLUMN)) { + throw new MSQException(new ColumnNameRestrictedFault(QueryKitUtils.PARTITION_BOOST_COLUMN)); + } else if (signature.contains(QueryKitUtils.SEGMENT_GRANULARITY_COLUMN)) { + throw new MSQException(new ColumnNameRestrictedFault(QueryKitUtils.SEGMENT_GRANULARITY_COLUMN)); + } + } + + /** + * Adds {@link #SEGMENT_GRANULARITY_COLUMN} to a {@link RowSignature} if needed. Signature should be verified prior + * to calling this function to ensure that {@link #SEGMENT_GRANULARITY_COLUMN} is not passed in by the user + */ + public static RowSignature signatureWithSegmentGranularity( + final RowSignature signature, + final Granularity segmentGranularity + ) + { + if (Granularities.ALL.equals(segmentGranularity)) { + return signature; + } else { + return RowSignature.builder() + .addAll(signature) + .add(QueryKitUtils.SEGMENT_GRANULARITY_COLUMN, ColumnType.LONG) + .build(); + } + } + + /** + * Returns a copy of "signature" with columns rearranged so the provided clusterByColumns appear as a prefix. + * Throws an error if any of the clusterByColumns are not present in the input signature, or if any of their + * types are unknown. + */ + public static RowSignature sortableSignature( + final RowSignature signature, + final List clusterByColumns + ) + { + final RowSignature.Builder builder = RowSignature.builder(); + + for (final SortColumn columnName : clusterByColumns) { + final Optional columnType = signature.getColumnType(columnName.columnName()); + if (!columnType.isPresent()) { + throw new IAE("Column [%s] not present in signature", columnName); + } + + builder.add(columnName.columnName(), columnType.get()); + } + + final Set clusterByColumnNames = + clusterByColumns.stream().map(SortColumn::columnName).collect(Collectors.toSet()); + + for (int i = 0; i < signature.size(); i++) { + final String columnName = signature.getColumnName(i); + if (!clusterByColumnNames.contains(columnName)) { + builder.add(columnName, signature.getColumnType(i).orElse(null)); + } + } + + return builder.build(); + } + + /** + * Returns a virtual column named {@link QueryKitUtils#SEGMENT_GRANULARITY_COLUMN} that computes a segment + * granularity based on a particular time column. Returns null if no virtual column is needed because the + * granularity is {@link Granularities#ALL}. Throws an exception if the provided granularity is not supported. + * + * @throws IllegalArgumentException if the provided granularity is not supported + */ + @Nullable + public static VirtualColumn makeSegmentGranularityVirtualColumn(final Query query) + { + final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(query.getContext()); + final String timeColumnName = query.getContextValue(QueryKitUtils.CTX_TIME_COLUMN_NAME); + + if (timeColumnName == null || Granularities.ALL.equals(segmentGranularity)) { + return null; + } + + if (!(segmentGranularity instanceof PeriodGranularity)) { + throw new IAE("Granularity [%s] is not supported", segmentGranularity); + } + + final PeriodGranularity periodSegmentGranularity = (PeriodGranularity) segmentGranularity; + + if (periodSegmentGranularity.getOrigin() != null + || !periodSegmentGranularity.getTimeZone().equals(DateTimeZone.UTC)) { + throw new IAE("Granularity [%s] is not supported", segmentGranularity); + } + + return new ExpressionVirtualColumn( + QueryKitUtils.SEGMENT_GRANULARITY_COLUMN, + StringUtils.format( + "timestamp_floor(%s, %s)", + CalciteSqlDialect.DEFAULT.quoteIdentifier(timeColumnName), + Calcites.escapeStringLiteral((periodSegmentGranularity).getPeriod().toString()) + ), + ColumnType.LONG, + new ExprMacroTable(Collections.singletonList(new TimestampFloorExprMacro())) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactories.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactories.java new file mode 100644 index 00000000000..848567b2fcb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactories.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.druid.msq.querykit; + +import org.apache.druid.msq.kernel.MaxCountShuffleSpec; + +/** + * Static factory methods for common implementations of {@link ShuffleSpecFactory}. + */ +public class ShuffleSpecFactories +{ + private ShuffleSpecFactories() + { + // No instantiation. + } + + /** + * Factory that produces a single output partition. + */ + public static ShuffleSpecFactory singlePartition() + { + return (clusterBy, aggregate) -> + new MaxCountShuffleSpec(clusterBy, 1, aggregate); + } + + /** + * Factory that produces a particular number of output partitions. + */ + public static ShuffleSpecFactory subQueryWithMaxWorkerCount(final int maxWorkerCount) + { + return (clusterBy, aggregate) -> + new MaxCountShuffleSpec(clusterBy, maxWorkerCount, aggregate); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactory.java new file mode 100644 index 00000000000..6b062761452 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ShuffleSpecFactory.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.druid.msq.querykit; + +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.msq.kernel.ShuffleSpec; + +/** + * Used by {@link QueryKit#makeQueryDefinition} to create the {@link ShuffleSpec} for the various stages of a query. + */ +public interface ShuffleSpecFactory +{ + /** + * Build a {@link ShuffleSpec} for given {@link ClusterBy}. The {@code aggregate} flag is used to populate + * {@link ShuffleSpec#doesAggregateByClusterKey()}. + */ + ShuffleSpec build(ClusterBy clusterBy, boolean aggregate); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java new file mode 100644 index 00000000000..a885ee1faa1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java @@ -0,0 +1,168 @@ +/* + * 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.druid.msq.querykit.common; + +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.HeapMemoryAllocator; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.FrameRowTooLargeException; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.write.FrameWriter; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.Cursor; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +public class OffsetLimitFrameProcessor implements FrameProcessor +{ + private final ReadableFrameChannel inputChannel; + private final WritableFrameChannel outputChannel; + private final FrameReader frameReader; + private final long offset; + private final long limit; + + long rowsProcessedSoFar = 0L; + + OffsetLimitFrameProcessor( + ReadableFrameChannel inputChannel, + WritableFrameChannel outputChannel, + FrameReader frameReader, + long offset, + long limit + ) + { + this.inputChannel = inputChannel; + this.outputChannel = outputChannel; + this.frameReader = frameReader; + this.offset = offset; + this.limit = limit; + + if (offset < 0 || limit < 0) { + throw new ISE("Offset and limit must be nonnegative"); + } + } + + @Override + public List inputChannels() + { + return Collections.singletonList(inputChannel); + } + + @Override + public List outputChannels() + { + return Collections.singletonList(outputChannel); + } + + @Override + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + { + if (readableInputs.isEmpty()) { + return ReturnOrAwait.awaitAll(1); + } else if (inputChannel.isFinished() || rowsProcessedSoFar == offset + limit) { + return ReturnOrAwait.returnObject(rowsProcessedSoFar); + } + + final Frame frame = inputChannel.read(); + final Frame truncatedFrame = chopAndProcess(frame, frameReader); + + if (truncatedFrame != null) { + outputChannel.write(new FrameWithPartition(truncatedFrame, FrameWithPartition.NO_PARTITION)); + } + + if (rowsProcessedSoFar == offset + limit) { + // This check is not strictly necessary, given the check above, but prevents one extra scheduling round. + return ReturnOrAwait.returnObject(rowsProcessedSoFar); + } else { + assert rowsProcessedSoFar < offset + limit; + return ReturnOrAwait.awaitAll(1); + } + } + + @Override + public void cleanup() throws IOException + { + FrameProcessors.closeAll(inputChannels(), outputChannels()); + } + + /** + * Chops a frame down to a smaller one, potentially on both ends. + * + * Increments {@link #rowsProcessedSoFar} as it does its work. Either returns the original frame, a chopped frame, + * or null if no rows from the current frame should be included. + */ + @Nullable + private Frame chopAndProcess(final Frame frame, final FrameReader frameReader) + { + final long startRow = Math.max(0, offset - rowsProcessedSoFar); + final long endRow = Math.min(frame.numRows(), offset + limit - rowsProcessedSoFar); + + if (startRow >= endRow) { + // Offset is past the end of the frame; skip it. + rowsProcessedSoFar += frame.numRows(); + return null; + } else if (startRow == 0 && endRow == frame.numRows()) { + rowsProcessedSoFar += frame.numRows(); + return frame; + } + + final Cursor cursor = FrameProcessors.makeCursor(frame, frameReader); + + // Using an unlimited memory allocator to make sure that atleast a single frame can always be generated + final HeapMemoryAllocator unlimitedAllocator = HeapMemoryAllocator.unlimited(); + + long rowsProcessedSoFarInFrame = 0; + + final FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( + FrameType.ROW_BASED, + unlimitedAllocator, + frameReader.signature(), + Collections.emptyList() + ); + + try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) { + while (!cursor.isDone() && rowsProcessedSoFarInFrame < endRow) { + if (rowsProcessedSoFarInFrame >= startRow && !frameWriter.addSelection()) { + // Don't retry; it can't work because the allocator is unlimited anyway. + // Also, I don't think this line can be reached, because the allocator is unlimited. + throw new FrameRowTooLargeException(unlimitedAllocator.capacity()); + } + + cursor.advance(); + rowsProcessedSoFarInFrame++; + } + + rowsProcessedSoFar += rowsProcessedSoFarInFrame; + return Frame.wrap(frameWriter.toByteArray()); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.java new file mode 100644 index 00000000000..ecf03a08c34 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.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.druid.msq.querykit.common; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import org.apache.druid.frame.channel.ReadableConcatFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.BaseFrameProcessorFactory; +import org.apache.druid.msq.util.SupplierIterator; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.function.Consumer; +import java.util.function.Supplier; + +@JsonTypeName("limit") +public class OffsetLimitFrameProcessorFactory extends BaseFrameProcessorFactory +{ + private final long offset; + + @Nullable + private final Long limit; + + @JsonCreator + public OffsetLimitFrameProcessorFactory( + @JsonProperty("offset") final long offset, + @Nullable @JsonProperty("limit") final Long limit + ) + { + this.offset = offset; + this.limit = limit; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public long getOffset() + { + return offset; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getLimit() + { + return limit; + } + + @Override + public ProcessorsAndChannels, Long> makeProcessors( + StageDefinition stageDefinition, + int workerNumber, + List inputSlices, + InputSliceReader inputSliceReader, + @Nullable Object extra, + OutputChannelFactory outputChannelFactory, + FrameContext frameContext, + int maxOutstandingProcessors, + CounterTracker counters, + Consumer warningPublisher + ) throws IOException + { + if (workerNumber > 0) { + // We use a simplistic limiting approach: funnel all data through a single worker, single processor, and + // single output partition. So limiting stages must have a single worker. + throw new ISE("%s must be configured with maxWorkerCount = 1", getClass().getSimpleName()); + } + + // Expect a single input slice. + final InputSlice slice = Iterables.getOnlyElement(inputSlices); + + if (inputSliceReader.numReadableInputs(slice) == 0) { + return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); + } + + final OutputChannel outputChannel = outputChannelFactory.openChannel(0); + + final Supplier> workerSupplier = () -> { + final ReadableInputs readableInputs = inputSliceReader.attach(0, slice, counters, warningPublisher); + + if (!readableInputs.isChannelBased()) { + throw new ISE("Processor inputs must be channels"); + } + + // Note: OffsetLimitFrameProcessor does not use allocator from the outputChannel; it uses unlimited instead. + return new OffsetLimitFrameProcessor( + ReadableConcatFrameChannel.open(Iterators.transform(readableInputs.iterator(), ReadableInput::getChannel)), + outputChannel.getWritableChannel(), + readableInputs.frameReader(), + offset, + // Limit processor will add limit + offset at various points; must avoid overflow + limit == null ? Long.MAX_VALUE - offset : limit + ); + }; + + final Sequence> processors = + Sequences.simple(() -> new SupplierIterator<>(workerSupplier)); + + return new ProcessorsAndChannels<>( + processors, + OutputChannels.wrapReadOnly(Collections.singletonList(outputChannel)) + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OffsetLimitFrameProcessorFactory that = (OffsetLimitFrameProcessorFactory) o; + return offset == that.offset && Objects.equals(limit, that.limit); + } + + @Override + public int hashCode() + { + return Objects.hash(offset, limit); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java new file mode 100644 index 00000000000..a44c14e3cdc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java @@ -0,0 +1,324 @@ +/* + * 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.druid.msq.querykit.groupby; + +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.FrameRowTooLargeException; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.write.FrameWriter; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper; +import org.apache.druid.query.groupby.having.AlwaysHavingSpec; +import org.apache.druid.query.groupby.having.DimFilterHavingSpec; +import org.apache.druid.query.groupby.having.HavingSpec; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.function.BinaryOperator; +import java.util.function.Consumer; +import java.util.function.Supplier; + +public class GroupByPostShuffleFrameProcessor implements FrameProcessor +{ + private final GroupByQuery query; + private final ReadableFrameChannel inputChannel; + private final WritableFrameChannel outputChannel; + private final MemoryAllocator allocator; + private final FrameReader frameReader; + private final RowSignature resultSignature; + private final ClusterBy clusterBy; + private final ColumnSelectorFactory columnSelectorFactoryForFrameWriter; + private final Comparator compareFn; + private final BinaryOperator mergeFn; + private final Consumer finalizeFn; + + @Nullable + private final HavingSpec havingSpec; + + private Cursor frameCursor = null; + private Supplier rowSupplierFromFrameCursor; + private ResultRow outputRow = null; + private FrameWriter frameWriter = null; + + public GroupByPostShuffleFrameProcessor( + final GroupByQuery query, + final GroupByStrategySelector strategySelector, + final ReadableFrameChannel inputChannel, + final WritableFrameChannel outputChannel, + final FrameReader frameReader, + final RowSignature resultSignature, + final ClusterBy clusterBy, + final MemoryAllocator allocator + ) + { + this.query = query; + this.inputChannel = inputChannel; + this.outputChannel = outputChannel; + this.frameReader = frameReader; + this.resultSignature = resultSignature; + this.clusterBy = clusterBy; + this.allocator = allocator; + this.compareFn = strategySelector.strategize(query).createResultComparator(query); + this.mergeFn = strategySelector.strategize(query).createMergeFn(query); + this.finalizeFn = makeFinalizeFn(query); + this.havingSpec = cloneHavingSpec(query); + this.columnSelectorFactoryForFrameWriter = + makeVirtualColumnsForFrameWriter(query).wrap( + RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory( + query, + () -> outputRow, + RowSignature.Finalization.YES + ) + ); + } + + @Override + public List inputChannels() + { + return Collections.singletonList(inputChannel); + } + + @Override + public List outputChannels() + { + return Collections.singletonList(outputChannel); + } + + @Override + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + { + if (frameCursor == null || frameCursor.isDone()) { + // Keep reading through the input channel. + if (readableInputs.isEmpty()) { + return ReturnOrAwait.awaitAll(1); + } else if (inputChannel.isFinished()) { + if (outputRow != null && writeOutputRow()) { + return ReturnOrAwait.runAgain(); + } + + writeCurrentFrameIfNeeded(); + return ReturnOrAwait.returnObject(0L); + } else { + final Frame frame = inputChannel.read(); + frameCursor = FrameProcessors.makeCursor(frame, frameReader); + final ColumnSelectorFactory frameColumnSelectorFactory = frameCursor.getColumnSelectorFactory(); + + //noinspection unchecked + final Supplier[] fieldSuppliers = new Supplier[query.getResultRowSizeWithoutPostAggregators()]; + for (int i = 0; i < fieldSuppliers.length; i++) { + final ColumnValueSelector selector = + frameColumnSelectorFactory.makeColumnValueSelector(frameReader.signature().getColumnName(i)); + fieldSuppliers[i] = selector::getObject; + } + + final int fullRowSize = query.getResultRowSignature().size(); + rowSupplierFromFrameCursor = () -> { + final ResultRow row = ResultRow.create(fullRowSize); + for (int i = 0; i < fieldSuppliers.length; i++) { + row.set(i, fieldSuppliers[i].get()); + } + + for (int i = fieldSuppliers.length; i < fullRowSize; i++) { + // Post-aggregators. + row.set(i, null); + } + + return row; + }; + } + } + + setUpFrameWriterIfNeeded(); + + while (!frameCursor.isDone()) { + final ResultRow currentRow = rowSupplierFromFrameCursor.get(); + + if (outputRow == null) { + outputRow = currentRow.copy(); + } else if (compareFn.compare(outputRow, currentRow) == 0) { + outputRow = mergeFn.apply(outputRow, currentRow); + } else { + if (writeOutputRow()) { + return ReturnOrAwait.runAgain(); + } + + outputRow = currentRow.copy(); + } + + frameCursor.advance(); + } + + return ReturnOrAwait.runAgain(); + } + + @Override + public void cleanup() throws IOException + { + FrameProcessors.closeAll(inputChannels(), outputChannels(), frameWriter); + } + + /** + * Writes the current {@link #outputRow} to a pending frame, if it matches the query's havingSpec. Either way, + * the {@link #outputRow} is cleared. + * + * If needed, writes the previously pending frame to the output channel. + * + * @return whether the previously pending frame was flushed + */ + private boolean writeOutputRow() throws IOException + { + final int resultRowSize = query.getResultRowSignature().size(); + + if (outputRow.length() < resultRowSize) { + final Object[] newArray = new Object[resultRowSize]; + System.arraycopy(outputRow.getArray(), 0, newArray, 0, outputRow.length()); + outputRow = ResultRow.of(newArray); + } + + // Apply post-aggregators. + final Map outputRowAsMap = outputRow.toMap(query); + + for (int i = 0; i < query.getPostAggregatorSpecs().size(); i++) { + final PostAggregator postAggregator = query.getPostAggregatorSpecs().get(i); + final Object value = postAggregator.compute(outputRowAsMap); + outputRow.set(query.getResultRowPostAggregatorStart() + i, value); + outputRowAsMap.put(postAggregator.getName(), value); + } + + // Finalize aggregators. + finalizeFn.accept(outputRow); + + if (havingSpec != null && !havingSpec.eval(outputRow)) { + // Didn't match HAVING. + outputRow = null; + return false; + } else if (frameWriter.addSelection()) { + outputRow = null; + return false; + } else if (frameWriter.getNumRows() > 0) { + writeCurrentFrameIfNeeded(); + setUpFrameWriterIfNeeded(); + + if (frameWriter.addSelection()) { + outputRow = null; + return true; + } else { + throw new FrameRowTooLargeException(allocator.capacity()); + } + } else { + throw new FrameRowTooLargeException(allocator.capacity()); + } + } + + private void writeCurrentFrameIfNeeded() throws IOException + { + if (frameWriter != null && frameWriter.getNumRows() > 0) { + final Frame frame = Frame.wrap(frameWriter.toByteArray()); + outputChannel.write(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION)); + frameWriter.close(); + frameWriter = null; + } + } + + private void setUpFrameWriterIfNeeded() + { + if (frameWriter == null) { + final FrameWriterFactory frameWriterFactory = + FrameWriters.makeFrameWriterFactory(FrameType.ROW_BASED, allocator, resultSignature, clusterBy.getColumns()); + frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactoryForFrameWriter); + } + } + + private static Consumer makeFinalizeFn(final GroupByQuery query) + { + if (GroupByQueryKit.isFinalize(query)) { + final int startIndex = query.getResultRowAggregatorStart(); + final List aggregators = query.getAggregatorSpecs(); + + return row -> { + for (int i = 0; i < aggregators.size(); i++) { + row.set(startIndex + i, aggregators.get(i).finalizeComputation(row.get(startIndex + i))); + } + }; + } else { + return row -> {}; + } + } + + @Nullable + private static HavingSpec cloneHavingSpec(final GroupByQuery query) + { + if (query.getHavingSpec() == null || query.getHavingSpec() instanceof AlwaysHavingSpec) { + return null; + } else if (query.getHavingSpec() instanceof DimFilterHavingSpec) { + final DimFilterHavingSpec dimFilterHavingSpec = (DimFilterHavingSpec) query.getHavingSpec(); + final DimFilterHavingSpec clonedHavingSpec = new DimFilterHavingSpec( + dimFilterHavingSpec.getDimFilter(), + dimFilterHavingSpec.isFinalize() + ); + clonedHavingSpec.setQuery(query); + return clonedHavingSpec; + } else { + throw new UnsupportedOperationException("Must use 'filter' or 'always' havingSpec"); + } + } + + /** + * Create virtual columns containing "bonus" fields that should be attached to the {@link FrameWriter} for + * this processor. Kept in sync with the signature generated by {@link GroupByQueryKit}. + */ + private static VirtualColumns makeVirtualColumnsForFrameWriter(final GroupByQuery query) + { + final VirtualColumn segmentGranularityVirtualColumn = QueryKitUtils.makeSegmentGranularityVirtualColumn(query); + + if (segmentGranularityVirtualColumn == null) { + return VirtualColumns.EMPTY; + } else { + return VirtualColumns.create(Collections.singletonList(segmentGranularityVirtualColumn)); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java new file mode 100644 index 00000000000..5987eb02fae --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java @@ -0,0 +1,131 @@ +/* + * 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.druid.msq.querykit.groupby; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.BaseFrameProcessorFactory; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; +import java.util.function.Consumer; + +@JsonTypeName("groupByPostShuffle") +public class GroupByPostShuffleFrameProcessorFactory extends BaseFrameProcessorFactory +{ + private final GroupByQuery query; + + @JsonCreator + public GroupByPostShuffleFrameProcessorFactory( + @JsonProperty("query") GroupByQuery query + ) + { + this.query = query; + } + + @JsonProperty + public GroupByQuery getQuery() + { + return query; + } + + @Override + public ProcessorsAndChannels, Long> makeProcessors( + StageDefinition stageDefinition, + int workerNumber, + List inputSlices, + InputSliceReader inputSliceReader, + @Nullable Object extra, + OutputChannelFactory outputChannelFactory, + FrameContext frameContext, + int maxOutstandingProcessors, + CounterTracker counters, + Consumer warningPublisher + ) + { + // Expecting a single input slice from some prior stage. + final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); + final GroupByStrategySelector strategySelector = frameContext.groupByStrategySelector(); + + final Int2ObjectMap outputChannels = new Int2ObjectOpenHashMap<>(); + for (final ReadablePartition partition : slice.getPartitions()) { + outputChannels.computeIfAbsent( + partition.getPartitionNumber(), + i -> { + try { + return outputChannelFactory.openChannel(i); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ); + } + + final Sequence readableInputs = + Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); + + final Sequence> processors = readableInputs.map( + readableInput -> { + final OutputChannel outputChannel = + outputChannels.get(readableInput.getStagePartition().getPartitionNumber()); + + return new GroupByPostShuffleFrameProcessor( + query, + strategySelector, + readableInput.getChannel(), + outputChannel.getWritableChannel(), + readableInput.getChannelFrameReader(), + stageDefinition.getSignature(), + stageDefinition.getClusterBy(), + outputChannel.getFrameMemoryAllocator() + ); + } + ); + + return new ProcessorsAndChannels<>( + processors, + OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values())) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java new file mode 100644 index 00000000000..af407f01442 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -0,0 +1,245 @@ +/* + * 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.druid.msq.querykit.groupby; + +import com.google.common.collect.Iterables; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameRowTooLargeException; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.segment.FrameSegment; +import org.apache.druid.frame.write.FrameWriter; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.SpecificSegmentSpec; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.timeline.SegmentId; + +import java.io.IOException; + +/** + * A {@link FrameProcessor} that reads one {@link Frame} at a time from a particular segment, writes them + * to a {@link WritableFrameChannel}, and returns the number of rows output. + */ +public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor +{ + private final GroupByQuery query; + private final GroupByStrategySelector strategySelector; + private final RowSignature aggregationSignature; + private final ClusterBy clusterBy; + private final ColumnSelectorFactory frameWriterColumnSelectorFactory; + private final Closer closer = Closer.create(); + + private Yielder resultYielder; + private FrameWriter frameWriter; + private long rowsOutput; + private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed + + public GroupByPreShuffleFrameProcessor( + final GroupByQuery query, + final ReadableInput baseInput, + final Int2ObjectMap sideChannels, + final GroupByStrategySelector strategySelector, + final JoinableFactoryWrapper joinableFactory, + final RowSignature aggregationSignature, + final ClusterBy clusterBy, + final ResourceHolder outputChannel, + final ResourceHolder allocator, + final long memoryReservedForBroadcastJoin + ) + { + super( + query, + baseInput, + sideChannels, + joinableFactory, + outputChannel, + allocator, + memoryReservedForBroadcastJoin + ); + this.query = query; + this.strategySelector = strategySelector; + this.aggregationSignature = aggregationSignature; + this.clusterBy = clusterBy; + this.frameWriterColumnSelectorFactory = RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory( + query, + () -> resultYielder.get(), + RowSignature.Finalization.NO + ); + } + + @Override + protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException + { + if (resultYielder == null) { + closer.register(segment); + + final Sequence rowSequence = + strategySelector.strategize(query) + .process( + query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), + mapSegment(segment.getOrLoadSegment()).asStorageAdapter(), + null + ); + + resultYielder = Yielders.each(rowSequence); + } + + populateFrameWriterAndFlushIfNeeded(); + + if (resultYielder == null || resultYielder.isDone()) { + return ReturnOrAwait.returnObject(rowsOutput); + } else { + return ReturnOrAwait.runAgain(); + } + } + + @Override + protected ReturnOrAwait runWithInputChannel( + final ReadableFrameChannel inputChannel, + final FrameReader inputFrameReader + ) throws IOException + { + if (resultYielder == null || resultYielder.isDone()) { + closeAndDiscardResultYielder(); + + if (inputChannel.canRead()) { + final Frame frame = inputChannel.read(); + final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("x")); + + final Sequence rowSequence = + strategySelector.strategize(query) + .process( + query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), + mapSegment(frameSegment).asStorageAdapter(), + null + ); + + resultYielder = Yielders.each(rowSequence); + } else if (inputChannel.isFinished()) { + flushFrameWriterIfNeeded(); + return ReturnOrAwait.returnObject(rowsOutput); + } else { + return ReturnOrAwait.awaitAll(inputChannels().size()); + } + } + + // Cursor has some more data in it. + populateFrameWriterAndFlushIfNeeded(); + + if (resultYielder == null || resultYielder.isDone()) { + closeAndDiscardResultYielder(); + return ReturnOrAwait.awaitAll(inputChannels().size()); + } else { + return ReturnOrAwait.runAgain(); + } + } + + @Override + public void cleanup() throws IOException + { + closer.register(this::closeAndDiscardResultYielder); + closer.register(frameWriter); + closer.register(super::cleanup); + closer.close(); + } + + private void populateFrameWriterAndFlushIfNeeded() throws IOException + { + createFrameWriterIfNeeded(); + + while (!resultYielder.isDone()) { + final boolean didAddToFrame = frameWriter.addSelection(); + + if (didAddToFrame) { + resultYielder = resultYielder.next(null); + } else if (frameWriter.getNumRows() == 0) { + throw new FrameRowTooLargeException(currentAllocatorCapacity); + } else { + flushFrameWriterIfNeeded(); + return; + } + } + + flushFrameWriterIfNeeded(); + closeAndDiscardResultYielder(); + } + + private void createFrameWriterIfNeeded() + { + if (frameWriter == null) { + final MemoryAllocator allocator = getAllocator(); + final FrameWriterFactory frameWriterFactory = + FrameWriters.makeFrameWriterFactory( + FrameType.ROW_BASED, + allocator, + aggregationSignature, + clusterBy.getColumns() + ); + frameWriter = frameWriterFactory.newFrameWriter(frameWriterColumnSelectorFactory); + currentAllocatorCapacity = allocator.capacity(); + } + } + + private void flushFrameWriterIfNeeded() throws IOException + { + if (frameWriter != null && frameWriter.getNumRows() > 0) { + final Frame frame = Frame.wrap(frameWriter.toByteArray()); + Iterables.getOnlyElement(outputChannels()).write(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION)); + frameWriter.close(); + frameWriter = null; + rowsOutput += frame.numRows(); + } + } + + private void closeAndDiscardResultYielder() throws IOException + { + final Yielder tmp = resultYielder; + resultYielder = null; + + if (tmp != null) { + tmp.close(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java new file mode 100644 index 00000000000..b73f00efa2e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.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.druid.msq.querykit.groupby; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.querykit.BaseLeafFrameProcessorFactory; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinableFactoryWrapper; + +@JsonTypeName("groupByPreShuffle") +public class GroupByPreShuffleFrameProcessorFactory extends BaseLeafFrameProcessorFactory +{ + private final GroupByQuery query; + + @JsonCreator + public GroupByPreShuffleFrameProcessorFactory(@JsonProperty("query") GroupByQuery query) + { + this.query = Preconditions.checkNotNull(query, "query"); + } + + @JsonProperty + public GroupByQuery getQuery() + { + return query; + } + + @Override + protected FrameProcessor makeProcessor( + final ReadableInput baseInput, + final Int2ObjectMap sideChannels, + final ResourceHolder outputChannelSupplier, + final ResourceHolder allocatorSupplier, + final RowSignature signature, + final ClusterBy clusterBy, + final FrameContext frameContext + ) + { + return new GroupByPreShuffleFrameProcessor( + query, + baseInput, + sideChannels, + frameContext.groupByStrategySelector(), + new JoinableFactoryWrapper(frameContext.joinableFactory()), + signature, + clusterBy, + outputChannelSupplier, + allocatorSupplier, + frameContext.memoryParameters().getBroadcastJoinMemory() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java new file mode 100644 index 00000000000..3e494c805b2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java @@ -0,0 +1,302 @@ +/* + * 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.druid.msq.querykit.groupby; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.kernel.MaxCountShuffleSpec; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.QueryDefinitionBuilder; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.DataSourcePlan; +import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ShuffleSpecFactories; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.having.AlwaysHavingSpec; +import org.apache.druid.query.groupby.having.DimFilterHavingSpec; +import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.NoopLimitSpec; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.ordering.StringComparator; +import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +public class GroupByQueryKit implements QueryKit +{ + @Override + public QueryDefinition makeQueryDefinition( + final String queryId, + final GroupByQuery originalQuery, + final QueryKit> queryKit, + final ShuffleSpecFactory resultShuffleSpecFactory, + final int maxWorkerCount, + final int minStageNumber + ) + { + validateQuery(originalQuery); + + final QueryDefinitionBuilder queryDefBuilder = QueryDefinition.builder().queryId(queryId); + final DataSourcePlan dataSourcePlan = DataSourcePlan.forDataSource( + queryKit, + queryId, + originalQuery.getDataSource(), + originalQuery.getQuerySegmentSpec(), + originalQuery.getFilter(), + maxWorkerCount, + minStageNumber, + false + ); + + dataSourcePlan.getSubQueryDefBuilder().ifPresent(queryDefBuilder::addAll); + + final GroupByQuery queryToRun = (GroupByQuery) originalQuery.withDataSource(dataSourcePlan.getNewDataSource()); + final int firstStageNumber = Math.max(minStageNumber, queryDefBuilder.getNextStageNumber()); + + final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(queryToRun.getContext()); + final RowSignature intermediateSignature = computeIntermediateSignature(queryToRun); + final ClusterBy resultClusterBy = + QueryKitUtils.clusterByWithSegmentGranularity(computeClusterByForResults(queryToRun), segmentGranularity); + final RowSignature resultSignature = + QueryKitUtils.sortableSignature( + QueryKitUtils.signatureWithSegmentGranularity(computeResultSignature(queryToRun), segmentGranularity), + resultClusterBy.getColumns() + ); + final ClusterBy intermediateClusterBy = computeIntermediateClusterBy(queryToRun); + final boolean doOrderBy = !resultClusterBy.equals(intermediateClusterBy); + final boolean doLimitOrOffset = + queryToRun.getLimitSpec() instanceof DefaultLimitSpec + && (((DefaultLimitSpec) queryToRun.getLimitSpec()).isLimited() + || ((DefaultLimitSpec) queryToRun.getLimitSpec()).isOffset()); + + final ShuffleSpecFactory shuffleSpecFactoryPreAggregation; + final ShuffleSpecFactory shuffleSpecFactoryPostAggregation; + + if (intermediateClusterBy.getColumns().isEmpty()) { + // Ignore shuffleSpecFactory, since we know only a single partition will come out, and we can save some effort. + shuffleSpecFactoryPreAggregation = ShuffleSpecFactories.singlePartition(); + shuffleSpecFactoryPostAggregation = ShuffleSpecFactories.singlePartition(); + } else if (doOrderBy) { + shuffleSpecFactoryPreAggregation = ShuffleSpecFactories.subQueryWithMaxWorkerCount(maxWorkerCount); + shuffleSpecFactoryPostAggregation = doLimitOrOffset + ? ShuffleSpecFactories.singlePartition() + : resultShuffleSpecFactory; + } else { + shuffleSpecFactoryPreAggregation = doLimitOrOffset + ? ShuffleSpecFactories.singlePartition() + : resultShuffleSpecFactory; + + // null: retain partitions from input (i.e. from preAggregation). + shuffleSpecFactoryPostAggregation = null; + } + + queryDefBuilder.add( + StageDefinition.builder(firstStageNumber) + .inputs(dataSourcePlan.getInputSpecs()) + .broadcastInputs(dataSourcePlan.getBroadcastInputs()) + .signature(intermediateSignature) + .shuffleSpec(shuffleSpecFactoryPreAggregation.build(intermediateClusterBy, true)) + .maxWorkerCount(dataSourcePlan.isSingleWorker() ? 1 : maxWorkerCount) + .processorFactory(new GroupByPreShuffleFrameProcessorFactory(queryToRun)) + ); + + queryDefBuilder.add( + StageDefinition.builder(firstStageNumber + 1) + .inputs(new StageInputSpec(firstStageNumber)) + .signature(resultSignature) + .maxWorkerCount(maxWorkerCount) + .shuffleSpec( + shuffleSpecFactoryPostAggregation != null + ? shuffleSpecFactoryPostAggregation.build(resultClusterBy, false) + : null + ) + .processorFactory(new GroupByPostShuffleFrameProcessorFactory(queryToRun)) + ); + + if (doLimitOrOffset) { + final DefaultLimitSpec limitSpec = (DefaultLimitSpec) queryToRun.getLimitSpec(); + + queryDefBuilder.add( + StageDefinition.builder(firstStageNumber + 2) + .inputs(new StageInputSpec(firstStageNumber + 1)) + .signature(resultSignature) + .maxWorkerCount(1) + .shuffleSpec(new MaxCountShuffleSpec(ClusterBy.none(), 1, false)) + .processorFactory( + new OffsetLimitFrameProcessorFactory( + limitSpec.getOffset(), + limitSpec.isLimited() ? (long) limitSpec.getLimit() : null + ) + ) + ); + } + + return queryDefBuilder.queryId(queryId).build(); + } + + /** + * Intermediate signature of a particular {@link GroupByQuery}. Does not include post-aggregators, and all + * aggregations are nonfinalized. + */ + static RowSignature computeIntermediateSignature(final GroupByQuery query) + { + final RowSignature postAggregationSignature = query.getResultRowSignature(RowSignature.Finalization.NO); + final RowSignature.Builder builder = RowSignature.builder(); + + for (int i = 0; i < query.getResultRowSizeWithoutPostAggregators(); i++) { + builder.add( + postAggregationSignature.getColumnName(i), + postAggregationSignature.getColumnType(i).orElse(null) + ); + } + + return builder.build(); + } + + /** + * Result signature of a particular {@link GroupByQuery}. Includes post-aggregators, and aggregations are + * finalized by default. (But may be nonfinalized, depending on {@link #isFinalize}. + */ + static RowSignature computeResultSignature(final GroupByQuery query) + { + final RowSignature.Finalization finalization = + isFinalize(query) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO; + return query.getResultRowSignature(finalization); + } + + /** + * Whether aggregations appearing in the result of a query must be finalized. + * + * There is a discrepancy here with native execution. By default, native execution finalizes outer queries only. + * Here, we finalize all queries, including subqueries. + */ + static boolean isFinalize(final GroupByQuery query) + { + return QueryContexts.isFinalize(query, true); + } + + /** + * Clustering for the intermediate shuffle in a groupBy query. + */ + static ClusterBy computeIntermediateClusterBy(final GroupByQuery query) + { + final List columns = new ArrayList<>(); + + for (final DimensionSpec dimension : query.getDimensions()) { + columns.add(new SortColumn(dimension.getOutputName(), false)); + } + + // Note: ignoring time because we assume granularity = all. + return new ClusterBy(columns, 0); + } + + /** + * Clustering for the results of a groupBy query. + */ + static ClusterBy computeClusterByForResults(final GroupByQuery query) + { + if (query.getLimitSpec() instanceof DefaultLimitSpec) { + final DefaultLimitSpec defaultLimitSpec = (DefaultLimitSpec) query.getLimitSpec(); + + if (!defaultLimitSpec.getColumns().isEmpty()) { + final List clusterByColumns = new ArrayList<>(); + + for (final OrderByColumnSpec orderBy : defaultLimitSpec.getColumns()) { + clusterByColumns.add( + new SortColumn( + orderBy.getDimension(), + orderBy.getDirection() == OrderByColumnSpec.Direction.DESCENDING + ) + ); + } + + return new ClusterBy(clusterByColumns, 0); + } + } + + return computeIntermediateClusterBy(query); + } + + /** + * Returns silently if the provided {@link GroupByQuery} is supported by this kit. Throws an exception otherwise. + * + * @throws IllegalStateException if the query is not supported + */ + private static void validateQuery(final GroupByQuery query) + { + // Misc features that we do not support right now. + Preconditions.checkState(!query.getContextSortByDimsFirst(), "Must not sort by dims first"); + Preconditions.checkState(query.getSubtotalsSpec() == null, "Must not have 'subtotalsSpec'"); + // Matches condition in GroupByPostShuffleWorker.makeHavingFilter. + Preconditions.checkState( + query.getHavingSpec() == null + || query.getHavingSpec() instanceof DimFilterHavingSpec + || query.getHavingSpec() instanceof AlwaysHavingSpec, + "Must use 'filter' or 'always' havingSpec" + ); + Preconditions.checkState(query.getGranularity().equals(Granularities.ALL), "Must have granularity 'all'"); + Preconditions.checkState( + query.getLimitSpec() instanceof NoopLimitSpec || query.getLimitSpec() instanceof DefaultLimitSpec, + "Must have noop or default limitSpec" + ); + + final RowSignature resultSignature = computeResultSignature(query); + QueryKitUtils.verifyRowSignature(resultSignature); + + if (query.getLimitSpec() instanceof DefaultLimitSpec) { + final DefaultLimitSpec defaultLimitSpec = (DefaultLimitSpec) query.getLimitSpec(); + + for (final OrderByColumnSpec column : defaultLimitSpec.getColumns()) { + final Optional type = resultSignature.getColumnType(column.getDimension()); + + if (!type.isPresent() || !isNaturalComparator(type.get().getType(), column.getDimensionComparator())) { + throw new ISE( + "Must use natural comparator for column [%s] of type [%s]", + column.getDimension(), + type.orElse(null) + ); + } + } + } + } + + private static boolean isNaturalComparator(final ValueType type, final StringComparator comparator) + { + return ((type == ValueType.STRING && StringComparators.LEXICOGRAPHIC.equals(comparator)) + || (type.isNumeric() && StringComparators.NUMERIC.equals(comparator))) + && !type.isArray(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java new file mode 100644 index 00000000000..1b8d21baae1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -0,0 +1,316 @@ +/* + * 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.druid.msq.querykit.scan; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameRowTooLargeException; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.segment.FrameSegment; +import org.apache.druid.frame.util.SettableLongVirtualColumn; +import org.apache.druid.frame.write.FrameWriter; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.SpecificSegmentSpec; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +/** + * A {@link FrameProcessor} that reads one {@link Frame} at a time from a particular segment, writes them + * to a {@link WritableFrameChannel}, and returns the number of rows output. + */ +public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor +{ + private final ScanQuery query; + private final RowSignature signature; + private final ClusterBy clusterBy; + private final AtomicLong runningCountForLimit; + private final SettableLongVirtualColumn partitionBoostVirtualColumn; + private final VirtualColumns frameWriterVirtualColumns; + private final Closer closer = Closer.create(); + + private long rowsOutput = 0; + private Cursor cursor; + private FrameWriter frameWriter; + private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed + + public ScanQueryFrameProcessor( + final ScanQuery query, + final RowSignature signature, + final ClusterBy clusterBy, + final ReadableInput baseInput, + final Int2ObjectMap sideChannels, + final JoinableFactoryWrapper joinableFactory, + final ResourceHolder outputChannel, + final ResourceHolder allocator, + @Nullable final AtomicLong runningCountForLimit, + final long memoryReservedForBroadcastJoin + ) + { + super( + query, + baseInput, + sideChannels, + joinableFactory, + outputChannel, + allocator, + memoryReservedForBroadcastJoin + ); + this.query = query; + this.signature = signature; + this.clusterBy = clusterBy; + this.runningCountForLimit = runningCountForLimit; + this.partitionBoostVirtualColumn = new SettableLongVirtualColumn(QueryKitUtils.PARTITION_BOOST_COLUMN); + + final List frameWriterVirtualColumns = new ArrayList<>(); + frameWriterVirtualColumns.add(partitionBoostVirtualColumn); + + final VirtualColumn segmentGranularityVirtualColumn = QueryKitUtils.makeSegmentGranularityVirtualColumn(query); + + if (segmentGranularityVirtualColumn != null) { + frameWriterVirtualColumns.add(segmentGranularityVirtualColumn); + } + + this.frameWriterVirtualColumns = VirtualColumns.create(frameWriterVirtualColumns); + } + + @Override + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + { + final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter"); + + if (legacy) { + throw new ISE("Cannot use this engine in legacy mode"); + } + + if (runningCountForLimit != null + && runningCountForLimit.get() > query.getScanRowsOffset() + query.getScanRowsLimit()) { + return ReturnOrAwait.returnObject(rowsOutput); + } + + return super.runIncrementally(readableInputs); + } + + @Override + public void cleanup() throws IOException + { + closer.register(frameWriter); + closer.register(super::cleanup); + closer.close(); + } + + @Override + protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException + { + if (cursor == null) { + closer.register(segment); + + final Yielder cursorYielder = Yielders.each( + makeCursors( + query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), + mapSegment(segment.getOrLoadSegment()).asStorageAdapter() + ) + ); + + if (cursorYielder.isDone()) { + // No cursors! + cursorYielder.close(); + return ReturnOrAwait.returnObject(rowsOutput); + } else { + setNextCursor(cursorYielder.get()); + closer.register(cursorYielder); + } + } + + populateFrameWriterAndFlushIfNeeded(); + + if (cursor.isDone()) { + flushFrameWriter(); + } + + if (cursor.isDone() && (frameWriter == null || frameWriter.getNumRows() == 0)) { + return ReturnOrAwait.returnObject(rowsOutput); + } else { + return ReturnOrAwait.runAgain(); + } + } + + @Override + protected ReturnOrAwait runWithInputChannel( + final ReadableFrameChannel inputChannel, + final FrameReader inputFrameReader + ) throws IOException + { + if (cursor == null || cursor.isDone()) { + if (inputChannel.canRead()) { + final Frame frame = inputChannel.read(); + final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("x")); + + setNextCursor( + Iterables.getOnlyElement( + makeCursors( + query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), + mapSegment(frameSegment).asStorageAdapter() + ).toList() + ) + ); + } else if (inputChannel.isFinished()) { + flushFrameWriter(); + return ReturnOrAwait.returnObject(rowsOutput); + } else { + return ReturnOrAwait.awaitAll(inputChannels().size()); + } + } + + // Cursor has some more data in it. + populateFrameWriterAndFlushIfNeeded(); + + if (cursor.isDone()) { + return ReturnOrAwait.awaitAll(inputChannels().size()); + } else { + return ReturnOrAwait.runAgain(); + } + } + + private void populateFrameWriterAndFlushIfNeeded() throws IOException + { + createFrameWriterIfNeeded(); + + while (!cursor.isDone()) { + if (!frameWriter.addSelection()) { + if (frameWriter.getNumRows() > 0) { + final long numRowsWritten = flushFrameWriter(); + + if (runningCountForLimit != null) { + runningCountForLimit.addAndGet(numRowsWritten); + } + + return; + } else { + throw new FrameRowTooLargeException(currentAllocatorCapacity); + } + } + + cursor.advance(); + partitionBoostVirtualColumn.setValue(partitionBoostVirtualColumn.getValue() + 1); + } + } + + private void createFrameWriterIfNeeded() + { + if (frameWriter == null) { + final MemoryAllocator allocator = getAllocator(); + final FrameWriterFactory frameWriterFactory = + FrameWriters.makeFrameWriterFactory(FrameType.ROW_BASED, allocator, signature, clusterBy.getColumns()); + final ColumnSelectorFactory frameWriterColumnSelectorFactory = + frameWriterVirtualColumns.wrap(cursor.getColumnSelectorFactory()); + frameWriter = frameWriterFactory.newFrameWriter(frameWriterColumnSelectorFactory); + currentAllocatorCapacity = allocator.capacity(); + } + } + + private long flushFrameWriter() throws IOException + { + if (frameWriter != null && frameWriter.getNumRows() > 0) { + final Frame frame = Frame.wrap(frameWriter.toByteArray()); + Iterables.getOnlyElement(outputChannels()).write(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION)); + frameWriter.close(); + frameWriter = null; + rowsOutput += frame.numRows(); + return frame.numRows(); + } else { + if (frameWriter != null) { + frameWriter.close(); + frameWriter = null; + } + + return 0; + } + } + + private void setNextCursor(final Cursor cursor) throws IOException + { + flushFrameWriter(); + this.cursor = cursor; + } + + private static Sequence makeCursors(final ScanQuery query, final StorageAdapter adapter) + { + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + + final List intervals = query.getQuerySegmentSpec().getIntervals(); + Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals); + + final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); + + return adapter.makeCursors( + filter, + intervals.get(0), + query.getVirtualColumns(), + Granularities.ALL, + ScanQuery.Order.DESCENDING.equals(query.getTimeOrder()), + null + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java new file mode 100644 index 00000000000..08b41ddfc3c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java @@ -0,0 +1,96 @@ +/* + * 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.druid.msq.querykit.scan; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.querykit.BaseLeafFrameProcessorFactory; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinableFactoryWrapper; + +import javax.annotation.Nullable; +import java.util.concurrent.atomic.AtomicLong; + +@JsonTypeName("scan") +public class ScanQueryFrameProcessorFactory extends BaseLeafFrameProcessorFactory +{ + private final ScanQuery query; + + /** + * Instantiated and passed to all the {@link ScanQueryFrameProcessor}s created from this factory to keep a track + * of the number of rows processed so far in case the query is limited (without any order by) because one doesn't need + * to scan through all the rows in that case. + * This is not ideal because nothing really guarantees this factory is used for a single makeWorkers call + */ + @Nullable + private final AtomicLong runningCountForLimit; + + @JsonCreator + public ScanQueryFrameProcessorFactory( + @JsonProperty("query") ScanQuery query + ) + { + this.query = Preconditions.checkNotNull(query, "query"); + this.runningCountForLimit = + query.isLimited() && query.getOrderBys().isEmpty() ? new AtomicLong() : null; + } + + @JsonProperty + public ScanQuery getQuery() + { + return query; + } + + @Override + protected FrameProcessor makeProcessor( + ReadableInput baseInput, + Int2ObjectMap sideChannels, + ResourceHolder outputChannelSupplier, + ResourceHolder allocatorSupplier, + RowSignature signature, + ClusterBy clusterBy, + FrameContext frameContext + ) + { + return new ScanQueryFrameProcessor( + query, + signature, + clusterBy, + baseInput, + sideChannels, + new JoinableFactoryWrapper(frameContext.joinableFactory()), + outputChannelSupplier, + allocatorSupplier, + runningCountForLimit, + frameContext.memoryParameters().getBroadcastJoinMemory() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java new file mode 100644 index 00000000000..c8341a1b476 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -0,0 +1,173 @@ +/* + * 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.druid.msq.querykit.scan; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.kernel.MaxCountShuffleSpec; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.QueryDefinitionBuilder; +import org.apache.druid.msq.kernel.ShuffleSpec; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.DataSourcePlan; +import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.query.Query; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.rel.DruidQuery; + +import java.util.ArrayList; +import java.util.List; + +public class ScanQueryKit implements QueryKit +{ + private final ObjectMapper jsonMapper; + + public ScanQueryKit(final ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + public static RowSignature getAndValidateSignature(final ScanQuery scanQuery, final ObjectMapper jsonMapper) + { + RowSignature scanSignature; + try { + final String s = scanQuery.getContextValue(DruidQuery.CTX_SCAN_SIGNATURE); + scanSignature = jsonMapper.readValue(s, RowSignature.class); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + // Verify the signature prior to any actual processing. + QueryKitUtils.verifyRowSignature(scanSignature); + return scanSignature; + } + + /** + * We ignore the resultShuffleSpecFactory in case: + * 1. There is no cluster by + * 2. This is an offset which means everything gets funneled into a single partition hence we use MaxCountShuffleSpec + */ + // No ordering, but there is a limit or an offset. These work by funneling everything through a single partition. + // So there is no point in forcing any particular partitioning. Since everything is funnelled into a single + // partition without a ClusterBy, we don't need to necessarily create it via the resultShuffleSpecFactory provided + @Override + public QueryDefinition makeQueryDefinition( + final String queryId, + final ScanQuery originalQuery, + final QueryKit> queryKit, + final ShuffleSpecFactory resultShuffleSpecFactory, + final int maxWorkerCount, + final int minStageNumber + ) + { + final QueryDefinitionBuilder queryDefBuilder = QueryDefinition.builder().queryId(queryId); + final DataSourcePlan dataSourcePlan = DataSourcePlan.forDataSource( + queryKit, + queryId, + originalQuery.getDataSource(), + originalQuery.getQuerySegmentSpec(), + originalQuery.getFilter(), + maxWorkerCount, + minStageNumber, + false + ); + + dataSourcePlan.getSubQueryDefBuilder().ifPresent(queryDefBuilder::addAll); + + final ScanQuery queryToRun = originalQuery.withDataSource(dataSourcePlan.getNewDataSource()); + final int firstStageNumber = Math.max(minStageNumber, queryDefBuilder.getNextStageNumber()); + final RowSignature scanSignature = getAndValidateSignature(queryToRun, jsonMapper); + final ShuffleSpec shuffleSpec; + final RowSignature signatureToUse; + final boolean hasLimitOrOffset = queryToRun.isLimited() || queryToRun.getScanRowsOffset() > 0; + + + // We ignore the resultShuffleSpecFactory in case: + // 1. There is no cluster by + // 2. There is an offset which means everything gets funneled into a single partition hence we use MaxCountShuffleSpec + if (queryToRun.getOrderBys().isEmpty() && hasLimitOrOffset) { + shuffleSpec = new MaxCountShuffleSpec(ClusterBy.none(), 1, false); + signatureToUse = scanSignature; + } else { + final RowSignature.Builder signatureBuilder = RowSignature.builder().addAll(scanSignature); + final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(queryToRun.getContext()); + final List clusterByColumns = new ArrayList<>(); + + // Add regular orderBys. + for (final ScanQuery.OrderBy orderBy : queryToRun.getOrderBys()) { + clusterByColumns.add( + new SortColumn( + orderBy.getColumnName(), + orderBy.getOrder() == ScanQuery.Order.DESCENDING + ) + ); + } + + // Add partition boosting column. + clusterByColumns.add(new SortColumn(QueryKitUtils.PARTITION_BOOST_COLUMN, false)); + signatureBuilder.add(QueryKitUtils.PARTITION_BOOST_COLUMN, ColumnType.LONG); + + final ClusterBy clusterBy = + QueryKitUtils.clusterByWithSegmentGranularity(new ClusterBy(clusterByColumns, 0), segmentGranularity); + shuffleSpec = resultShuffleSpecFactory.build(clusterBy, false); + signatureToUse = QueryKitUtils.sortableSignature( + QueryKitUtils.signatureWithSegmentGranularity(signatureBuilder.build(), segmentGranularity), + clusterBy.getColumns() + ); + } + + queryDefBuilder.add( + StageDefinition.builder(Math.max(minStageNumber, queryDefBuilder.getNextStageNumber())) + .inputs(dataSourcePlan.getInputSpecs()) + .broadcastInputs(dataSourcePlan.getBroadcastInputs()) + .shuffleSpec(shuffleSpec) + .signature(signatureToUse) + .maxWorkerCount(dataSourcePlan.isSingleWorker() ? 1 : maxWorkerCount) + .processorFactory(new ScanQueryFrameProcessorFactory(queryToRun)) + ); + + if (hasLimitOrOffset) { + queryDefBuilder.add( + StageDefinition.builder(firstStageNumber + 1) + .inputs(new StageInputSpec(firstStageNumber)) + .signature(signatureToUse) + .maxWorkerCount(1) + .shuffleSpec(new MaxCountShuffleSpec(ClusterBy.none(), 1, false)) + .processorFactory( + new OffsetLimitFrameProcessorFactory( + queryToRun.getScanRowsOffset(), + queryToRun.isLimited() ? queryToRun.getScanRowsLimit() : null + ) + ) + ); + } + + return queryDefBuilder.build(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClient.java new file mode 100644 index 00000000000..1278a52ae08 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClient.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.druid.msq.rpc; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.timeline.DataSegment; + +/** + * Interface for {@link org.apache.druid.rpc.ServiceClient}-backed communication with the Coordinator. + */ +public interface CoordinatorServiceClient +{ + /** + * Fetches segment metadata for the given dataSource and segmentId from the Coordinator + */ + ListenableFuture fetchUsedSegment(String dataSource, String segmentId); + + /** + * Returns a new CoordinatorServiceClient backed by a ServiceClient which follows the provided retryPolicy + */ + CoordinatorServiceClient withRetryPolicy(ServiceRetryPolicy retryPolicy); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClientImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClientImpl.java new file mode 100644 index 00000000000..b8281578de6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/CoordinatorServiceClientImpl.java @@ -0,0 +1,90 @@ +/* + * 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.druid.msq.rpc; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.timeline.DataSegment; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import java.io.IOException; + +/** + * Production implementation of {@link CoordinatorServiceClient}. + */ +public class CoordinatorServiceClientImpl implements CoordinatorServiceClient +{ + private final ServiceClient client; + private final ObjectMapper jsonMapper; + + public CoordinatorServiceClientImpl(final ServiceClient client, final ObjectMapper jsonMapper) + { + this.client = Preconditions.checkNotNull(client, "client"); + this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); + } + + @Override + public ListenableFuture fetchUsedSegment(String dataSource, String segmentId) + { + final String path = StringUtils.format( + "/druid/coordinator/v1/metadata/datasources/%s/segments/%s", + StringUtils.urlEncode(dataSource), + StringUtils.urlEncode(segmentId) + ); + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, new TypeReference() {}) + ); + } + + @Override + public CoordinatorServiceClient withRetryPolicy(ServiceRetryPolicy retryPolicy) + { + return new CoordinatorServiceClientImpl(client.withRetryPolicy(retryPolicy), jsonMapper); + } + + /** + * Deserialize a {@link BytesFullResponseHolder} as JSON. + * + * It would be reasonable to move this to {@link BytesFullResponseHolder} itself, or some shared utility class. + */ + private T deserialize(final BytesFullResponseHolder bytesHolder, final TypeReference typeReference) + { + try { + return jsonMapper.readValue(bytesHolder.getContent(), typeReference); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java new file mode 100644 index 00000000000..f7beba6c25d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java @@ -0,0 +1,122 @@ +/* + * 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.druid.msq.shuffle; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableInputStreamFrameChannel; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.storage.StorageConnector; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.function.Supplier; + +/** + * Provides input channels connected to durable storage. + */ +public class DurableStorageInputChannelFactory implements InputChannelFactory +{ + private final StorageConnector storageConnector; + private final ExecutorService remoteInputStreamPool; + private final String controllerTaskId; + private final Supplier> taskList; + + public DurableStorageInputChannelFactory( + final String controllerTaskId, + final Supplier> taskList, + final StorageConnector storageConnector, + final ExecutorService remoteInputStreamPool + ) + { + this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId"); + this.taskList = Preconditions.checkNotNull(taskList, "taskList"); + this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector"); + this.remoteInputStreamPool = Preconditions.checkNotNull(remoteInputStreamPool, "remoteInputStreamPool"); + } + + /** + * Creates an instance that is the standard production implementation. Closeable items are registered with + * the provided Closer. + */ + public static DurableStorageInputChannelFactory createStandardImplementation( + final String controllerTaskId, + final Supplier> taskList, + final StorageConnector storageConnector, + final Closer closer + ) + { + final ExecutorService remoteInputStreamPool = + Executors.newCachedThreadPool(Execs.makeThreadFactory(controllerTaskId + "-remote-fetcher-%d")); + closer.register(remoteInputStreamPool::shutdownNow); + return new DurableStorageInputChannelFactory(controllerTaskId, taskList, storageConnector, remoteInputStreamPool); + } + + @Override + public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException + { + final String workerTaskId = taskList.get().get(workerNumber); + + try { + final String remotePartitionPath = DurableStorageOutputChannelFactory.getPartitionFileName( + controllerTaskId, + workerTaskId, + stageId.getStageNumber(), + partitionNumber + ); + RetryUtils.retry(() -> { + if (!storageConnector.pathExists(remotePartitionPath)) { + throw new ISE( + "Could not find remote output of worker task[%s] stage[%d] partition[%d]", + workerTaskId, + stageId.getStageNumber(), + partitionNumber + ); + } + return Boolean.TRUE; + }, (throwable) -> true, 10); + final InputStream inputStream = storageConnector.read(remotePartitionPath); + + return ReadableInputStreamFrameChannel.open( + inputStream, + remotePartitionPath, + remoteInputStreamPool + ); + } + catch (Exception e) { + throw new IOE( + e, + "Could not find remote output of worker task[%s] stage[%d] partition[%d]", + workerTaskId, + stageId.getStageNumber(), + partitionNumber + ); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java new file mode 100644 index 00000000000..1f4361a9397 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java @@ -0,0 +1,142 @@ +/* + * 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.druid.msq.shuffle; + +import com.google.common.base.Preconditions; +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.ReadableNilFrameChannel; +import org.apache.druid.frame.channel.WritableFrameFileChannel; +import org.apache.druid.frame.file.FrameFileWriter; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.storage.StorageConnector; + +import java.io.IOException; +import java.nio.channels.Channels; + +public class DurableStorageOutputChannelFactory implements OutputChannelFactory +{ + private final String controllerTaskId; + private final String workerTaskId; + private final int stageNumber; + private final int frameSize; + private final StorageConnector storageConnector; + + public DurableStorageOutputChannelFactory( + final String controllerTaskId, + final String workerTaskId, + final int stageNumber, + final int frameSize, + final StorageConnector storageConnector + ) + { + this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId"); + this.workerTaskId = Preconditions.checkNotNull(workerTaskId, "workerTaskId"); + this.stageNumber = stageNumber; + this.frameSize = frameSize; + this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector"); + } + + /** + * Creates an instance that is the standard production implementation. Closeable items are registered with + * the provided Closer. + */ + public static DurableStorageOutputChannelFactory createStandardImplementation( + final String controllerTaskId, + final String workerTaskId, + final int stageNumber, + final int frameSize, + final StorageConnector storageConnector + ) + { + return new DurableStorageOutputChannelFactory( + controllerTaskId, + workerTaskId, + stageNumber, + frameSize, + storageConnector + ); + } + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + final String fileName = getPartitionFileName(controllerTaskId, workerTaskId, stageNumber, partitionNumber); + final WritableFrameFileChannel writableChannel = + new WritableFrameFileChannel( + FrameFileWriter.open( + Channels.newChannel(storageConnector.write(fileName)), + null + ) + ); + + return OutputChannel.pair( + writableChannel, + ArenaMemoryAllocator.createOnHeap(frameSize), + () -> ReadableNilFrameChannel.INSTANCE, // remote reads should happen via the DurableStorageInputChannelFactory + partitionNumber + ); + } + + @Override + public OutputChannel openNilChannel(int partitionNumber) + { + final String fileName = getPartitionFileName(controllerTaskId, workerTaskId, stageNumber, partitionNumber); + // As tasks dependent on output of this partition will forever block if no file is present in RemoteStorage. Hence, writing a dummy frame. + try { + + FrameFileWriter.open(Channels.newChannel(storageConnector.write(fileName)), null).close(); + return OutputChannel.nil(partitionNumber); + } + catch (IOException e) { + throw new ISE( + e, + "Unable to create empty remote output of workerTask[%s] stage[%d] partition[%d]", + workerTaskId, + stageNumber, + partitionNumber + ); + } + } + + public static String getControllerDirectory(final String controllerTaskId) + { + return StringUtils.format("controller_%s", IdUtils.validateId("controller task ID", controllerTaskId)); + } + + public static String getPartitionFileName( + final String controllerTaskId, + final String workerTaskId, + final int stageNumber, + final int partitionNumber + ) + { + return StringUtils.format( + "%s/worker_%s/stage_%d/part_%d", + getControllerDirectory(controllerTaskId), + IdUtils.validateId("worker task ID", workerTaskId), + stageNumber, + partitionNumber + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java new file mode 100644 index 00000000000..4aa43910671 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java @@ -0,0 +1,101 @@ +/* + * 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.druid.msq.shuffle; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.kernel.StageId; + +import java.util.List; +import java.util.function.Supplier; + +/** + * Provides input channels connected to workers via {@link WorkerClient#fetchChannelData}. + */ +public class WorkerInputChannelFactory implements InputChannelFactory +{ + private final WorkerClient workerClient; + private final Supplier> taskList; + + public WorkerInputChannelFactory(final WorkerClient workerClient, final Supplier> taskList) + { + this.workerClient = Preconditions.checkNotNull(workerClient, "workerClient"); + this.taskList = Preconditions.checkNotNull(taskList, "taskList"); + } + + @Override + public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) + { + final String taskId = taskList.get().get(workerNumber); + final ReadableByteChunksFrameChannel channel = + ReadableByteChunksFrameChannel.create(makeChannelId(taskId, stageId, partitionNumber)); + fetch(taskId, stageId, partitionNumber, 0, channel); + return channel; + } + + /** + * Start a fetch chain for a particular channel starting at a particular offset. + */ + private void fetch( + final String taskId, + final StageId stageId, + final int partitionNumber, + final long offset, + final ReadableByteChunksFrameChannel channel + ) + { + final ListenableFuture fetchFuture = + workerClient.fetchChannelData(taskId, stageId, partitionNumber, offset, channel); + + Futures.addCallback( + fetchFuture, + new FutureCallback() + { + @Override + public void onSuccess(final Boolean lastFetch) + { + if (lastFetch) { + channel.doneWriting(); + } else { + fetch(taskId, stageId, partitionNumber, channel.getBytesAdded(), channel); + } + } + + @Override + public void onFailure(Throwable t) + { + channel.setError(t); + } + } + ); + } + + private static String makeChannelId(final String workerTaskId, final StageId stageId, final int partitionNumber) + { + return StringUtils.format("%s:%s:%s", workerTaskId, stageId, partitionNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQMode.java new file mode 100644 index 00000000000..e1daafadf29 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQMode.java @@ -0,0 +1,81 @@ +/* + * 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.druid.msq.sql; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.indexing.error.MSQWarnings; +import org.apache.druid.query.QueryContext; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +public enum MSQMode +{ + NON_STRICT_MODE("nonStrict", ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, -1)), + STRICT_MODE("strict", ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0)); + + private final String value; + private final Map defaultQueryContext; + + private static final Logger log = new Logger(MSQMode.class); + + MSQMode(final String value, final Map defaultQueryContext) + { + this.value = value; + this.defaultQueryContext = new HashMap<>(defaultQueryContext); + } + + @Nullable + public static MSQMode fromString(String str) + { + for (MSQMode msqMode : MSQMode.values()) { + if (msqMode.value.equalsIgnoreCase(str)) { + return msqMode; + } + } + return null; + } + + @Override + public String toString() + { + return value; + } + + public static void populateDefaultQueryContext(final String modeStr, final QueryContext originalQueryContext) + { + MSQMode mode = MSQMode.fromString(modeStr); + if (mode == null) { + throw new ISE( + "%s is an unknown multi stage query mode. Acceptable modes: %s", + modeStr, + Arrays.stream(MSQMode.values()).map(m -> m.value).collect(Collectors.toList()) + ); + } + Map defaultQueryContext = mode.defaultQueryContext; + log.debug("Populating default query context with %s for the %s multi stage query mode", defaultQueryContext, mode); + originalQueryContext.addDefaultParams(defaultQueryContext); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java new file mode 100644 index 00000000000..53bb5bf8461 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -0,0 +1,298 @@ +/* + * 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.druid.msq.sql; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.util.Pair; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.msq.exec.MSQTasks; +import org.apache.druid.msq.indexing.ColumnMapping; +import org.apache.druid.msq.indexing.ColumnMappings; +import org.apache.druid.msq.indexing.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQDestination; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; +import org.apache.druid.sql.calcite.parser.DruidSqlReplace; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.sql.calcite.rel.Grouping; +import org.apache.druid.sql.calcite.run.QueryMaker; +import org.apache.druid.sql.calcite.table.RowSignatures; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class MSQTaskQueryMaker implements QueryMaker +{ + + private static final String DESTINATION_DATASOURCE = "dataSource"; + private static final String DESTINATION_REPORT = "taskReport"; + + private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; + private static final int DEFAULT_ROWS_PER_SEGMENT = 3000000; + + // Lower than the default to minimize the impact of per-row overheads that are not accounted for by + // OnheapIncrementalIndex. For example: overheads related to creating bitmaps during persist. + private static final int DEFAULT_ROWS_IN_MEMORY = 100000; + + private final String targetDataSource; + private final OverlordClient overlordClient; + private final PlannerContext plannerContext; + private final ObjectMapper jsonMapper; + private final List> fieldMapping; + + MSQTaskQueryMaker( + @Nullable final String targetDataSource, + final OverlordClient overlordClient, + final PlannerContext plannerContext, + final ObjectMapper jsonMapper, + final List> fieldMapping + ) + { + this.targetDataSource = targetDataSource; + this.overlordClient = Preconditions.checkNotNull(overlordClient, "indexingServiceClient"); + this.plannerContext = Preconditions.checkNotNull(plannerContext, "plannerContext"); + this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); + this.fieldMapping = Preconditions.checkNotNull(fieldMapping, "fieldMapping"); + } + + @Override + public Sequence runQuery(final DruidQuery druidQuery) + { + String taskId = MSQTasks.controllerTaskId(plannerContext.getSqlQueryId()); + + String msqMode = MultiStageQueryContext.getMSQMode(plannerContext.getQueryContext()); + if (msqMode != null) { + MSQMode.populateDefaultQueryContext(msqMode, plannerContext.getQueryContext()); + } + + final String ctxDestination = + DimensionHandlerUtils.convertObjectToString(MultiStageQueryContext.getDestination(plannerContext.getQueryContext())); + + Object segmentGranularity; + try { + segmentGranularity = Optional.ofNullable(plannerContext.getQueryContext() + .get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY)) + .orElse(jsonMapper.writeValueAsString(DEFAULT_SEGMENT_GRANULARITY)); + } + catch (JsonProcessingException e) { + throw new IAE("Unable to deserialize the insert granularity. Please retry the query with a valid " + + "segment graularity"); + } + + final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(plannerContext.getQueryContext()); + + if (maxNumTasks < 2) { + throw new IAE(MultiStageQueryContext.CTX_MAX_NUM_TASKS + + " cannot be less than 2 since at least 1 controller and 1 worker is necessary."); + } + + // This parameter is used internally for the number of worker tasks only, so we subtract 1 + final int maxNumWorkers = maxNumTasks - 1; + + final int rowsPerSegment = MultiStageQueryContext.getRowsPerSegment( + plannerContext.getQueryContext(), + DEFAULT_ROWS_PER_SEGMENT + ); + + final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory( + plannerContext.getQueryContext(), + DEFAULT_ROWS_IN_MEMORY + ); + + final boolean finalizeAggregations = MultiStageQueryContext.isFinalizeAggregations(plannerContext.getQueryContext()); + + final List replaceTimeChunks = + Optional.ofNullable(plannerContext.getQueryContext().get(DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS)) + .map( + s -> { + if (s instanceof String && "all".equals(StringUtils.toLowerCase((String) s))) { + return Intervals.ONLY_ETERNITY; + } else { + final String[] parts = ((String) s).split("\\s*,\\s*"); + final List intervals = new ArrayList<>(); + + for (final String part : parts) { + intervals.add(Intervals.of(part)); + } + + return intervals; + } + } + ) + .orElse(null); + + // For assistance computing return types if !finalizeAggregations. + final Map aggregationIntermediateTypeMap = + finalizeAggregations ? null /* Not needed */ : buildAggregationIntermediateTypeMap(druidQuery); + + final List sqlTypeNames = new ArrayList<>(); + final List columnMappings = new ArrayList<>(); + + for (final Pair entry : fieldMapping) { + // Note: SQL generally allows output columns to be duplicates, but MultiStageQueryMakerFactory.validateNoDuplicateAliases + // will prevent duplicate output columns from appearing here. So no need to worry about it. + + final String queryColumn = druidQuery.getOutputRowSignature().getColumnName(entry.getKey()); + final String outputColumns = entry.getValue(); + + final SqlTypeName sqlTypeName; + + if (!finalizeAggregations && aggregationIntermediateTypeMap.containsKey(queryColumn)) { + final ColumnType druidType = aggregationIntermediateTypeMap.get(queryColumn); + sqlTypeName = new RowSignatures.ComplexSqlType(SqlTypeName.OTHER, druidType, true).getSqlTypeName(); + } else { + sqlTypeName = druidQuery.getOutputRowType().getFieldList().get(entry.getKey()).getType().getSqlTypeName(); + } + + sqlTypeNames.add(sqlTypeName.getName()); + columnMappings.add(new ColumnMapping(queryColumn, outputColumns)); + } + + final MSQDestination destination; + + if (targetDataSource != null) { + if (ctxDestination != null && !DESTINATION_DATASOURCE.equals(ctxDestination)) { + throw new IAE("Cannot INSERT with destination [%s]", ctxDestination); + } + + Granularity segmentGranularityObject; + try { + segmentGranularityObject = jsonMapper.readValue((String) segmentGranularity, Granularity.class); + } + catch (Exception e) { + throw new ISE("Unable to convert %s to a segment granularity", segmentGranularity); + } + + final List segmentSortOrder = MultiStageQueryContext.decodeSortOrder( + MultiStageQueryContext.getSortOrder(plannerContext.getQueryContext()) + ); + + validateSegmentSortOrder( + segmentSortOrder, + fieldMapping.stream().map(f -> f.right).collect(Collectors.toList()) + ); + + destination = new DataSourceMSQDestination( + targetDataSource, + segmentGranularityObject, + segmentSortOrder, + replaceTimeChunks + ); + } else { + if (ctxDestination != null && !DESTINATION_REPORT.equals(ctxDestination)) { + throw new IAE("Cannot SELECT with destination [%s]", ctxDestination); + } + + destination = TaskReportMSQDestination.instance(); + } + + final Map nativeQueryContextOverrides = new HashMap<>(); + + // Add appropriate finalization to native query context. + nativeQueryContextOverrides.put(QueryContexts.FINALIZE_KEY, finalizeAggregations); + + final MSQSpec querySpec = + MSQSpec.builder() + .query(druidQuery.getQuery().withOverriddenContext(nativeQueryContextOverrides)) + .columnMappings(new ColumnMappings(columnMappings)) + .destination(destination) + .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(plannerContext.getQueryContext())) + .tuningConfig(new MSQTuningConfig(maxNumWorkers, maxRowsInMemory, rowsPerSegment)) + .build(); + + final MSQControllerTask controllerTask = new MSQControllerTask( + taskId, + querySpec, + plannerContext.getSql(), + plannerContext.getQueryContext().getMergedParams(), + sqlTypeNames, + null + ); + + FutureUtils.getUnchecked(overlordClient.runTask(taskId, controllerTask), true); + return Sequences.simple(Collections.singletonList(new Object[]{taskId})); + } + + private static Map buildAggregationIntermediateTypeMap(final DruidQuery druidQuery) + { + final Grouping grouping = druidQuery.getGrouping(); + + if (grouping == null) { + return Collections.emptyMap(); + } + + final Map retVal = new HashMap<>(); + + for (final AggregatorFactory aggregatorFactory : grouping.getAggregatorFactories()) { + retVal.put(aggregatorFactory.getName(), aggregatorFactory.getIntermediateType()); + } + + return retVal; + } + + static void validateSegmentSortOrder(final List sortOrder, final Collection allOutputColumns) + { + final Set allOutputColumnsSet = new HashSet<>(allOutputColumns); + + for (final String column : sortOrder) { + if (!allOutputColumnsSet.contains(column)) { + throw new IAE("Column [%s] in segment sort order does not appear in the query output", column); + } + } + + if (sortOrder.size() > 0 + && allOutputColumns.contains(ColumnHolder.TIME_COLUMN_NAME) + && !ColumnHolder.TIME_COLUMN_NAME.equals(sortOrder.get(0))) { + throw new IAE("Segment sort order must begin with column [%s]", ColumnHolder.TIME_COLUMN_NAME); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java new file mode 100644 index 00000000000..593eccac4fb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -0,0 +1,299 @@ +/* + * 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.druid.msq.sql; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Inject; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelRoot; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.ValidationException; +import org.apache.calcite.util.Pair; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; +import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.run.EngineFeature; +import org.apache.druid.sql.calcite.run.NativeSqlEngine; +import org.apache.druid.sql.calcite.run.QueryMaker; +import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.sql.calcite.run.SqlEngines; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +@LazySingleton +public class MSQTaskSqlEngine implements SqlEngine +{ + public static final Set SYSTEM_CONTEXT_PARAMETERS = + ImmutableSet.builder() + .addAll(NativeSqlEngine.SYSTEM_CONTEXT_PARAMETERS) + .add(MultiStageQueryContext.CTX_DESTINATION) + .add(QueryKitUtils.CTX_TIME_COLUMN_NAME) + .build(); + + public static final List TASK_STRUCT_FIELD_NAMES = ImmutableList.of("TASK"); + private static final String NAME = "msq-task"; + + private final OverlordClient overlordClient; + private final ObjectMapper jsonMapper; + + @Inject + public MSQTaskSqlEngine( + final OverlordClient overlordClient, + final ObjectMapper jsonMapper + ) + { + this.overlordClient = overlordClient; + this.jsonMapper = jsonMapper; + } + + @Override + public String name() + { + return NAME; + } + + @Override + public void validateContext(QueryContext queryContext) throws ValidationException + { + SqlEngines.validateNoSpecialContextKeys(queryContext, SYSTEM_CONTEXT_PARAMETERS); + } + + @Override + public RelDataType resultTypeForSelect(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + { + return getMSQStructType(typeFactory); + } + + @Override + public RelDataType resultTypeForInsert(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + { + return getMSQStructType(typeFactory); + } + + @Override + public boolean feature(EngineFeature feature, PlannerContext plannerContext) + { + switch (feature) { + case ALLOW_BINDABLE_PLAN: + case TIMESERIES_QUERY: + case TOPN_QUERY: + case TIME_BOUNDARY_QUERY: + return false; + case CAN_SELECT: + case CAN_INSERT: + case CAN_REPLACE: + case READ_EXTERNAL_DATA: + case SCAN_ORDER_BY_NON_TIME: + case SCAN_NEEDS_SIGNATURE: + return true; + default: + throw new IAE("Unrecognized feature: %s", feature); + } + } + + @Override + public QueryMaker buildQueryMakerForSelect( + final RelRoot relRoot, + final PlannerContext plannerContext + ) throws ValidationException + { + validateSelect(relRoot.fields, plannerContext); + + return new MSQTaskQueryMaker( + null, + overlordClient, + plannerContext, + jsonMapper, + relRoot.fields + ); + } + + @Override + public QueryMaker buildQueryMakerForInsert( + final String targetDataSource, + final RelRoot relRoot, + final PlannerContext plannerContext + ) throws ValidationException + { + validateInsert(relRoot.rel, relRoot.fields, plannerContext); + + return new MSQTaskQueryMaker( + targetDataSource, + overlordClient, + plannerContext, + jsonMapper, + relRoot.fields + ); + } + + private static void validateSelect( + final List> fieldMappings, + final PlannerContext plannerContext + ) throws ValidationException + { + validateNoDuplicateAliases(fieldMappings); + + if (plannerContext.getQueryContext().containsKey(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY)) { + throw new ValidationException( + StringUtils.format("Cannot use \"%s\" without INSERT", DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY) + ); + } + } + + private static void validateInsert( + final RelNode rootRel, + final List> fieldMappings, + final PlannerContext plannerContext + ) throws ValidationException + { + validateNoDuplicateAliases(fieldMappings); + + // Find the __time field. + int timeFieldIndex = -1; + + for (final Pair field : fieldMappings) { + if (field.right.equals(ColumnHolder.TIME_COLUMN_NAME)) { + timeFieldIndex = field.left; + + // Validate the __time field has the proper type. + final SqlTypeName timeType = rootRel.getRowType().getFieldList().get(field.left).getType().getSqlTypeName(); + if (timeType != SqlTypeName.TIMESTAMP) { + throw new ValidationException( + StringUtils.format( + "Field \"%s\" must be of type TIMESTAMP (was %s)", + ColumnHolder.TIME_COLUMN_NAME, + timeType + ) + ); + } + } + } + + // Validate that if segmentGranularity is not ALL then there is also a __time field. + final Granularity segmentGranularity; + + try { + segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext( + plannerContext.getQueryContext().getMergedParams() + ); + } + catch (Exception e) { + throw new ValidationException( + StringUtils.format( + "Invalid segmentGranularity: %s", + plannerContext.getQueryContext().get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY) + ), + e + ); + } + + final boolean hasSegmentGranularity = !Granularities.ALL.equals(segmentGranularity); + + // Validate that the query does not have an inappropriate LIMIT or OFFSET. LIMIT prevents gathering result key + // statistics, which INSERT execution logic depends on. (In QueryKit, LIMIT disables statistics generation and + // funnels everything through a single partition.) + validateLimitAndOffset(rootRel, !hasSegmentGranularity); + + if (hasSegmentGranularity && timeFieldIndex < 0) { + throw new ValidationException( + StringUtils.format( + "INSERT queries with segment granularity other than \"all\" must have a \"%s\" field.", + ColumnHolder.TIME_COLUMN_NAME + ) + ); + } + } + + /** + * SQL allows multiple output columns with the same name, but multi-stage queries doesn't. + */ + private static void validateNoDuplicateAliases(final List> fieldMappings) + throws ValidationException + { + final Set aliasesSeen = new HashSet<>(); + + for (final Pair field : fieldMappings) { + if (!aliasesSeen.add(field.right)) { + throw new ValidationException("Duplicate field in SELECT: " + field.right); + } + } + } + + private static void validateLimitAndOffset(final RelNode topRel, final boolean limitOk) throws ValidationException + { + Sort sort = null; + + if (topRel instanceof Sort) { + sort = (Sort) topRel; + } else if (topRel instanceof Project) { + // Look for Project after a Sort, then validate the sort. + final Project project = (Project) topRel; + if (project.isMapping()) { + final RelNode projectInput = project.getInput(); + if (projectInput instanceof Sort) { + sort = (Sort) projectInput; + } + } + } + + if (sort != null && sort.fetch != null && !limitOk) { + // Found an outer LIMIT that is not allowed. + // The segment generator relies on shuffle statistics to determine segment intervals when PARTITIONED BY is not ALL, + // and LIMIT/OFFSET prevent shuffle statistics from being generated. This is because they always send everything + // to a single partition, so there are no shuffle statistics. + throw new ValidationException( + StringUtils.format( + "INSERT and REPLACE queries cannot have a LIMIT unless %s is \"all\".", + DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY + ) + ); + } + if (sort != null && sort.offset != null) { + // Found an outer OFFSET that is not allowed. + throw new ValidationException("INSERT and REPLACE queries cannot have an OFFSET."); + } + } + + private static RelDataType getMSQStructType(RelDataTypeFactory typeFactory) + { + return typeFactory.createStructType( + ImmutableList.of(Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR)), + TASK_STRUCT_FIELD_NAMES + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java new file mode 100644 index 00000000000..e49fda973a9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java @@ -0,0 +1,336 @@ +/* + * 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.druid.msq.sql; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.CountingOutputStream; +import com.google.inject.Inject; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.druid.common.exception.SanitizableException; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.BadQueryException; +import org.apache.druid.query.QueryCapacityExceededException; +import org.apache.druid.query.QueryException; +import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.QueryTimeoutException; +import org.apache.druid.query.QueryUnsupportedException; +import org.apache.druid.query.ResourceLimitExceededException; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.HttpStatement; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.SqlStatementFactoryFactory; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlQuery; +import org.apache.druid.sql.http.SqlResource; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; +import javax.ws.rs.core.StreamingOutput; +import java.io.IOException; +import java.util.Collections; + +/** + * Endpoint for SQL execution using MSQ tasks. + * + * Unlike the SQL endpoint in {@link SqlResource}, this endpoint returns task IDs instead of inline results. Queries + * are executed asynchronously using MSQ tasks via the indexing service (Overlord + MM or Indexer). This endpoint + * does not provide a way for users to get the status or results of a query. That must be done using Overlord APIs + * for status and reports. + * + * One exception: EXPLAIN query results are returned inline by this endpoint, in the same way as {@link SqlResource} + * would return them. + * + * This endpoint does not support system tables or INFORMATION_SCHEMA. Queries on those tables result in errors. + */ +@Path("/druid/v2/sql/task/") +public class SqlTaskResource +{ + private static final Logger log = new Logger(SqlTaskResource.class); + + private final SqlStatementFactory sqlStatementFactory; + private final ServerConfig serverConfig; + private final AuthorizerMapper authorizerMapper; + private final ObjectMapper jsonMapper; + + @Inject + public SqlTaskResource( + final MSQTaskSqlEngine engine, + final SqlStatementFactoryFactory sqlStatementFactoryFactory, + final ServerConfig serverConfig, + final AuthorizerMapper authorizerMapper, + final ObjectMapper jsonMapper + ) + { + this.sqlStatementFactory = sqlStatementFactoryFactory.factorize(engine); + this.serverConfig = serverConfig; + this.authorizerMapper = authorizerMapper; + this.jsonMapper = jsonMapper; + } + + /** + * API that allows callers to check if this resource is installed without actually issuing a query. If installed, + * this call returns 200 OK. If not installed, callers get 404 Not Found. + */ + @GET + @Path("/enabled") + @Produces(MediaType.APPLICATION_JSON) + public Response doGetEnabled(@Context final HttpServletRequest request) + { + // All authenticated users are authorized for this API: check an empty resource list. + final Access authResult = AuthorizationUtils.authorizeAllResourceActions( + request, + Collections.emptyList(), + authorizerMapper + ); + + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + + return Response.ok(ImmutableMap.of("enabled", true)).build(); + } + + /** + * Post a query task. + * + * Execution uses {@link MSQTaskSqlEngine} to ship the query off to the Overlord as an indexing task using + * {@link org.apache.druid.msq.indexing.MSQControllerTask}. The task ID is returned immediately to the caller, + * and execution proceeds asynchronously. + */ + @POST + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response doPost( + final SqlQuery sqlQuery, + @Context final HttpServletRequest req + ) + { + // Queries run as MSQ tasks look like regular queries, but return the task ID as their only output. + final HttpStatement stmt = sqlStatementFactory.httpStatement(sqlQuery, req); + final String sqlQueryId = stmt.sqlQueryId(); + try { + final DirectStatement.ResultSet plan = stmt.plan(); + final Sequence sequence = plan.run(); + final SqlRowTransformer rowTransformer = plan.createRowTransformer(); + final boolean isTaskStruct = MSQTaskSqlEngine.TASK_STRUCT_FIELD_NAMES.equals(rowTransformer.getFieldList()); + + if (isTaskStruct) { + return buildTaskResponse(sequence); + } else { + // Used for EXPLAIN + return buildStandardResponse(sequence, sqlQuery, sqlQueryId, rowTransformer); + } + } + // Kitchen-sinking the errors since they are all unchecked. + // Just copied from SqlResource. + catch (QueryCapacityExceededException cap) { + stmt.reporter().failed(cap); + return buildNonOkResponse(QueryCapacityExceededException.STATUS_CODE, cap, sqlQueryId); + } + catch (QueryUnsupportedException unsupported) { + stmt.reporter().failed(unsupported); + return buildNonOkResponse(QueryUnsupportedException.STATUS_CODE, unsupported, sqlQueryId); + } + catch (QueryTimeoutException timeout) { + stmt.reporter().failed(timeout); + return buildNonOkResponse(QueryTimeoutException.STATUS_CODE, timeout, sqlQueryId); + } + catch (SqlPlanningException | ResourceLimitExceededException e) { + stmt.reporter().failed(e); + return buildNonOkResponse(BadQueryException.STATUS_CODE, e, sqlQueryId); + } + catch (ForbiddenException e) { + // No request logs for forbidden queries; same as SqlResource + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() + .transformIfNeeded(e); // let ForbiddenExceptionMapper handle this + } + catch (RelOptPlanner.CannotPlanException e) { + stmt.reporter().failed(e); + SqlPlanningException spe = new SqlPlanningException( + SqlPlanningException.PlanningError.UNSUPPORTED_SQL_ERROR, + e.getMessage() + ); + return buildNonOkResponse(BadQueryException.STATUS_CODE, spe, sqlQueryId); + } + // Calcite throws a java.lang.AssertionError which is type Error not Exception. Using Throwable catches both. + catch (Throwable e) { + stmt.reporter().failed(e); + log.noStackTrace().warn(e, "Failed to handle query: %s", sqlQueryId); + + return buildNonOkResponse( + Status.INTERNAL_SERVER_ERROR.getStatusCode(), + QueryInterruptedException.wrapIfNeeded(e), + sqlQueryId + ); + } + finally { + stmt.close(); + } + } + + /** + * Generates a task response using {@link SqlTaskStatus}. + */ + private Response buildTaskResponse(Sequence sequence) throws IOException + { + Yielder yielder = Yielders.each(sequence); + String taskId = null; + + try { + while (!yielder.isDone()) { + final Object[] row = yielder.get(); + if (taskId == null && row.length > 0) { + taskId = (String) row[0]; + } + yielder = yielder.next(null); + } + } + catch (Throwable e) { + try { + yielder.close(); + } + catch (Throwable e2) { + e.addSuppressed(e2); + } + + throw e; + } + + yielder.close(); + + if (taskId == null) { + // Note: no ID to include in error: that is the problem we're reporting. + return genericError( + Response.Status.INTERNAL_SERVER_ERROR, + "Internal error", + "Failed to issue query task", + null + ); + } + + return Response + .status(Response.Status.ACCEPTED) + .entity(new SqlTaskStatus(taskId, TaskState.RUNNING, null)) + .build(); + } + + private Response buildStandardResponse( + Sequence sequence, + SqlQuery sqlQuery, + String sqlQueryId, + SqlRowTransformer rowTransformer + ) throws IOException + { + final Yielder yielder0 = Yielders.each(sequence); + + try { + final Response.ResponseBuilder responseBuilder = Response + .ok( + (StreamingOutput) outputStream -> { + CountingOutputStream os = new CountingOutputStream(outputStream); + Yielder yielder = yielder0; + + try (final ResultFormat.Writer writer = sqlQuery.getResultFormat() + .createFormatter(os, jsonMapper)) { + writer.writeResponseStart(); + + if (sqlQuery.includeHeader()) { + writer.writeHeader( + rowTransformer.getRowType(), + sqlQuery.includeTypesHeader(), + sqlQuery.includeSqlTypesHeader() + ); + } + + while (!yielder.isDone()) { + final Object[] row = yielder.get(); + writer.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value = rowTransformer.transform(row, i); + writer.writeRowField(rowTransformer.getFieldList().get(i), value); + } + writer.writeRowEnd(); + yielder = yielder.next(null); + } + + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to send SQL response [%s]", sqlQueryId); + throw new RuntimeException(e); + } + finally { + yielder.close(); + } + } + ); + + if (sqlQuery.includeHeader()) { + responseBuilder.header(SqlResource.SQL_HEADER_RESPONSE_HEADER, SqlResource.SQL_HEADER_VALUE); + } + + return responseBuilder.build(); + } + catch (Throwable e) { + // make sure to close yielder if anything happened before starting to serialize the response. + yielder0.close(); + throw e; + } + } + + private Response buildNonOkResponse(int status, SanitizableException e, String sqlQueryId) + { + QueryException cleaned = (QueryException) serverConfig + .getErrorResponseTransformStrategy() + .transformIfNeeded(e); + return Response + .status(status) + .entity(new SqlTaskStatus(sqlQueryId, TaskState.FAILED, cleaned)) + .build(); + } + + private Response genericError(Response.Status status, String code, String msg, String id) + { + return Response + .status(status) + .entity(new SqlTaskStatus(id, TaskState.FAILED, new QueryException("FAILED", msg, null, null))) + .build(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java new file mode 100644 index 00000000000..d47533315e2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java @@ -0,0 +1,104 @@ +/* + * 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.druid.msq.sql; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.query.QueryException; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Response object for {@link SqlTaskResource#doPost}. + */ +public class SqlTaskStatus +{ + private final String taskId; + private final TaskState state; + @Nullable + private final QueryException error; + + @JsonCreator + public SqlTaskStatus( + @JsonProperty("taskId") final String taskId, + @JsonProperty("state") final TaskState state, + @JsonProperty("error") @Nullable final QueryException error + ) + { + this.taskId = Preconditions.checkNotNull(taskId, "taskId"); + this.state = Preconditions.checkNotNull(state, "state"); + this.error = error; + } + + @JsonProperty + public String getTaskId() + { + return taskId; + } + + @JsonProperty + public TaskState getState() + { + return state; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public QueryException getError() + { + return error; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SqlTaskStatus response = (SqlTaskStatus) o; + return Objects.equals(taskId, response.taskId) + && state == response.state + && Objects.equals(error, response.error); + } + + @Override + public int hashCode() + { + return Objects.hash(taskId, state, error); + } + + @Override + public String toString() + { + return "SqlTaskStatus{" + + "taskId='" + taskId + '\'' + + ", state=" + state + + ", error=" + error + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollector.java new file mode 100644 index 00000000000..fb8c8232fb8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollector.java @@ -0,0 +1,97 @@ +/* + * 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.druid.msq.statistics; + +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; + +/** + * Collects statistics that help determine how to optimally partition a dataset to achieve a desired {@link ClusterBy}. + * + * Not thread-safe. + */ +public interface ClusterByStatisticsCollector +{ + /** + * Returns the {@link ClusterBy} that this collector uses to bucket and sort keys. + */ + ClusterBy getClusterBy(); + + /** + * Add a key to this collector. + * + * The "weight" parameter must be a positive integer. It should be 1 for "normal" reasonably-sized rows, and a + * larger-than-1-but-still-small integer for "jumbo" rows. This allows {@link #generatePartitionsWithTargetWeight} + * to behave reasonably when passed a row count for the target weight: if all rows are reasonably sized, weight + * is equivalent to rows; however, if rows are jumbo then the generated partition ranges will have fewer rows to + * accommodate the extra weight. + */ + ClusterByStatisticsCollector add(RowKey key, int weight); + + /** + * Add another collector's data to this collector. Does not modify the other collector. + */ + ClusterByStatisticsCollector addAll(ClusterByStatisticsCollector other); + + /** + * Add a snapshot to this collector. + */ + ClusterByStatisticsCollector addAll(ClusterByStatisticsSnapshot other); + + /** + * Estimated total amount of row weight in the dataset, based on what keys have been added so far. + */ + long estimatedTotalWeight(); + + /** + * Whether this collector has encountered any multi-valued input at a particular key position. + * + * This method exists because {@link org.apache.druid.timeline.partition.DimensionRangeShardSpec} does not + * support partitioning on multi-valued strings, so we need to know if any multi-valued strings exist in order + * to decide whether we can use this kind of shard spec. + * + * @throws IllegalArgumentException if keyPosition is outside the range of {@link #getClusterBy()} + * @throws IllegalStateException if this collector was not checking keys for multiple-values + */ + boolean hasMultipleValues(int keyPosition); + + /** + * Removes all data from this collector. + */ + ClusterByStatisticsCollector clear(); + + /** + * Generates key ranges, targeting a particular row weight per range. The actual amount of row weight per range + * may be higher or lower than the provided target. + */ + ClusterByPartitions generatePartitionsWithTargetWeight(long targetWeight); + + /** + * Generates up to "maxNumPartitions" key ranges. The actual number of generated partitions may be less than the + * provided maximum. + */ + ClusterByPartitions generatePartitionsWithMaxCount(int maxNumPartitions); + + /** + * Returns an immutable, JSON-serializable snapshot of this collector. + */ + ClusterByStatisticsSnapshot snapshot(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java new file mode 100644 index 00000000000..02d1036cc16 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java @@ -0,0 +1,440 @@ +/* + * 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.druid.msq.statistics; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.ints.IntRBTreeSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.key.RowKeyReader; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.column.RowSignature; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.stream.Collectors; + +public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsCollector +{ + // Check if this can be done via binary search (accounting for the fuzziness of the datasketches) + // for an objectively faster and more accurate solution instead of finding the best match with the following parameters + private static final int MAX_COUNT_MAX_ITERATIONS = 500; + private static final double MAX_COUNT_ITERATION_GROWTH_FACTOR = 1.05; + + private final ClusterBy clusterBy; + private final RowKeyReader keyReader; + private final KeyCollectorFactory, ? extends KeyCollectorSnapshot> keyCollectorFactory; + private final SortedMap buckets; + private final boolean checkHasMultipleValues; + + private final boolean[] hasMultipleValues; + + // This can be reworked to accommodate maxSize instead of maxRetainedKeys to account for the skewness in the size of hte + // keys depending on the datasource + private final int maxRetainedKeys; + private final int maxBuckets; + private int totalRetainedKeys; + + private ClusterByStatisticsCollectorImpl( + final ClusterBy clusterBy, + final RowKeyReader keyReader, + final KeyCollectorFactory keyCollectorFactory, + final int maxRetainedKeys, + final int maxBuckets, + final boolean checkHasMultipleValues + ) + { + this.clusterBy = clusterBy; + this.keyReader = keyReader; + this.keyCollectorFactory = keyCollectorFactory; + this.maxRetainedKeys = maxRetainedKeys; + this.buckets = new TreeMap<>(clusterBy.bucketComparator()); + this.maxBuckets = maxBuckets; + this.checkHasMultipleValues = checkHasMultipleValues; + this.hasMultipleValues = checkHasMultipleValues ? new boolean[clusterBy.getColumns().size()] : null; + + if (maxBuckets > maxRetainedKeys) { + throw new IAE("maxBuckets[%s] cannot be larger than maxRetainedKeys[%s]", maxBuckets, maxRetainedKeys); + } + } + + public static ClusterByStatisticsCollector create( + final ClusterBy clusterBy, + final RowSignature signature, + final int maxRetainedKeys, + final int maxBuckets, + final boolean aggregate, + final boolean checkHasMultipleValues + ) + { + final RowKeyReader keyReader = clusterBy.keyReader(signature); + final KeyCollectorFactory keyCollectorFactory = KeyCollectors.makeStandardFactory(clusterBy, aggregate); + + return new ClusterByStatisticsCollectorImpl( + clusterBy, + keyReader, + keyCollectorFactory, + maxRetainedKeys, + maxBuckets, + checkHasMultipleValues + ); + } + + @Override + public ClusterBy getClusterBy() + { + return clusterBy; + } + + @Override + public ClusterByStatisticsCollector add(final RowKey key, final int weight) + { + if (checkHasMultipleValues) { + for (int i = 0; i < clusterBy.getColumns().size(); i++) { + hasMultipleValues[i] = hasMultipleValues[i] || keyReader.hasMultipleValues(key, i); + } + } + + final BucketHolder bucketHolder = getOrCreateBucketHolder(keyReader.trim(key, clusterBy.getBucketByCount())); + + bucketHolder.keyCollector.add(key, weight); + + totalRetainedKeys += bucketHolder.updateRetainedKeys(); + if (totalRetainedKeys > maxRetainedKeys) { + downSample(); + } + + return this; + } + + @Override + public ClusterByStatisticsCollector addAll(final ClusterByStatisticsCollector other) + { + if (other instanceof ClusterByStatisticsCollectorImpl) { + ClusterByStatisticsCollectorImpl that = (ClusterByStatisticsCollectorImpl) other; + + // Add all key collectors from the other collector. + for (Map.Entry otherBucketEntry : that.buckets.entrySet()) { + final BucketHolder bucketHolder = getOrCreateBucketHolder(otherBucketEntry.getKey()); + + //noinspection rawtypes, unchecked + ((KeyCollector) bucketHolder.keyCollector).addAll(otherBucketEntry.getValue().keyCollector); + + totalRetainedKeys += bucketHolder.updateRetainedKeys(); + if (totalRetainedKeys > maxRetainedKeys) { + downSample(); + } + } + + if (checkHasMultipleValues) { + for (int i = 0; i < clusterBy.getColumns().size(); i++) { + hasMultipleValues[i] |= that.hasMultipleValues[i]; + } + } + } else { + addAll(other.snapshot()); + } + + return this; + } + + @Override + public ClusterByStatisticsCollector addAll(final ClusterByStatisticsSnapshot snapshot) + { + // Add all key collectors from the other collector. + for (ClusterByStatisticsSnapshot.Bucket otherBucket : snapshot.getBuckets()) { + //noinspection rawtypes, unchecked + final KeyCollector otherKeyCollector = + ((KeyCollectorFactory) keyCollectorFactory).fromSnapshot(otherBucket.getKeyCollectorSnapshot()); + final BucketHolder bucketHolder = getOrCreateBucketHolder(otherBucket.getBucketKey()); + + //noinspection rawtypes, unchecked + ((KeyCollector) bucketHolder.keyCollector).addAll(otherKeyCollector); + + totalRetainedKeys += bucketHolder.updateRetainedKeys(); + if (totalRetainedKeys > maxRetainedKeys) { + downSample(); + } + } + + if (checkHasMultipleValues) { + for (int keyPosition : snapshot.getHasMultipleValues()) { + hasMultipleValues[keyPosition] = true; + } + } + + return this; + } + + @Override + public long estimatedTotalWeight() + { + long count = 0L; + for (final BucketHolder bucketHolder : buckets.values()) { + count += bucketHolder.keyCollector.estimatedTotalWeight(); + } + return count; + } + + @Override + public boolean hasMultipleValues(final int keyPosition) + { + if (checkHasMultipleValues) { + if (keyPosition < 0 || keyPosition >= clusterBy.getColumns().size()) { + throw new IAE("Invalid keyPosition [%d]", keyPosition); + } + + return hasMultipleValues[keyPosition]; + } else { + throw new ISE("hasMultipleValues not available for this collector"); + } + } + + @Override + public ClusterByStatisticsCollector clear() + { + buckets.clear(); + totalRetainedKeys = 0; + return this; + } + + @Override + public ClusterByPartitions generatePartitionsWithTargetWeight(final long targetWeight) + { + if (targetWeight < 1) { + throw new IAE("Target weight must be positive"); + } + + assertRetainedKeyCountsAreTrackedCorrectly(); + + if (buckets.isEmpty()) { + return ClusterByPartitions.oneUniversalPartition(); + } + + final List partitions = new ArrayList<>(); + + for (final BucketHolder bucket : buckets.values()) { + final List bucketPartitions = + bucket.keyCollector.generatePartitionsWithTargetWeight(targetWeight).ranges(); + + if (!partitions.isEmpty() && !bucketPartitions.isEmpty()) { + // Stitch up final partition of previous bucket to match the first partition of this bucket. + partitions.set( + partitions.size() - 1, + new ClusterByPartition( + partitions.get(partitions.size() - 1).getStart(), + bucketPartitions.get(0).getStart() + ) + ); + } + + partitions.addAll(bucketPartitions); + } + + final ClusterByPartitions retVal = new ClusterByPartitions(partitions); + + if (!retVal.allAbutting()) { + // It's a bug if this happens. + throw new ISE("Partitions are not all abutting"); + } + + return retVal; + } + + @Override + public ClusterByPartitions generatePartitionsWithMaxCount(final int maxNumPartitions) + { + if (maxNumPartitions < 1) { + throw new IAE("Must have at least one partition"); + } else if (buckets.isEmpty()) { + return ClusterByPartitions.oneUniversalPartition(); + } else if (maxNumPartitions == 1 && clusterBy.getBucketByCount() == 0) { + return new ClusterByPartitions( + Collections.singletonList( + new ClusterByPartition( + buckets.get(buckets.firstKey()).keyCollector.minKey(), + null + ) + ) + ); + } + + long totalWeight = 0; + + for (final BucketHolder bucketHolder : buckets.values()) { + totalWeight += bucketHolder.keyCollector.estimatedTotalWeight(); + } + + // Gradually increase targetPartitionSize until we get the right number of partitions. + ClusterByPartitions ranges; + long targetPartitionWeight = (long) Math.ceil((double) totalWeight / maxNumPartitions); + int iterations = 0; + + do { + if (iterations++ > MAX_COUNT_MAX_ITERATIONS) { + // Could happen if there are a large number of partition-by keys, or if there are more buckets than + // the max partition count. + throw new ISE("Unable to compute partition ranges"); + } + + ranges = generatePartitionsWithTargetWeight(targetPartitionWeight); + + targetPartitionWeight = (long) Math.ceil(targetPartitionWeight * MAX_COUNT_ITERATION_GROWTH_FACTOR); + } while (ranges.size() > maxNumPartitions); + + return ranges; + } + + @Override + public ClusterByStatisticsSnapshot snapshot() + { + assertRetainedKeyCountsAreTrackedCorrectly(); + + final List bucketSnapshots = new ArrayList<>(); + + for (final Map.Entry bucketEntry : buckets.entrySet()) { + //noinspection rawtypes, unchecked + final KeyCollectorSnapshot keyCollectorSnapshot = + ((KeyCollectorFactory) keyCollectorFactory).toSnapshot(bucketEntry.getValue().keyCollector); + bucketSnapshots.add(new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot)); + } + + final IntSet hasMultipleValuesSet; + + if (checkHasMultipleValues) { + hasMultipleValuesSet = new IntRBTreeSet(); + + for (int i = 0; i < hasMultipleValues.length; i++) { + if (hasMultipleValues[i]) { + hasMultipleValuesSet.add(i); + } + } + } else { + hasMultipleValuesSet = null; + } + + return new ClusterByStatisticsSnapshot(bucketSnapshots, hasMultipleValuesSet); + } + + @VisibleForTesting + List> getKeyCollectors() + { + return buckets.values().stream().map(holder -> holder.keyCollector).collect(Collectors.toList()); + } + + private BucketHolder getOrCreateBucketHolder(final RowKey bucketKey) + { + final BucketHolder existingHolder = buckets.get(Preconditions.checkNotNull(bucketKey, "bucketKey")); + + if (existingHolder != null) { + return existingHolder; + } else if (buckets.size() < maxBuckets) { + final BucketHolder newHolder = new BucketHolder(keyCollectorFactory.newKeyCollector()); + buckets.put(bucketKey, newHolder); + return newHolder; + } else { + throw new TooManyBucketsException(maxBuckets); + } + } + + /** + * Reduce the number of retained keys by about half, if possible. May reduce by less than that, or keep the + * number the same, if downsampling is not possible. (For example: downsampling is not possible if all buckets + * have been downsampled all the way to one key each.) + */ + private void downSample() + { + int newTotalRetainedKeys = totalRetainedKeys; + final int targetTotalRetainedKeys = totalRetainedKeys / 2; + + final List sortedHolders = new ArrayList<>(buckets.size()); + + // Only consider holders with more than one retained key. Holders with a single retained key cannot be downsampled. + for (final BucketHolder holder : buckets.values()) { + if (holder.retainedKeys > 1) { + sortedHolders.add(holder); + } + } + + // Downsample least-dense buckets first. (They're less likely to need high resolution.) + sortedHolders.sort( + Comparator.comparing((BucketHolder holder) -> + (double) holder.keyCollector.estimatedTotalWeight() / holder.retainedKeys) + ); + + int i = 0; + while (i < sortedHolders.size() && newTotalRetainedKeys > targetTotalRetainedKeys) { + final BucketHolder bucketHolder = sortedHolders.get(i); + + // Ignore false return, because we wrap all collectors in DelegateOrMinKeyCollector and can be assured that + // it will downsample all the way to one if needed. Can't do better than that. + bucketHolder.keyCollector.downSample(); + newTotalRetainedKeys += bucketHolder.updateRetainedKeys(); + + if (i == sortedHolders.size() - 1 || sortedHolders.get(i + 1).retainedKeys > bucketHolder.retainedKeys) { + i++; + } + } + + totalRetainedKeys = newTotalRetainedKeys; + } + + private void assertRetainedKeyCountsAreTrackedCorrectly() + { + // Check cached value of retainedKeys in each holder. + assert buckets.values() + .stream() + .allMatch(holder -> holder.retainedKeys == holder.keyCollector.estimatedRetainedKeys()); + + // Check cached value of totalRetainedKeys. + assert totalRetainedKeys == + buckets.values().stream().mapToInt(holder -> holder.keyCollector.estimatedRetainedKeys()).sum(); + } + + private static class BucketHolder + { + private final KeyCollector keyCollector; + private int retainedKeys; + + public BucketHolder(final KeyCollector keyCollector) + { + this.keyCollector = keyCollector; + this.retainedKeys = keyCollector.estimatedRetainedKeys(); + } + + public int updateRetainedKeys() + { + final int newRetainedKeys = keyCollector.estimatedRetainedKeys(); + final int difference = newRetainedKeys - retainedKeys; + retainedKeys = newRetainedKeys; + return difference; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java new file mode 100644 index 00000000000..6b16890254a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -0,0 +1,133 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.frame.key.RowKey; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +public class ClusterByStatisticsSnapshot +{ + private final List buckets; + private final Set hasMultipleValues; + + @JsonCreator + ClusterByStatisticsSnapshot( + @JsonProperty("buckets") final List buckets, + @JsonProperty("hasMultipleValues") @Nullable final Set hasMultipleValues + ) + { + this.buckets = Preconditions.checkNotNull(buckets, "buckets"); + this.hasMultipleValues = hasMultipleValues != null ? hasMultipleValues : Collections.emptySet(); + } + + public static ClusterByStatisticsSnapshot empty() + { + return new ClusterByStatisticsSnapshot(Collections.emptyList(), null); + } + + @JsonProperty("buckets") + List getBuckets() + { + return buckets; + } + + @JsonProperty("hasMultipleValues") + @JsonInclude(JsonInclude.Include.NON_EMPTY) + Set getHasMultipleValues() + { + return hasMultipleValues; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClusterByStatisticsSnapshot that = (ClusterByStatisticsSnapshot) o; + return Objects.equals(buckets, that.buckets) && Objects.equals(hasMultipleValues, that.hasMultipleValues); + } + + @Override + public int hashCode() + { + return Objects.hash(buckets, hasMultipleValues); + } + + static class Bucket + { + private final RowKey bucketKey; + private final KeyCollectorSnapshot keyCollectorSnapshot; + + @JsonCreator + Bucket( + @JsonProperty("bucketKey") RowKey bucketKey, + @JsonProperty("data") KeyCollectorSnapshot keyCollectorSnapshot + ) + { + this.bucketKey = Preconditions.checkNotNull(bucketKey, "bucketKey"); + this.keyCollectorSnapshot = Preconditions.checkNotNull(keyCollectorSnapshot, "data"); + } + + @JsonProperty + public RowKey getBucketKey() + { + return bucketKey; + } + + @JsonProperty("data") + public KeyCollectorSnapshot getKeyCollectorSnapshot() + { + return keyCollectorSnapshot; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Bucket bucket = (Bucket) o; + return Objects.equals(bucketKey, bucket.bucketKey) + && Objects.equals(keyCollectorSnapshot, bucket.keyCollectorSnapshot); + } + + @Override + public int hashCode() + { + return Objects.hash(bucketKey, keyCollectorSnapshot); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollector.java new file mode 100644 index 00000000000..32936e41c20 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollector.java @@ -0,0 +1,170 @@ +/* + * 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.druid.msq.statistics; + +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.Comparator; +import java.util.NoSuchElementException; +import java.util.Optional; + +/** + * Delegates to some other kind of {@link KeyCollector} at first, until its {@link #downSample()} fails to downsample. + * At that point, the delegate collector is nulled out and this collector starts tracking the min key instead. + * + * This is useful because it allows us to wrap any {@link KeyCollector} and enable downsampling to a single key, even + * if the original collector does not support that. For example, {@link QuantilesSketchKeyCollector} cannot downsample + * below K = 2, which retains more than one key. + * + * Created by {@link DelegateOrMinKeyCollectorFactory}. + */ +public class DelegateOrMinKeyCollector> + implements KeyCollector> +{ + private final Comparator comparator; + + // Null means we have been downsampled all the way to a single key. + @Nullable + private TDelegate delegate; + + @Nullable + private RowKey minKey; + + DelegateOrMinKeyCollector( + final Comparator comparator, + @Nullable final TDelegate delegate, + @Nullable final RowKey minKey + ) + { + this.comparator = comparator; + this.delegate = delegate; + this.minKey = minKey; + + if (delegate != null && minKey != null) { + throw new ISE("Cannot have both 'delegate' and 'minKey'"); + } + } + + public Optional getDelegate() + { + return Optional.ofNullable(delegate); + } + + @Override + public void add(RowKey key, long weight) + { + if (delegate != null) { + delegate.add(key, weight); + } else if (minKey == null || comparator.compare(key, minKey) < 0) { + minKey = key; + } + } + + @Override + public void addAll(DelegateOrMinKeyCollector other) + { + if (delegate != null) { + if (other.delegate != null) { + delegate.addAll(other.delegate); + } else if (other.minKey != null) { + delegate.add(other.minKey, 1); + } + } else if (!other.isEmpty()) { + add(other.minKey(), 1); + } + } + + @Override + public boolean isEmpty() + { + if (delegate != null) { + return delegate.isEmpty(); + } else { + return minKey == null; + } + } + + @Override + public long estimatedTotalWeight() + { + if (delegate != null) { + return delegate.estimatedTotalWeight(); + } else { + return minKey != null ? 1 : 0; + } + } + + @Override + public int estimatedRetainedKeys() + { + if (delegate != null) { + return delegate.estimatedRetainedKeys(); + } else { + return minKey != null ? 1 : 0; + } + } + + @Override + public boolean downSample() + { + if (delegate != null && !delegate.downSample()) { + minKey = delegate.isEmpty() ? null : minKey(); + delegate = null; + } + + return true; + } + + @Override + public RowKey minKey() + { + if (delegate != null) { + return delegate.minKey(); + } else if (minKey != null) { + return minKey; + } else { + throw new NoSuchElementException(); + } + } + + /** + * Generates partitions using the delegate if it is valid (i.e., if it has not been discarded due to extreme + * downsampling). If the delegate is not valid, then this method will always return a single partition. + */ + @Override + public ClusterByPartitions generatePartitionsWithTargetWeight(final long targetWeight) + { + if (targetWeight <= 0) { + throw new IAE("targetPartitionWeight must be positive, but was [%d]", targetWeight); + } else if (delegate != null) { + return delegate.generatePartitionsWithTargetWeight(targetWeight); + } else if (minKey != null) { + return new ClusterByPartitions(Collections.singletonList(new ClusterByPartition(minKey, null))); + } else { + return ClusterByPartitions.oneUniversalPartition(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorFactory.java new file mode 100644 index 00000000000..043c5056257 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorFactory.java @@ -0,0 +1,122 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import org.apache.druid.frame.key.RowKey; + +import java.io.IOException; +import java.util.Comparator; +import java.util.Optional; + +/** + * See {@link DelegateOrMinKeyCollector} for details. + */ +public class DelegateOrMinKeyCollectorFactory, TSnapshot extends KeyCollectorSnapshot> + implements KeyCollectorFactory, DelegateOrMinKeyCollectorSnapshot> +{ + private final Comparator comparator; + private final KeyCollectorFactory delegateFactory; + + public DelegateOrMinKeyCollectorFactory( + final Comparator comparator, + final KeyCollectorFactory delegateFactory + ) + { + this.comparator = comparator; + this.delegateFactory = delegateFactory; + } + + @Override + public DelegateOrMinKeyCollector newKeyCollector() + { + return new DelegateOrMinKeyCollector<>(comparator, delegateFactory.newKeyCollector(), null); + } + + @Override + public JsonDeserializer> snapshotDeserializer() + { + final JsonDeserializer delegateDeserializer = delegateFactory.snapshotDeserializer(); + + return new JsonDeserializer>() + { + @Override + public DelegateOrMinKeyCollectorSnapshot deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException + { + TSnapshot delegateSnapshot = null; + RowKey minKey = null; + + if (!jp.isExpectedStartObjectToken()) { + ctxt.reportWrongTokenException(this, JsonToken.START_OBJECT, null); + } + + JsonToken token; + + while ((token = jp.nextToken()) != JsonToken.END_OBJECT) { + if (token != JsonToken.FIELD_NAME) { + ctxt.reportWrongTokenException(this, JsonToken.FIELD_NAME, null); + } + + final String fieldName = jp.getText(); + jp.nextToken(); + + if (DelegateOrMinKeyCollectorSnapshot.FIELD_SNAPSHOT.equals(fieldName)) { + delegateSnapshot = delegateDeserializer.deserialize(jp, ctxt); + } else if (DelegateOrMinKeyCollectorSnapshot.FIELD_MIN_KEY.equals(fieldName)) { + minKey = jp.readValueAs(RowKey.class); + } + } + + return new DelegateOrMinKeyCollectorSnapshot<>(delegateSnapshot, minKey); + } + }; + } + + @Override + public DelegateOrMinKeyCollectorSnapshot toSnapshot(final DelegateOrMinKeyCollector collector) + { + final RowKey minKeyForSnapshot; + + if (!collector.getDelegate().isPresent() && !collector.isEmpty()) { + minKeyForSnapshot = collector.minKey(); + } else { + minKeyForSnapshot = null; + } + + return new DelegateOrMinKeyCollectorSnapshot<>( + collector.getDelegate().map(delegateFactory::toSnapshot).orElse(null), + minKeyForSnapshot + ); + } + + @Override + public DelegateOrMinKeyCollector fromSnapshot(final DelegateOrMinKeyCollectorSnapshot snapshot) + { + return new DelegateOrMinKeyCollector<>( + comparator, + Optional.ofNullable(snapshot.getSnapshot()).map(delegateFactory::fromSnapshot).orElse(null), + snapshot.getMinKey() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java new file mode 100644 index 00000000000..f0191da126b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java @@ -0,0 +1,87 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.java.util.common.ISE; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class DelegateOrMinKeyCollectorSnapshot implements KeyCollectorSnapshot +{ + static final String FIELD_SNAPSHOT = "snapshot"; + static final String FIELD_MIN_KEY = "minKey"; + + private final T snapshot; + private final RowKey minKey; + + @JsonCreator + public DelegateOrMinKeyCollectorSnapshot( + @JsonProperty(FIELD_SNAPSHOT) final T snapshot, + @JsonProperty(FIELD_MIN_KEY) final RowKey minKey + ) + { + this.snapshot = snapshot; + this.minKey = minKey; + + if (snapshot != null && minKey != null) { + throw new ISE("Cannot have both '%s' and '%s'", FIELD_SNAPSHOT, FIELD_MIN_KEY); + } + } + + @Nullable + @JsonProperty(FIELD_SNAPSHOT) + @JsonInclude(JsonInclude.Include.NON_NULL) + public T getSnapshot() + { + return snapshot; + } + + @Nullable + @JsonProperty(FIELD_MIN_KEY) + @JsonInclude(JsonInclude.Include.NON_NULL) + public RowKey getMinKey() + { + return minKey; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DelegateOrMinKeyCollectorSnapshot that = (DelegateOrMinKeyCollectorSnapshot) o; + return Objects.equals(snapshot, that.snapshot) && Objects.equals(minKey, that.minKey); + } + + @Override + public int hashCode() + { + return Objects.hash(snapshot, minKey); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollector.java new file mode 100644 index 00000000000..c27bef375f7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollector.java @@ -0,0 +1,305 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.longs.LongIterator; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongRBTreeMap; +import it.unimi.dsi.fastutil.objects.Object2LongSortedMap; +import it.unimi.dsi.fastutil.objects.ObjectBidirectionalIterator; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +/** + * A key collector that is used when aggregating. It tracks distinct keys. The key must be the aggregation key (i.e. + * the GROUP BY columns.) + */ +public class DistinctKeyCollector implements KeyCollector +{ + static final int INITIAL_MAX_KEYS = 2 << 15 /* 65,536 */; + static final int SMALLEST_MAX_KEYS = 16; + private static final int MISSING_KEY_WEIGHT = 0; + + private final Comparator comparator; + + /** + * Key -> weight first encountered for that key. + * + * Each retained key represents the chunk of keyspace that starts with that key (inclusive) and ends with the + * next key (exclusive). The estimated number of keys in that chunk is pow(2, spaceReductionFactor). + * The weight is treated as the average row weight for the entire keyspace represented by this key. This isn't + * super precise, for two reasons: + * + * (1) nothing guarantees that each key is always added with the same weight; + * (2) nothing guarantees that the weight of a retained key is going to be representative of the nonretained keys + * that it represents. + * + * Item (1) is mitigated by the fact that row weight tends to be correlated with key. (This collector is used + * when aggregating, and when aggregating, a big chunk of the row weight is driven by the aggregation key.) + * + * Item (2) is more likely to be an issue in real life. It is mitigated if the rows with "nearby" keys are likely to have + * similar weight. That happens sometimes, but nothing guarantees it. + * + * The approach here is certainly more fragile than the one in QuantilesSketchKeyCollector, the other major key + * collector type, which is based on a more solid statistical foundation. + */ + private final Object2LongSortedMap retainedKeys; + private int maxKeys; + + /** + * Each key is retained with probability 2^(-spaceReductionFactor). This value is incremented on calls to + * {@link #downSample()}, since it is used to control the size of the {@link #retainedKeys} map as more keys + * are added. + */ + private int spaceReductionFactor; + + // Sum of all values of retainedKeys. + private long totalWeightUnadjusted; + + DistinctKeyCollector( + final Comparator comparator, + final Object2LongSortedMap retainedKeys, + final int spaceReductionFactor + ) + { + this.comparator = Preconditions.checkNotNull(comparator, "comparator"); + this.retainedKeys = Preconditions.checkNotNull(retainedKeys, "retainedKeys"); + this.retainedKeys.defaultReturnValue(MISSING_KEY_WEIGHT); + this.maxKeys = INITIAL_MAX_KEYS; + this.spaceReductionFactor = spaceReductionFactor; + this.totalWeightUnadjusted = 0; + + final LongIterator weightIterator = retainedKeys.values().iterator(); + while (weightIterator.hasNext()) { + totalWeightUnadjusted += weightIterator.nextLong(); + } + } + + DistinctKeyCollector(final Comparator comparator) + { + this(comparator, new Object2LongRBTreeMap<>(comparator), 0); + } + + @Override + public void add(RowKey key, long weight) + { + if (weight <= 0) { + throw new IAE("Weight must be positive"); + } + + final boolean isNewMin = retainedKeys.isEmpty() || comparator.compare(key, retainedKeys.firstKey()) < 0; + + if (isNewMin || isKeySelected(key)) { + if (isNewMin && !retainedKeys.isEmpty() && !isKeySelected(retainedKeys.firstKey())) { + // Old min should be kicked out. + totalWeightUnadjusted -= retainedKeys.removeLong(retainedKeys.firstKey()); + } + + if (retainedKeys.putIfAbsent(key, weight) == MISSING_KEY_WEIGHT) { + // We did add this key. (Previous value was zero, meaning absent.) + totalWeightUnadjusted += weight; + } + + while (retainedKeys.size() >= maxKeys) { + increaseSpaceReductionFactorIfPossible(); + } + } + } + + @Override + public void addAll(DistinctKeyCollector other) + { + while (!retainedKeys.isEmpty() && spaceReductionFactor < other.spaceReductionFactor) { + increaseSpaceReductionFactorIfPossible(); + } + + if (retainedKeys.isEmpty()) { + this.spaceReductionFactor = other.spaceReductionFactor; + } + + for (final Object2LongMap.Entry otherEntry : other.retainedKeys.object2LongEntrySet()) { + add(otherEntry.getKey(), otherEntry.getLongValue()); + } + } + + @Override + public boolean isEmpty() + { + return retainedKeys.isEmpty(); + } + + @Override + public long estimatedTotalWeight() + { + assert totalWeightUnadjusted == retainedKeys.values().longStream().sum(); + return totalWeightUnadjusted << spaceReductionFactor; + } + + @Override + public int estimatedRetainedKeys() + { + return retainedKeys.size(); + } + + @Override + public RowKey minKey() + { + // Throws NoSuchElementException when empty, as required by minKey contract. + return retainedKeys.firstKey(); + } + + @Override + public boolean downSample() + { + if (retainedKeys.size() <= 1) { + return true; + } + + if (maxKeys == SMALLEST_MAX_KEYS) { + return false; + } + + maxKeys /= 2; + + while (retainedKeys.size() >= maxKeys) { + if (!increaseSpaceReductionFactorIfPossible()) { + return false; + } + } + + return true; + } + + @Override + public ClusterByPartitions generatePartitionsWithTargetWeight(final long targetPartitionWeight) + { + if (targetPartitionWeight <= 0) { + throw new IAE("targetPartitionWeight must be positive, but was [%d]", targetPartitionWeight); + } else if (retainedKeys.isEmpty()) { + return ClusterByPartitions.oneUniversalPartition(); + } + + final List partitions = new ArrayList<>(); + final ObjectBidirectionalIterator> iterator = + retainedKeys.object2LongEntrySet().iterator(); + RowKey startKey = retainedKeys.firstKey(); + long partitionWeight = 0; + + while (iterator.hasNext()) { + final Object2LongMap.Entry entry = iterator.next(); + final long keyWeight = entry.getLongValue() << spaceReductionFactor; + final long partitionCountAfterKey = partitionWeight + keyWeight; + + if (partitionWeight > 0 + && partitionCountAfterKey > targetPartitionWeight + && partitionCountAfterKey - targetPartitionWeight > targetPartitionWeight - partitionWeight) { + // New partition *not* including the current key. + partitions.add(new ClusterByPartition(startKey, entry.getKey())); + startKey = entry.getKey(); + partitionWeight = keyWeight; + } else { + // Add to existing partition. + partitionWeight = partitionCountAfterKey; + } + } + + // Add the last partition. + partitions.add(new ClusterByPartition(startKey, null)); + + return new ClusterByPartitions(partitions); + } + + @JsonProperty("keys") + Map getRetainedKeys() + { + return retainedKeys; + } + + @JsonProperty("maxKeys") + int getMaxKeys() + { + return maxKeys; + } + + @JsonProperty("spaceReductionFactor") + int getSpaceReductionFactor() + { + return spaceReductionFactor; + } + + /** + * Returns whether a key would be selected by the current {@link #spaceReductionFactor}. + */ + private boolean isKeySelected(final RowKey key) + { + return spaceReductionFactor == 0 || Long.numberOfTrailingZeros(key.longHashCode()) >= spaceReductionFactor; + } + + /** + * Increment {@link #spaceReductionFactor} and throw away keys from {@link #retainedKeys} as appropriate. + * {@link #retainedKeys} must be nonempty. + * + * Returns true if the space reduction factor was increased, false otherwise. + */ + private boolean increaseSpaceReductionFactorIfPossible() + { + if (spaceReductionFactor == Long.SIZE) { + // This is the biggest possible spaceReductionFactor. It's unlikely to happen unless maxKeys is very low. + return false; + } + + if (retainedKeys.isEmpty()) { + // Incorrect usage by code elsewhere in this class. + throw new ISE("Cannot increase space reduction factor when keys are empty"); + } + + spaceReductionFactor++; + + final ObjectBidirectionalIterator> iterator = + retainedKeys.object2LongEntrySet().iterator(); + + // Never remove the first key. + if (iterator.hasNext()) { + iterator.next(); + } + + while (iterator.hasNext()) { + final Object2LongMap.Entry entry = iterator.next(); + final RowKey key = entry.getKey(); + + if (!isKeySelected(key)) { + totalWeightUnadjusted -= entry.getLongValue(); + iterator.remove(); + } + } + + return true; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java new file mode 100644 index 00000000000..741b5096c77 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java @@ -0,0 +1,87 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import it.unimi.dsi.fastutil.objects.Object2LongRBTreeMap; +import org.apache.druid.collections.SerializablePair; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.RowKey; + +import java.io.IOException; +import java.util.Comparator; +import java.util.stream.Collectors; + +public class DistinctKeyCollectorFactory implements KeyCollectorFactory +{ + private final Comparator comparator; + + private DistinctKeyCollectorFactory(Comparator comparator) + { + this.comparator = comparator; + } + + static DistinctKeyCollectorFactory create(final ClusterBy clusterBy) + { + return new DistinctKeyCollectorFactory(clusterBy.keyComparator()); + } + + @Override + public DistinctKeyCollector newKeyCollector() + { + return new DistinctKeyCollector(comparator); + } + + @Override + public JsonDeserializer snapshotDeserializer() + { + return new JsonDeserializer() + { + @Override + public DistinctKeySnapshot deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException + { + return jp.readValueAs(DistinctKeySnapshot.class); + } + }; + } + + @Override + public DistinctKeySnapshot toSnapshot(final DistinctKeyCollector collector) + { + return new DistinctKeySnapshot( + collector.getRetainedKeys() + .entrySet() + .stream() + .map(entry -> new SerializablePair<>(entry.getKey(), entry.getValue())) + .collect(Collectors.toList()), + collector.getSpaceReductionFactor() + ); + } + + @Override + public DistinctKeyCollector fromSnapshot(final DistinctKeySnapshot snapshot) + { + final Object2LongRBTreeMap retainedKeys = new Object2LongRBTreeMap<>(comparator); + retainedKeys.putAll(snapshot.getKeysAsMap()); + return new DistinctKeyCollector(comparator, retainedKeys, snapshot.getSpaceReductionFactor()); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java new file mode 100644 index 00000000000..44b95d3874f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.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.druid.msq.statistics; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.collections.SerializablePair; +import org.apache.druid.frame.key.RowKey; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class DistinctKeySnapshot implements KeyCollectorSnapshot +{ + private final List> keys; + private final int spaceReductionFactor; + + @JsonCreator + DistinctKeySnapshot( + @JsonProperty("keys") final List> keys, + @JsonProperty("spaceReductionFactor") final int spaceReductionFactor + ) + { + this.keys = Preconditions.checkNotNull(keys, "keys"); + this.spaceReductionFactor = spaceReductionFactor; + } + + @JsonProperty + public List> getKeys() + { + return keys; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public int getSpaceReductionFactor() + { + return spaceReductionFactor; + } + + public Map getKeysAsMap() + { + final Map keysMap = new HashMap<>(); + + for (final SerializablePair key : keys) { + keysMap.put(key.lhs, key.rhs); + } + + return keysMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DistinctKeySnapshot that = (DistinctKeySnapshot) o; + + // Not expected to be called in production, so it's OK that this calls getKeysAsMap() each time. + return spaceReductionFactor == that.spaceReductionFactor && Objects.equals(getKeysAsMap(), that.getKeysAsMap()); + } + + @Override + public int hashCode() + { + // Not expected to be called in production, so it's OK that this calls getKeysAsMap() each time. + return Objects.hash(getKeysAsMap(), spaceReductionFactor); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollector.java new file mode 100644 index 00000000000..1aada32a210 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollector.java @@ -0,0 +1,77 @@ +/* + * 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.druid.msq.statistics; + +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; + +public interface KeyCollector> +{ + /** + * Add a key with a certain weight to this collector. + * + * See {@link ClusterByStatisticsCollector#add} for the meaning of "weight". + */ + void add(RowKey key, long weight); + + /** + * Fold another collector into this one. + */ + void addAll(CollectorType other); + + /** + * Returns whether this collector is empty. + */ + boolean isEmpty(); + + /** + * Returns an estimate of the amount of total weight currently tracked by this collector. This may change over + * time as more keys are added. + */ + long estimatedTotalWeight(); + + /** + * Returns an estimate of the number of keys currently retained by this collector. This may change over time as + * more keys are added. + */ + int estimatedRetainedKeys(); + + /** + * Downsample this collector, dropping about half of the keys that are currently retained. Returns true if + * the collector was downsampled, or if it is already retaining zero or one keys. Returns false if the collector is + * retaining more than one key, yet cannot be downsampled any further. + */ + boolean downSample(); + + /** + * Returns the minimum key encountered by this collector so far, if any have been encountered. + * + * @throws java.util.NoSuchElementException if the collector is empty; i.e. if {@link #isEmpty()} is true. + */ + RowKey minKey(); + + /** + * Generates key ranges, targeting a particular row weight per range. + * + * @param targetWeight row weight per partition. The actual amount of row weight per range may be higher + * or lower than the provided target. + */ + ClusterByPartitions generatePartitionsWithTargetWeight(long targetWeight); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorFactory.java new file mode 100644 index 00000000000..f7956b6dcd6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorFactory.java @@ -0,0 +1,46 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.databind.JsonDeserializer; + +public interface KeyCollectorFactory, TSnapshot extends KeyCollectorSnapshot> +{ + /** + * Create a new {@link KeyCollector} + */ + TCollector newKeyCollector(); + + /** + * Fetches the deserializer that can be used to deserialize the snapshots created by the KeyCollectors corresponding + * to this factory + */ + JsonDeserializer snapshotDeserializer(); + + /** + * Serializes a {@link KeyCollector} to a {@link KeyCollectorSnapshot} + */ + TSnapshot toSnapshot(TCollector collector); + + /** + * Deserializes a {@link KeyCollectorSnapshot} to a {@link KeyCollector} + */ + TCollector fromSnapshot(TSnapshot snapshot); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java new file mode 100644 index 00000000000..ad359cb1e60 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java @@ -0,0 +1,27 @@ +/* + * 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.druid.msq.statistics; + +/** + * Marker interface for deserialization. + */ +public interface KeyCollectorSnapshot +{ +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshotDeserializerModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshotDeserializerModule.java new file mode 100644 index 00000000000..b1aec6296d5 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshotDeserializerModule.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.statistics; + +import com.fasterxml.jackson.databind.module.SimpleModule; + +/** + * A module for deserialization of {@link KeyCollectorSnapshot}. + */ +public class KeyCollectorSnapshotDeserializerModule extends SimpleModule +{ + public KeyCollectorSnapshotDeserializerModule(final KeyCollectorFactory keyCollectorFactory) + { + addDeserializer(KeyCollectorSnapshot.class, keyCollectorFactory.snapshotDeserializer()); + } +} + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.java new file mode 100644 index 00000000000..17490d6f148 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.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.druid.msq.statistics; + +import org.apache.druid.frame.key.ClusterBy; + +public class KeyCollectors +{ + private KeyCollectors() + { + // No instantiation. + } + + /** + * Used by {@link ClusterByStatisticsCollectorImpl#create} and anything else that seeks to have the same behavior. + */ + public static KeyCollectorFactory makeStandardFactory( + final ClusterBy clusterBy, + final boolean aggregate + ) + { + final KeyCollectorFactory baseFactory; + + if (aggregate) { + baseFactory = DistinctKeyCollectorFactory.create(clusterBy); + } else { + baseFactory = QuantilesSketchKeyCollectorFactory.create(clusterBy); + } + + // Wrap in DelegateOrMinKeyCollectorFactory, so we are guaranteed to be able to downsample to a single key. This + // is important because it allows us to better handle large numbers of small buckets. + return new DelegateOrMinKeyCollectorFactory<>(clusterBy.keyComparator(), baseFactory); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java new file mode 100644 index 00000000000..99fb8a23e86 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java @@ -0,0 +1,168 @@ +/* + * 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.druid.msq.statistics; + +import com.google.common.math.LongMath; +import com.google.common.primitives.Ints; +import org.apache.datasketches.quantiles.ItemsSketch; +import org.apache.datasketches.quantiles.ItemsUnion; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.java.util.common.IAE; + +import javax.annotation.Nullable; +import java.math.RoundingMode; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * A key collector that is used when not aggregating. It uses a quantiles sketch to track keys. + */ +public class QuantilesSketchKeyCollector implements KeyCollector +{ + private final Comparator comparator; + private ItemsSketch sketch; + + QuantilesSketchKeyCollector( + final Comparator comparator, + @Nullable final ItemsSketch sketch + ) + { + this.comparator = comparator; + this.sketch = sketch; + } + + @Override + public void add(RowKey key, long weight) + { + for (int i = 0; i < weight; i++) { + // Add the same key multiple times to make it "heavier". + sketch.update(key); + } + } + + @Override + public void addAll(QuantilesSketchKeyCollector other) + { + final ItemsUnion union = ItemsUnion.getInstance( + Math.max(sketch.getK(), other.sketch.getK()), + comparator + ); + + union.update(sketch); + union.update(other.sketch); + sketch = union.getResultAndReset(); + } + + @Override + public boolean isEmpty() + { + return sketch.isEmpty(); + } + + @Override + public long estimatedTotalWeight() + { + return sketch.getN(); + } + + @Override + public int estimatedRetainedKeys() + { + // Rough estimation of retained keys for a given K for ~billions of total items, based on the table from + // https://datasketches.apache.org/docs/Quantiles/OrigQuantilesSketch.html. + final int estimatedMaxRetainedKeys = 11 * sketch.getK(); + + // Cast to int is safe because estimatedMaxRetainedKeys is always within int range. + return (int) Math.min(sketch.getN(), estimatedMaxRetainedKeys); + } + + @Override + public boolean downSample() + { + if (sketch.getN() <= 1) { + return true; + } else if (sketch.getK() == 2) { + return false; + } else { + sketch = sketch.downSample(sketch.getK() / 2); + return true; + } + } + + @Override + public RowKey minKey() + { + final RowKey minValue = sketch.getMinValue(); + + if (minValue != null) { + return minValue; + } else { + throw new NoSuchElementException(); + } + } + + @Override + public ClusterByPartitions generatePartitionsWithTargetWeight(final long targetWeight) + { + if (targetWeight <= 0) { + throw new IAE("targetPartitionWeight must be positive, but was [%d]", targetWeight); + } + + if (sketch.getN() == 0) { + return ClusterByPartitions.oneUniversalPartition(); + } + + final int numPartitions = Ints.checkedCast(LongMath.divide(sketch.getN(), targetWeight, RoundingMode.CEILING)); + + // numPartitions + 1, because the final quantile is the max, and we won't build a partition based on that. + final RowKey[] quantiles = sketch.getQuantiles(numPartitions + 1); + final List partitions = new ArrayList<>(); + + for (int i = 0; i < numPartitions; i++) { + final boolean isFinalPartition = i == numPartitions - 1; + + if (isFinalPartition) { + partitions.add(new ClusterByPartition(quantiles[i], null)); + } else { + final ClusterByPartition partition = new ClusterByPartition(quantiles[i], quantiles[i + 1]); + final int cmp = comparator.compare(partition.getStart(), partition.getEnd()); + if (cmp < 0) { + // Skip partitions where start == end. + // I don't think start can be greater than end, but if that happens, skip them too! + partitions.add(partition); + } + } + } + + return new ClusterByPartitions(partitions); + } + + /** + * Retrieves the backing sketch. Exists for usage by {@link QuantilesSketchKeyCollectorFactory}. + */ + ItemsSketch getSketch() + { + return sketch; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java new file mode 100644 index 00000000000..613a7dc4970 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java @@ -0,0 +1,149 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.google.common.annotations.VisibleForTesting; +import org.apache.datasketches.ArrayOfItemsSerDe; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.quantiles.ItemsSketch; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.java.util.common.StringUtils; + +import java.io.IOException; +import java.nio.ByteOrder; +import java.util.Comparator; + +public class QuantilesSketchKeyCollectorFactory + implements KeyCollectorFactory +{ + // smallest value with normalized rank error < 0.1%; retain up to ~86k elements + @VisibleForTesting + static final int SKETCH_INITIAL_K = 1 << 12; + + private final Comparator comparator; + + private QuantilesSketchKeyCollectorFactory(final Comparator comparator) + { + this.comparator = comparator; + } + + static QuantilesSketchKeyCollectorFactory create(final ClusterBy clusterBy) + { + return new QuantilesSketchKeyCollectorFactory(clusterBy.keyComparator()); + } + + @Override + public QuantilesSketchKeyCollector newKeyCollector() + { + return new QuantilesSketchKeyCollector(comparator, ItemsSketch.getInstance(SKETCH_INITIAL_K, comparator)); + } + + @Override + public JsonDeserializer snapshotDeserializer() + { + return new JsonDeserializer() + { + @Override + public QuantilesSketchKeyCollectorSnapshot deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException + { + return jp.readValueAs(QuantilesSketchKeyCollectorSnapshot.class); + } + }; + } + + @Override + public QuantilesSketchKeyCollectorSnapshot toSnapshot(QuantilesSketchKeyCollector collector) + { + final String encodedSketch = + StringUtils.encodeBase64String(collector.getSketch().toByteArray(RowKeySerde.INSTANCE)); + return new QuantilesSketchKeyCollectorSnapshot(encodedSketch); + } + + @Override + public QuantilesSketchKeyCollector fromSnapshot(QuantilesSketchKeyCollectorSnapshot snapshot) + { + final String encodedSketch = snapshot.getEncodedSketch(); + final byte[] bytes = StringUtils.decodeBase64String(encodedSketch); + final ItemsSketch sketch = + ItemsSketch.getInstance(Memory.wrap(bytes), comparator, RowKeySerde.INSTANCE); + return new QuantilesSketchKeyCollector(comparator, sketch); + } + + private static class RowKeySerde extends ArrayOfItemsSerDe + { + private static final RowKeySerde INSTANCE = new RowKeySerde(); + + private RowKeySerde() + { + } + + @Override + public byte[] serializeToByteArray(final RowKey[] items) + { + int serializedSize = Integer.BYTES * items.length; + + for (final RowKey key : items) { + serializedSize += key.array().length; + } + + final byte[] serializedBytes = new byte[serializedSize]; + final WritableMemory writableMemory = WritableMemory.writableWrap(serializedBytes, ByteOrder.LITTLE_ENDIAN); + long keyWritePosition = (long) Integer.BYTES * items.length; + + for (int i = 0; i < items.length; i++) { + final RowKey key = items[i]; + final byte[] keyBytes = key.array(); + + writableMemory.putInt((long) Integer.BYTES * i, keyBytes.length); + writableMemory.putByteArray(keyWritePosition, keyBytes, 0, keyBytes.length); + + keyWritePosition += keyBytes.length; + } + + assert keyWritePosition == serializedSize; + return serializedBytes; + } + + @Override + public RowKey[] deserializeFromMemory(final Memory mem, final int numItems) + { + final RowKey[] keys = new RowKey[numItems]; + long keyPosition = (long) Integer.BYTES * numItems; + + for (int i = 0; i < numItems; i++) { + final int keyLength = mem.getInt((long) Integer.BYTES * i); + final byte[] keyBytes = new byte[keyLength]; + + mem.getByteArray(keyPosition, keyBytes, 0, keyLength); + keys[i] = RowKey.wrap(keyBytes); + + keyPosition += keyLength; + } + + return keys; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java new file mode 100644 index 00000000000..4e9fce437f0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java @@ -0,0 +1,61 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +import java.util.Objects; + +public class QuantilesSketchKeyCollectorSnapshot implements KeyCollectorSnapshot +{ + private final String encodedSketch; + + @JsonCreator + public QuantilesSketchKeyCollectorSnapshot(String encodedSketch) + { + this.encodedSketch = encodedSketch; + } + + @JsonValue + public String getEncodedSketch() + { + return encodedSketch; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + QuantilesSketchKeyCollectorSnapshot that = (QuantilesSketchKeyCollectorSnapshot) o; + return Objects.equals(encodedSketch, that.encodedSketch); + } + + @Override + public int hashCode() + { + return Objects.hash(encodedSketch); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/TooManyBucketsException.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/TooManyBucketsException.java new file mode 100644 index 00000000000..1c83a8276bb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/TooManyBucketsException.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.druid.msq.statistics; + +import org.apache.druid.java.util.common.StringUtils; + +public class TooManyBucketsException extends RuntimeException +{ + private final int maxBuckets; + + public TooManyBucketsException(final int maxBuckets) + { + super(StringUtils.format("Too many buckets; maximum is [%s]", maxBuckets)); + this.maxBuckets = maxBuckets; + } + + public int getMaxBuckets() + { + return maxBuckets; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DecoratedExecutorService.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DecoratedExecutorService.java new file mode 100644 index 00000000000..68095f6dab6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DecoratedExecutorService.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.druid.msq.util; + +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * A {@link ListeningExecutorService} where all tasks are automatically decorated before being submitted to a + * delegate executor service. + */ +public class DecoratedExecutorService implements ListeningExecutorService +{ + private final ListeningExecutorService exec; + private final Decorator decorator; + + public DecoratedExecutorService( + final ListeningExecutorService exec, + final Decorator decorator + ) + { + this.exec = exec; + this.decorator = decorator; + } + + @Override + public ListenableFuture submit(Callable task) + { + return exec.submit(decorator.decorateCallable(task)); + } + + @Override + public ListenableFuture submit(Runnable task) + { + return exec.submit(decorator.decorateRunnable(task)); + } + + @Override + public ListenableFuture submit(Runnable task, T result) + { + return exec.submit(decorator.decorateRunnable(task), result); + } + + @Override + public List> invokeAll(Collection> tasks) throws InterruptedException + { + final List> decoratedTasks = new ArrayList<>(); + + for (final Callable task : tasks) { + decoratedTasks.add(decorator.decorateCallable(task)); + } + + return exec.invokeAll(decoratedTasks); + } + + @Override + public List> invokeAll(Collection> tasks, long timeout, TimeUnit unit) + throws InterruptedException + { + final List> decoratedTasks = new ArrayList<>(); + + for (final Callable task : tasks) { + decoratedTasks.add(decorator.decorateCallable(task)); + } + + return exec.invokeAll(decoratedTasks, timeout, unit); + } + + @Override + public void shutdown() + { + exec.shutdown(); + } + + @Override + public List shutdownNow() + { + return exec.shutdownNow(); + } + + @Override + public boolean isShutdown() + { + return exec.isShutdown(); + } + + @Override + public boolean isTerminated() + { + return exec.isTerminated(); + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException + { + return exec.awaitTermination(timeout, unit); + } + + @Override + public T invokeAny(Collection> tasks) throws InterruptedException, ExecutionException + { + final List> decoratedTasks = new ArrayList<>(); + + for (final Callable task : tasks) { + decoratedTasks.add(decorator.decorateCallable(task)); + } + + return exec.invokeAny(decoratedTasks); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException + { + final List> decoratedTasks = new ArrayList<>(); + + for (final Callable task : tasks) { + decoratedTasks.add(decorator.decorateCallable(task)); + } + + return exec.invokeAny(decoratedTasks, timeout, unit); + } + + @Override + public void execute(Runnable command) + { + exec.execute(decorator.decorateRunnable(command)); + } + + public interface Decorator + { + Callable decorateCallable(Callable callable); + + Runnable decorateRunnable(Runnable runnable); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java new file mode 100644 index 00000000000..73e2ff04880 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java @@ -0,0 +1,70 @@ +/* + * 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.druid.msq.util; + +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DoubleDimensionSchema; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; + +/** + * Dimension-schema-related utility functions that would make sense in {@link DimensionSchema} if this + * were not an extension. + */ +public class DimensionSchemaUtils +{ + public static DimensionSchema createDimensionSchema(final String column, @Nullable final ColumnType type) + { + // if schema information not available, create a string dimension + if (type == null) { + return new StringDimensionSchema(column); + } + + switch (type.getType()) { + case STRING: + return new StringDimensionSchema(column); + case LONG: + return new LongDimensionSchema(column); + case FLOAT: + return new FloatDimensionSchema(column); + case DOUBLE: + return new DoubleDimensionSchema(column); + case ARRAY: + switch (type.getElementType().getType()) { + case STRING: + return new StringDimensionSchema(column, DimensionSchema.MultiValueHandling.ARRAY, null); + default: + throw new ISE("Cannot create dimension for type [%s]", type.toString()); + } + default: + final ColumnCapabilities capabilities = ColumnCapabilitiesImpl.createDefault().setType(type); + return DimensionHandlerUtils.getHandlerFromCapabilities(column, capabilities, null) + .getDimensionSchema(capabilities); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.java new file mode 100644 index 00000000000..43a844b5a6f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.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.druid.msq.util; + +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.List; + +/** + * Things that would make sense in {@link org.apache.druid.java.util.common.Intervals} if this were not an extension. + */ +public class IntervalUtils +{ + public static List difference(final List list1, final List list2) + { + final List retVal = new ArrayList<>(); + + int i = 0, j = 0; + while (i < list1.size()) { + while (j < list2.size() && list2.get(j).isBefore(list1.get(i))) { + j++; + } + + if (j == list2.size() || list2.get(j).isAfter(list1.get(i))) { + retVal.add(list1.get(i)); + i++; + } else { + final Interval overlap = list1.get(i).overlap(list2.get(j)); + final Interval a = new Interval(list1.get(i).getStart(), overlap.getStart()); + final Interval b = new Interval(overlap.getEnd(), list1.get(i).getEnd()); + + if (a.toDurationMillis() > 0) { + retVal.add(a); + } + + if (b.toDurationMillis() > 0) { + list1.set(i, b); + } else { + i++; + } + } + } + + return retVal; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQFutureUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQFutureUtils.java new file mode 100644 index 00000000000..03013f08a06 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQFutureUtils.java @@ -0,0 +1,70 @@ +/* + * 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.druid.msq.util; + +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * Future-related utility functions that haven't been moved to {@link FutureUtils}. + */ +public class MSQFutureUtils +{ + /** + * Similar to {@link Futures#allAsList}, but provides a "cancelOnErrorOrInterrupt" option that cancels all input + * futures if the returned future is canceled or fails. + */ + public static ListenableFuture> allAsList( + final Iterable> futures, + final boolean cancelOnErrorOrInterrupt + ) + { + final ListenableFuture> retVal = Futures.allAsList(futures); + + if (cancelOnErrorOrInterrupt) { + Futures.addCallback( + retVal, + new FutureCallback>() + { + @Override + public void onSuccess(@Nullable List result) + { + // Do nothing. + } + + @Override + public void onFailure(Throwable t) + { + for (final ListenableFuture inputFuture : futures) { + inputFuture.cancel(true); + } + } + } + ); + } + + return retVal; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java new file mode 100644 index 00000000000..7a6b576e68d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -0,0 +1,237 @@ +/* + * 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.druid.msq.util; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.opencsv.RFC4180Parser; +import com.opencsv.RFC4180ParserBuilder; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Numbers; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.sql.MSQMode; +import org.apache.druid.query.QueryContext; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** + * Class for all MSQ context params + */ +public class MultiStageQueryContext +{ + public static final String CTX_MSQ_MODE = "mode"; + public static final String DEFAULT_MSQ_MODE = MSQMode.STRICT_MODE.toString(); + + public static final String CTX_MAX_NUM_TASKS = "maxNumTasks"; + @VisibleForTesting + static final int DEFAULT_MAX_NUM_TASKS = 2; + + public static final String CTX_TASK_ASSIGNMENT_STRATEGY = "taskAssignment"; + private static final String DEFAULT_TASK_ASSIGNMENT_STRATEGY = WorkerAssignmentStrategy.MAX.toString(); + + public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; + private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; + + public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; + private static final String DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = "false"; + + public static final String CTX_DESTINATION = "destination"; + private static final String DEFAULT_DESTINATION = null; + + public static final String CTX_ROWS_PER_SEGMENT = "rowsPerSegment"; + + public static final String CTX_ROWS_IN_MEMORY = "rowsInMemory"; + + /** + * Controls sort order within segments. Normally, this is the same as the overall order of the query (from the + * CLUSTERED BY clause) but it can be overridden. + */ + public static final String CTX_SORT_ORDER = "segmentSortOrder"; + private static final String DEFAULT_SORT_ORDER = null; + + private static final Pattern LOOKS_LIKE_JSON_ARRAY = Pattern.compile("^\\s*\\[.*", Pattern.DOTALL); + + public static String getMSQMode(QueryContext queryContext) + { + return (String) MultiStageQueryContext.getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_MSQ_MODE, + null, + DEFAULT_MSQ_MODE + ); + } + + public static boolean isDurableStorageEnabled(Map propertyMap) + { + return Boolean.parseBoolean( + String.valueOf( + getValueFromPropertyMap( + propertyMap, + CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, + null, + DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE + ) + ) + ); + } + + public static boolean isFinalizeAggregations(final QueryContext queryContext) + { + return Numbers.parseBoolean( + getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_FINALIZE_AGGREGATIONS, + null, + DEFAULT_FINALIZE_AGGREGATIONS + ) + ); + } + + public static WorkerAssignmentStrategy getAssignmentStrategy(final QueryContext queryContext) + { + String assignmentStrategyString = (String) getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_TASK_ASSIGNMENT_STRATEGY, + null, + DEFAULT_TASK_ASSIGNMENT_STRATEGY + ); + + return WorkerAssignmentStrategy.fromString(assignmentStrategyString); + } + + public static int getMaxNumTasks(final QueryContext queryContext) + { + return Numbers.parseInt( + getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_MAX_NUM_TASKS, + null, + DEFAULT_MAX_NUM_TASKS + ) + ); + } + + public static Object getDestination(final QueryContext queryContext) + { + return getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_DESTINATION, + null, + DEFAULT_DESTINATION + ); + } + + public static int getRowsPerSegment(final QueryContext queryContext, int defaultRowsPerSegment) + { + return Numbers.parseInt( + getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_ROWS_PER_SEGMENT, + null, + defaultRowsPerSegment + ) + ); + } + + public static int getRowsInMemory(final QueryContext queryContext, int defaultRowsInMemory) + { + return Numbers.parseInt( + getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_ROWS_IN_MEMORY, + null, + defaultRowsInMemory + ) + ); + } + + @Nullable + public static Object getValueFromPropertyMap( + Map propertyMap, + String key, + @Nullable List aliases, + @Nullable Object defaultValue + ) + { + if (propertyMap.get(key) != null) { + return propertyMap.get(key); + } + + if (aliases != null) { + for (String legacyKey : aliases) { + if (propertyMap.get(legacyKey) != null) { + return propertyMap.get(legacyKey); + } + } + } + + return defaultValue; + } + + public static String getSortOrder(final QueryContext queryContext) + { + return (String) getValueFromPropertyMap( + queryContext.getMergedParams(), + CTX_SORT_ORDER, + null, + DEFAULT_SORT_ORDER + ); + } + + /** + * Decodes {@link #CTX_SORT_ORDER} from either a JSON or CSV string. + */ + public static List decodeSortOrder(@Nullable final String sortOrderString) + { + if (sortOrderString == null) { + return Collections.emptyList(); + } else if (LOOKS_LIKE_JSON_ARRAY.matcher(sortOrderString).matches()) { + try { + // Not caching this ObjectMapper in a static, because we expect to use it infrequently (once per INSERT + // query that uses this feature) and there is no need to keep it around longer than that. + return new ObjectMapper().readValue(sortOrderString, new TypeReference>() {}); + } + catch (JsonProcessingException e) { + throw new IAE("Invalid JSON provided for [%s]", CTX_SORT_ORDER); + } + } else { + final RFC4180Parser csvParser = new RFC4180ParserBuilder().withSeparator(',').build(); + + try { + return Arrays.stream(csvParser.parseLine(sortOrderString)) + .filter(s -> s != null && !s.isEmpty()) + .map(String::trim) + .collect(Collectors.toList()); + } + catch (IOException e) { + throw new IAE("Invalid CSV provided for [%s]", CTX_SORT_ORDER); + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregator.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregator.java new file mode 100644 index 00000000000..2dde7ab6ce8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregator.java @@ -0,0 +1,78 @@ +/* + * 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.druid.msq.util; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.segment.BaseObjectColumnValueSelector; + +import javax.annotation.Nullable; + +public class PassthroughAggregator implements Aggregator +{ + private final BaseObjectColumnValueSelector selector; + private boolean didSet = false; + private Object val; + + public PassthroughAggregator(final BaseObjectColumnValueSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate() + { + if (didSet) { + throw new ISE("Cannot set twice"); + } + + val = selector.getObject(); + didSet = true; + } + + @Nullable + @Override + public Object get() + { + if (!didSet) { + throw new ISE("Cannot call get() before aggregate()"); + } + + return val; + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + // Nothing to do. + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java new file mode 100644 index 00000000000..4acf50cfb7a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java @@ -0,0 +1,208 @@ +/* + * 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.druid.msq.util; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.annotations.EverythingIsNonnullByDefault; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + * Hack that allows "passing through" arbitrary complex types into + * {@link org.apache.druid.segment.incremental.IncrementalIndex}. + * + * Used by {@link org.apache.druid.msq.exec.ControllerImpl#makeDimensionsAndAggregatorsForIngestion}. + * + * To move away from this, it would need to be possible to create complex columns in segments only knowing the complex + * type; in particular, without knowing the type of an aggregator factory or dimension schema that corresponds to + * the complex type. + */ +@JsonTypeName(PassthroughAggregatorFactory.TYPE) +@EverythingIsNonnullByDefault +public class PassthroughAggregatorFactory extends AggregatorFactory +{ + private static final int ESTIMATED_HEAP_FOOTPRINT = 800; + + static final String TYPE = "passthrough"; + + private final String columnName; + private final String complexTypeName; + + @JsonCreator + public PassthroughAggregatorFactory( + @JsonProperty("columnName") String columnName, + @JsonProperty("complexTypeName") String complexTypeName + ) + { + this.columnName = columnName; + this.complexTypeName = complexTypeName; + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + @Override + @JsonProperty + public String getComplexTypeName() + { + return complexTypeName; + } + + @Override + public byte[] getCacheKey() + { + throw new UnsupportedOperationException(); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new PassthroughAggregator(metricFactory.makeColumnValueSelector(columnName)); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + throw new UnsupportedOperationException(); + } + + @Override + @SuppressWarnings("rawtypes") + public Comparator getComparator() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Object combine(@Nullable Object lhs, @Nullable Object rhs) + { + throw new UnsupportedOperationException(); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return this; + } + + @Override + public List getRequiredColumns() + { + throw new UnsupportedOperationException(); + } + + @Override + public Object deserialize(Object object) + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Object finalizeComputation(@Nullable Object object) + { + throw new UnsupportedOperationException(); + } + + @Override + public String getName() + { + return columnName; + } + + @Override + public List requiredFields() + { + return Collections.singletonList(columnName); + } + + @Override + public int guessAggregatorHeapFootprint(long rows) + { + return ESTIMATED_HEAP_FOOTPRINT; + } + + @Override + public AggregatorFactory withName(String newName) + { + return new PassthroughAggregatorFactory(newName, complexTypeName); + } + + @Override + public int getMaxIntermediateSize() + { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnType getIntermediateType() + { + return ColumnType.ofComplex(complexTypeName); + } + + @Override + public ColumnType getResultType() + { + return ColumnType.ofComplex(complexTypeName); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PassthroughAggregatorFactory that = (PassthroughAggregatorFactory) o; + return columnName.equals(that.columnName) && complexTypeName.equals(that.complexTypeName); + } + + @Override + public int hashCode() + { + return Objects.hash(columnName, complexTypeName); + } + + @Override + public String toString() + { + return "PassthroughAggregatorFactory{" + + "columnName='" + columnName + '\'' + + ", complexTypeName='" + complexTypeName + '\'' + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SequenceUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SequenceUtils.java new file mode 100644 index 00000000000..ffe4f4fa6d1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SequenceUtils.java @@ -0,0 +1,44 @@ +/* + * 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.druid.msq.util; + +import org.apache.druid.java.util.common.guava.Sequence; + +import java.util.function.Consumer; + +/** + * Sequence-related utility functions that would make sense in {@link Sequence} if this were not an extension. + */ +public class SequenceUtils +{ + /** + * Executes "action" for each element of "sequence". + */ + public static void forEach(Sequence sequence, Consumer action) + { + sequence.accumulate( + null, + (accumulated, in) -> { + action.accept(in); + return null; + } + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SupplierIterator.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SupplierIterator.java new file mode 100644 index 00000000000..f3ddb82f999 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SupplierIterator.java @@ -0,0 +1,57 @@ +/* + * 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.druid.msq.util; + +import com.google.common.base.Preconditions; + +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.Supplier; + +/** + * An Iterator that returns a single element from a {@link Supplier}. + */ +public class SupplierIterator implements Iterator +{ + private Supplier supplier; + + public SupplierIterator(final Supplier supplier) + { + this.supplier = Preconditions.checkNotNull(supplier, "supplier"); + } + + @Override + public boolean hasNext() + { + return supplier != null; + } + + @Override + public T next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final T thing = supplier.get(); + supplier = null; + return thing; + } +} diff --git a/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 00000000000..647356f9da6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1,21 @@ +# 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. + +org.apache.druid.msq.guice.MSQExternalDataSourceModule +org.apache.druid.msq.guice.MSQIndexingModule +org.apache.druid.msq.guice.MSQDurableStorageModule +org.apache.druid.msq.guice.MSQServiceClientModule +org.apache.druid.msq.guice.MSQSqlModule +org.apache.druid.msq.guice.SqlTaskModule diff --git a/extensions-core/multi-stage-query/src/main/resources/log4j2.xml b/extensions-core/multi-stage-query/src/main/resources/log4j2.xml new file mode 100644 index 00000000000..e99abd74336 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/resources/log4j2.xml @@ -0,0 +1,38 @@ + + + + + + + + + + + + + + + + + + + + + diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/TestArrayStorageAdapter.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/TestArrayStorageAdapter.java new file mode 100644 index 00000000000..78113827d91 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/TestArrayStorageAdapter.java @@ -0,0 +1,56 @@ +/* + * 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.druid.msq; + +import com.google.common.collect.Iterables; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; + +import java.util.Optional; + +public class TestArrayStorageAdapter extends QueryableIndexStorageAdapter +{ + public TestArrayStorageAdapter(QueryableIndex index) + { + super(index); + } + + @Override + public RowSignature getRowSignature() + { + final RowSignature.Builder builder = RowSignature.builder(); + builder.addTimeColumn(); + + for (final String column : Iterables.concat(getAvailableDimensions(), getAvailableMetrics())) { + Optional columnCapabilities = Optional.ofNullable(getColumnCapabilities(column)); + ColumnType columnType = columnCapabilities.isPresent() ? columnCapabilities.get().toColumnType() : null; + //change MV columns to Array + if (columnCapabilities.isPresent() && columnCapabilities.get().hasMultipleValues().isMaybeTrue()) { + columnType = ColumnType.STRING_ARRAY; + } + builder.add(column, columnType); + } + + return builder.build(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/counters/CountersSnapshotTreeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/counters/CountersSnapshotTreeTest.java new file mode 100644 index 00000000000..a6b0e830a20 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/counters/CountersSnapshotTreeTest.java @@ -0,0 +1,49 @@ +/* + * 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.druid.msq.counters; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class CountersSnapshotTreeTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = + TestHelper.makeJsonMapper().registerModules(new MSQIndexingModule().getJacksonModules()); + + final ChannelCounters channelCounters = new ChannelCounters(); + channelCounters.addFile(10, 13); + channelCounters.setTotalFiles(14); + + final CounterSnapshotsTree snapshotsTree = new CounterSnapshotsTree(); + snapshotsTree.put(1, 2, new CounterSnapshots(ImmutableMap.of("ctr", channelCounters.snapshot()))); + + final String json = mapper.writeValueAsString(snapshotsTree); + final CounterSnapshotsTree snapshotsTree2 = mapper.readValue(json, CounterSnapshotsTree.class); + + Assert.assertEquals(snapshotsTree.copyMap(), snapshotsTree2.copyMap()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java new file mode 100644 index 00000000000..a1934f070a3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -0,0 +1,661 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import org.apache.druid.hll.HyperLogLogCollector; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.msq.indexing.error.ColumnNameRestrictedFault; +import org.apache.druid.msq.indexing.error.InsertTimeNullFault; +import org.apache.druid.msq.indexing.error.RowTooLargeFault; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.timeline.SegmentId; +import org.hamcrest.CoreMatchers; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.mockito.Mockito; + +import javax.annotation.Nonnull; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +public class MSQInsertTest extends MSQTestBase +{ + private final HashFunction fn = Hashing.murmur3_128(); + + @Test + public void testInsertOnFoo1() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + testIngestQuery().setSql( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(expectedFooSegments()) + .setExpectedResultRows(expectedFooRows()) + .verifyResults(); + + } + + @Test + public void testInsertOnExternalDataSource() throws IOException + { + final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); + final String toReadFileNameAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("cnt", ColumnType.LONG) + .build(); + + testIngestQuery().setSql(" insert into foo1 SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1 PARTITIONED by day ") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo1", + Intervals.of("2016-06-27/P1D"), + "test", + 0 + ))) + .setExpectedResultRows(ImmutableList.of(new Object[]{1466985600000L, 20L})) + .verifyResults(); + + } + + @Test + public void testInsertOnFoo1WithTimeFunction() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + testIngestQuery().setSql( + "insert into foo1 select floor(__time to day) as __time , dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(expectedFooSegments()) + .setExpectedResultRows(expectedFooRows()) + .verifyResults(); + + } + + @Test + public void testInsertOnFoo1WithMultiValueDim() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim3", ColumnType.STRING).build(); + + testIngestQuery().setSql( + "INSERT INTO foo1 SELECT dim3 FROM foo WHERE dim3 IS NOT NULL PARTITIONED BY ALL TIME") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedResultRows(expectedMultiValueFooRows()) + .verifyResults(); + } + + @Test + public void testInsertOnFoo1WithMultiValueDimGroupBy() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim3", ColumnType.STRING).build(); + + testIngestQuery().setSql( + "INSERT INTO foo1 SELECT dim3 FROM foo WHERE dim3 IS NOT NULL GROUP BY 1 PARTITIONED BY ALL TIME") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedResultRows(expectedMultiValueFooRowsGroupBy()) + .verifyResults(); + } + + @Test + public void testInsertOnFoo1WithMultiValueMeasureGroupBy() + { + testIngestQuery().setSql( + "INSERT INTO foo1 SELECT count(dim3) FROM foo WHERE dim3 IS NOT NULL GROUP BY 1 PARTITIONED BY ALL TIME") + .setExpectedDataSource("foo1") + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "Aggregate expression is illegal in GROUP BY clause")) + )) + .verifyPlanningErrors(); + } + + + @Test + public void testInsertOnFoo1WithMultiValueToArrayGroupBy() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim3", ColumnType.STRING).build(); + + testIngestQuery().setSql( + "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedResultRows(expectedMultiValueFooRowsToArray()) + .verifyResults(); + } + + @Test + public void testInsertOnFoo1WithMultiValueDimGroupByWithoutGroupByEnable() + { + Map context = ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put("groupByEnableMultiValueUnnesting", false) + .build(); + + testIngestQuery().setSql( + "INSERT INTO foo1 SELECT dim3, count(*) AS cnt1 FROM foo GROUP BY dim3 PARTITIONED BY ALL TIME") + .setQueryContext(context) + .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(ISE.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.")) + )) + .verifyExecutionError(); + } + + @Test + public void testRolltestRollUpOnFoo1UpOnFoo1() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + testIngestQuery().setSql( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setQueryContext(ROLLUP_CONTEXT) + .setExpectedRollUp(true) + .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(expectedFooSegments()) + .setExpectedResultRows(expectedFooRows()) + .verifyResults(); + + } + + @Test + public void testRollUpOnFoo1WithTimeFunction() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + + testIngestQuery().setSql( + "insert into foo1 select floor(__time to day) as __time , dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setQueryContext(ROLLUP_CONTEXT) + .setExpectedRollUp(true) + .setExpectedQueryGranularity(Granularities.DAY) + .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(expectedFooSegments()) + .setExpectedResultRows(expectedFooRows()) + .verifyResults(); + + } + + @Test + public void testRollUpOnFoo1WithTimeFunctionComplexCol() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", new ColumnType(ValueType.COMPLEX, "hyperUnique", null)) + .build(); + + + testIngestQuery().setSql( + "insert into foo1 select floor(__time to day) as __time , dim1 , count(distinct m1) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setQueryContext(ROLLUP_CONTEXT) + .setExpectedRollUp(true) + .setExpectedQueryGranularity(Granularities.DAY) + .addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true)) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(expectedFooSegments()) + .setExpectedResultRows(expectedFooRowsWithAggregatedComplexColumn()) + .verifyResults(); + + } + + + @Test + public void testRollUpOnFoo1ComplexCol() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", new ColumnType(ValueType.COMPLEX, "hyperUnique", null)) + .build(); + + testIngestQuery().setSql( + "insert into foo1 select __time , dim1 , count(distinct m1) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setQueryContext(ROLLUP_CONTEXT) + .setExpectedRollUp(true) + .addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true)) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(expectedFooSegments()) + .setExpectedResultRows(expectedFooRowsWithAggregatedComplexColumn()) + .verifyResults(); + + } + + @Test + public void testRollUpOnExternalDataSource() throws IOException + { + final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); + final String toReadFileNameAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("cnt", ColumnType.LONG) + .build(); + + testIngestQuery().setSql(" insert into foo1 SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1 PARTITIONED by day ") + .setQueryContext(ROLLUP_CONTEXT) + .setExpectedRollUp(true) + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo1", + Intervals.of("2016-06-27/P1D"), + "test", + 0 + ))) + .setExpectedResultRows(ImmutableList.of(new Object[]{1466985600000L, 20L})) + .verifyResults(); + } + + @Test() + public void testRollUpOnExternalDataSourceWithCompositeKey() throws IOException + { + final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); + final String toReadFileNameAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("namespace", ColumnType.STRING) + .add("cnt", ColumnType.LONG) + .build(); + + testIngestQuery().setSql(" insert into foo1 SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " namespace , count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"namespace\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1,2 PARTITIONED by day ") + .setQueryContext(ROLLUP_CONTEXT) + .setExpectedRollUp(true) + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo1", + Intervals.of("2016-06-27/P1D"), + "test", + 0 + ))) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1466985600000L, "Benutzer Diskussion", 2L}, + new Object[]{1466985600000L, "File", 1L}, + new Object[]{1466985600000L, "Kategoria", 1L}, + new Object[]{1466985600000L, "Main", 14L}, + new Object[]{1466985600000L, "Wikipedia", 1L}, + new Object[]{1466985600000L, "Википедия", 1L} + )) + .verifyResults(); + + } + + @Test + public void testInsertNullTimestamp() + { + final RowSignature rowSignature = + RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .build(); + + testIngestQuery() + .setSql( + "INSERT INTO foo1\n" + + "SELECT TIME_PARSE(dim1) AS __time, dim1 as cnt\n" + + "FROM foo\n" + + "PARTITIONED BY DAY\n" + + "CLUSTERED BY dim1") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedMSQFault(InsertTimeNullFault.instance()) + .verifyResults(); + } + + @Test + public void testInsertWrongTypeTimestamp() + { + final RowSignature rowSignature = + RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .build(); + + testIngestQuery() + .setSql( + "INSERT INTO foo1\n" + + "SELECT dim1 AS __time, cnt\n" + + "FROM foo\n" + + "PARTITIONED BY DAY\n" + + "CLUSTERED BY dim1") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "Field \"__time\" must be of type TIMESTAMP")) + )) + .verifyPlanningErrors(); + } + + @Test + public void testIncorrectInsertQuery() + { + testIngestQuery().setSql( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 clustered by dim1") + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( + "CLUSTERED BY found before PARTITIONED BY. In druid, the CLUSTERED BY clause has to be specified after the PARTITIONED BY clause")) + )) + .verifyPlanningErrors(); + } + + @Test + public void testInsertRestrictedColumns() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("namespace", ColumnType.STRING) + .add("__bucket", ColumnType.LONG) + .build(); + + + testIngestQuery() + .setSql(" insert into foo1 SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " namespace, __bucket\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"ignored\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"namespace\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}, {\"name\": \"__bucket\", \"type\": \"string\"}]'\n" + + " )\n" + + ") PARTITIONED by day") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedMSQFault(new ColumnNameRestrictedFault("__bucket")) + .verifyResults(); + } + + @Test + public void testInsertQueryWithInvalidSubtaskCount() + { + Map context = ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 1) + .build(); + testIngestQuery().setSql( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setQueryContext(context) + .setExpectedExecutionErrorMatcher( + ThrowableMessageMatcher.hasMessage( + CoreMatchers.startsWith( + MultiStageQueryContext.CTX_MAX_NUM_TASKS + + " cannot be less than 2 since at least 1 controller and 1 worker is necessary." + ) + ) + ) + .verifyExecutionError(); + } + + @Test + public void testInsertWithTooLargeRowShouldThrowException() throws IOException + { + final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); + final String toReadFileNameAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + Mockito.doReturn(500).when(workerMemoryParameters).getLargeFrameSize(); + + testIngestQuery().setSql(" insert into foo1 SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1 PARTITIONED by day ") + .setExpectedDataSource("foo") + .setExpectedMSQFault(new RowTooLargeFault(500)) + .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(ISE.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "Row too large to add to frame")) + )) + .verifyExecutionError(); + } + + @Test + public void testInsertLimitWithPeriodGranularityThrowsException() + { + testIngestQuery().setSql(" INSERT INTO foo " + + "SELECT __time, m1 " + + "FROM foo " + + "LIMIT 50 " + + "PARTITIONED BY MONTH") + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "INSERT and REPLACE queries cannot have a LIMIT unless sqlInsertSegmentGranularity is \"all\"")) + )) + .verifyPlanningErrors(); + } + + @Test + public void testInsertOffsetThrowsException() + { + testIngestQuery().setSql(" INSERT INTO foo " + + "SELECT __time, m1 " + + "FROM foo " + + "LIMIT 50 " + + "OFFSET 10" + + "PARTITIONED BY ALL TIME") + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "INSERT and REPLACE queries cannot have an OFFSET")) + )) + .verifyPlanningErrors(); + } + + @Nonnull + private List expectedFooRows() + { + List expectedRows = new ArrayList<>(); + if (!useDefault) { + expectedRows.add(new Object[]{946684800000L, "", 1L}); + } + expectedRows.addAll(ImmutableList.of( + new Object[]{946771200000L, "10.1", 1L}, + new Object[]{946857600000L, "2", 1L}, + new Object[]{978307200000L, "1", 1L}, + new Object[]{978393600000L, "def", 1L}, + new Object[]{978480000000L, "abc", 1L} + )); + return expectedRows; + } + + @Nonnull + private List expectedFooRowsWithAggregatedComplexColumn() + { + List expectedRows = new ArrayList<>(); + HyperLogLogCollector hyperLogLogCollector = HyperLogLogCollector.makeLatestCollector(); + hyperLogLogCollector.add(fn.hashInt(1).asBytes()); + if (!useDefault) { + expectedRows.add(new Object[]{946684800000L, "", hyperLogLogCollector.estimateCardinalityRound()}); + } + expectedRows.addAll(ImmutableList.of( + new Object[]{946771200000L, "10.1", hyperLogLogCollector.estimateCardinalityRound()}, + new Object[]{946857600000L, "2", hyperLogLogCollector.estimateCardinalityRound()}, + new Object[]{978307200000L, "1", hyperLogLogCollector.estimateCardinalityRound()}, + new Object[]{978393600000L, "def", hyperLogLogCollector.estimateCardinalityRound()}, + new Object[]{978480000000L, "abc", hyperLogLogCollector.estimateCardinalityRound()} + )); + return expectedRows; + } + + @Nonnull + private List expectedMultiValueFooRows() + { + List expectedRows = new ArrayList<>(); + if (!useDefault) { + expectedRows.add(new Object[]{0L, ""}); + } + + expectedRows.addAll( + ImmutableList.of( + new Object[]{0L, ImmutableList.of("a", "b")}, + new Object[]{0L, ImmutableList.of("b", "c")}, + new Object[]{0L, "d"} + )); + return expectedRows; + } + + @Nonnull + private List expectedMultiValueFooRowsToArray() + { + List expectedRows = new ArrayList<>(); + expectedRows.add(new Object[]{0L, null}); + if (!useDefault) { + expectedRows.add(new Object[]{0L, ""}); + } + + expectedRows.addAll(ImmutableList.of( + new Object[]{0L, ImmutableList.of("a", "b")}, + new Object[]{0L, ImmutableList.of("b", "c")}, + new Object[]{0L, "d"} + )); + return expectedRows; + } + + @Nonnull + private List expectedMultiValueFooRowsGroupBy() + { + List expectedRows = new ArrayList<>(); + if (!useDefault) { + expectedRows.add(new Object[]{0L, ""}); + } + expectedRows.addAll(ImmutableList.of( + new Object[]{0L, "a"}, + new Object[]{0L, "b"}, + new Object[]{0L, "c"}, + new Object[]{0L, "d"} + )); + return expectedRows; + } + + @Nonnull + private Set expectedFooSegments() + { + Set expectedSegments = new TreeSet<>(); + + if (!useDefault) { + expectedSegments.add(SegmentId.of("foo1", Intervals.of("2000-01-01T/P1D"), "test", 0)); + } + expectedSegments.addAll( + ImmutableSet.of( + SegmentId.of("foo1", Intervals.of("2000-01-02T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2000-01-03T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2001-01-01T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2001-01-02T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2001-01-03T/P1D"), "test", 0) + )); + + return expectedSegments; + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java new file mode 100644 index 00000000000..131a9fa91b5 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -0,0 +1,506 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.DimensionRangeShardSpec; +import org.hamcrest.CoreMatchers; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import javax.annotation.Nonnull; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +public class MSQReplaceTest extends MSQTestBase +{ + @Test + public void testReplaceOnFooWithAll() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + + "SELECT __time, m1 " + + "FROM foo " + + "PARTITIONED BY DAY ") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegment( + ImmutableSet.of( + SegmentId.of("foo", Intervals.of("2000-01-01T/P1D"), "test", 0), + SegmentId.of("foo", Intervals.of("2000-01-02T/P1D"), "test", 0), + SegmentId.of("foo", Intervals.of("2000-01-03T/P1D"), "test", 0), + SegmentId.of("foo", Intervals.of("2001-01-01T/P1D"), "test", 0), + SegmentId.of("foo", Intervals.of("2001-01-02T/P1D"), "test", 0), + SegmentId.of("foo", Intervals.of("2001-01-03T/P1D"), "test", 0) + ) + ) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f}, + new Object[]{946857600000L, 3.0f}, + new Object[]{978307200000L, 4.0f}, + new Object[]{978393600000L, 5.0f}, + new Object[]{978480000000L, 6.0f} + ) + ) + .verifyResults(); + } + + @Test + public void testReplaceOnFooWithWhere() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE WHERE __time >= TIMESTAMP '2000-01-02' AND __time < TIMESTAMP '2000-01-03' " + + "SELECT __time, m1 " + + "FROM foo " + + "WHERE __time >= TIMESTAMP '2000-01-02' AND __time < TIMESTAMP '2000-01-03' " + + "PARTITIONED by DAY ") + .setExpectedDataSource("foo") + .setExpectedDestinationIntervals(ImmutableList.of(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"))) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-02T/P1D"), "test", 0))) + .setExpectedResultRows(ImmutableList.of(new Object[]{946771200000L, 2.0f})) + .verifyResults(); + } + + @Test + public void testReplaceOnFoo1WithAllExtern() throws IOException + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("cnt", ColumnType.LONG).build(); + + final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); + final String toReadFileNameAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + testIngestQuery().setSql(" REPLACE INTO foo1 OVERWRITE ALL SELECT " + + " floor(TIME_PARSE(\"timestamp\") to hour) AS __time, " + + " count(*) AS cnt " + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") GROUP BY 1 PARTITIONED BY HOUR ") + .setExpectedDataSource("foo1") + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of( + SegmentId.of("foo1", Intervals.of("2016-06-27T00:00:00.000Z/2016-06-27T01:00:00.000Z"), "test", 0), + SegmentId.of("foo1", Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"), "test", 0), + SegmentId.of("foo1", Intervals.of("2016-06-27T02:00:00.000Z/2016-06-27T03:00:00.000Z"), "test", 0)) + ) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{1466985600000L, 10L}, + new Object[]{1466989200000L, 4L}, + new Object[]{1466992800000L, 6L} + ) + ) + .verifyResults(); + } + + @Test + public void testReplaceOnFoo1WithWhereExtern() throws IOException + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("user", ColumnType.STRING).build(); + + final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); + final String toReadFileNameAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + testIngestQuery().setSql(" REPLACE INTO foo1 OVERWRITE WHERE __time >= TIMESTAMP '2016-06-27 01:00:00.00' AND __time < TIMESTAMP '2016-06-27 02:00:00.00' " + + " SELECT " + + " floor(TIME_PARSE(\"timestamp\") to hour) AS __time, " + + " user " + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") " + + "where \"timestamp\" >= TIMESTAMP '2016-06-27 01:00:00.00' AND \"timestamp\" < TIMESTAMP '2016-06-27 02:00:00.00' " + + "PARTITIONED BY HOUR ") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(ImmutableList.of(Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"), "test", 0))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{1466989200000L, "2001:DA8:207:E132:94DC:BA03:DFDF:8F9F"}, + new Object[]{1466989200000L, "Ftihikam"}, + new Object[]{1466989200000L, "Guly600"}, + new Object[]{1466989200000L, "Kolega2357"} + ) + ) + .verifyResults(); + } + + @Test + public void testReplaceIncorrectSyntax() + { + testIngestQuery().setSql("REPLACE INTO foo1 OVERWRITE SELECT * FROM foo PARTITIONED BY ALL TIME") + .setExpectedDataSource("foo1") + .setExpectedValidationErrorMatcher( + CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "Missing time chunk information in OVERWRITE clause for REPLACE, set it to OVERWRITE WHERE <__time based condition> or set it to overwrite the entire table with OVERWRITE ALL.")) + ) + ) + .verifyPlanningErrors(); + } + + @Test + public void testReplaceSegmentEntireTable() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE ALL " + + "SELECT __time, m1 " + + "FROM foo " + + "PARTITIONED BY ALL TIME ") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f}, + new Object[]{946857600000L, 3.0f}, + new Object[]{978307200000L, 4.0f}, + new Object[]{978393600000L, 5.0f}, + new Object[]{978480000000L, 6.0f} + ) + ) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .verifyResults(); + } + + @Test + public void testReplaceSegmentsRepartitionTable() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE ALL " + + "SELECT __time, m1 " + + "FROM foo " + + "PARTITIONED BY MONTH") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f}, + new Object[]{946857600000L, 3.0f}, + new Object[]{978307200000L, 4.0f}, + new Object[]{978393600000L, 5.0f}, + new Object[]{978480000000L, 6.0f} + ) + ) + .setExpectedSegment(ImmutableSet.of( + SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0), + SegmentId.of("foo", Intervals.of("2001-01-01T/P1M"), "test", 0)) + ) + .verifyResults(); + } + + @Test + public void testReplaceWithWhereClause() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2000-03-01' " + + "SELECT __time, m1 " + + "FROM foo " + + "WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2000-01-03' " + + "PARTITIONED BY MONTH") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f} + ) + ) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .verifyResults(); + } + + @Test + public void testReplaceWhereClauseLargerThanData() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2002-01-01' " + + "SELECT __time, m1 " + + "FROM foo " + + "WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2000-01-03' " + + "PARTITIONED BY MONTH") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T"))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f} + ) + ) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .verifyResults(); + } + + @Test + public void testReplaceLimitWithPeriodGranularityThrowsException() + { + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE ALL " + + "SELECT __time, m1 " + + "FROM foo " + + "LIMIT 50" + + "PARTITIONED BY MONTH") + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "INSERT and REPLACE queries cannot have a LIMIT unless sqlInsertSegmentGranularity is \"all\"")) + )) + .verifyPlanningErrors(); + } + + @Test + public void testReplaceOffsetThrowsException() + { + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE ALL " + + "SELECT __time, m1 " + + "FROM foo " + + "LIMIT 50 " + + "OFFSET 10" + + "PARTITIONED BY ALL TIME") + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "INSERT and REPLACE queries cannot have an OFFSET")) + )) + .verifyPlanningErrors(); + } + + @Test + public void testReplaceTimeChunks() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2000-03-01'" + + "SELECT __time, m1 " + + "FROM foo " + + "WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2000-01-03' " + + "PARTITIONED BY MONTH") + .setExpectedDataSource("foo") + .setQueryContext(DEFAULT_MSQ_CONTEXT) + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f} + ) + ) + .verifyResults(); + } + + @Test + public void testReplaceTimeChunksLargerThanData() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foo " + + "OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2002-01-01'" + + "SELECT __time, m1 " + + "FROM foo " + + "WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2000-01-03' " + + "PARTITIONED BY MONTH") + .setExpectedDataSource("foo") + .setQueryContext(DEFAULT_MSQ_CONTEXT) + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T"))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f} + ) + ) + .verifyResults(); + } + + @Test + public void testInsertOnFoo1Range() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG) + .build(); + + testIngestQuery().setSql( + "REPLACE INTO foo1 OVERWRITE ALL " + + "select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setQueryContext(DEFAULT_MSQ_CONTEXT) + .setExpectedShardSpec(DimensionRangeShardSpec.class) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(expectedFooSegments()) + .setExpectedResultRows(expectedFooRows()) + .verifyResults(); + + } + + @Test + public void testReplaceSegmentsInsertIntoNewTable() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("m1", ColumnType.FLOAT) + .build(); + + testIngestQuery().setSql(" REPLACE INTO foobar " + + "OVERWRITE ALL " + + "SELECT __time, m1 " + + "FROM foo " + + "PARTITIONED BY ALL TIME ") + .setExpectedDataSource("foobar") + .setExpectedRowSignature(rowSignature) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegment(ImmutableSet.of(SegmentId.of("foobar", Intervals.ETERNITY, "test", 0))) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1.0f}, + new Object[]{946771200000L, 2.0f}, + new Object[]{946857600000L, 3.0f}, + new Object[]{978307200000L, 4.0f}, + new Object[]{978393600000L, 5.0f}, + new Object[]{978480000000L, 6.0f} + ) + ) + .verifyResults(); + } + + @Nonnull + private Set expectedFooSegments() + { + Set expectedSegments = new TreeSet<>(); + + if (!useDefault) { + expectedSegments.add(SegmentId.of("foo1", Intervals.of("2000-01-01T/P1D"), "test", 0)); + } + expectedSegments.addAll( + ImmutableSet.of( + SegmentId.of("foo1", Intervals.of("2000-01-02T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2000-01-03T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2001-01-01T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2001-01-02T/P1D"), "test", 0), + SegmentId.of("foo1", Intervals.of("2001-01-03T/P1D"), "test", 0) + )); + + return expectedSegments; + } + + @Nonnull + private List expectedFooRows() + { + List expectedRows = new ArrayList<>(); + if (!useDefault) { + expectedRows.add(new Object[]{946684800000L, "", 1L}); + } + expectedRows.addAll(ImmutableList.of( + new Object[]{946771200000L, "10.1", 1L}, + new Object[]{946857600000L, "2", 1L}, + new Object[]{978307200000L, "1", 1L}, + new Object[]{978393600000L, "def", 1L}, + new Object[]{978480000000L, "abc", 1L} + )); + return expectedRows; + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java new file mode 100644 index 00000000000..4cf33b3e1c4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -0,0 +1,1052 @@ +/* + * 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.druid.msq.exec; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.indexing.ColumnMapping; +import org.apache.druid.msq.indexing.ColumnMappings; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.FilteredAggregatorFactory; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.ExpressionPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.sql.calcite.expression.DruidExpression; +import org.apache.druid.sql.calcite.external.ExternalDataSource; +import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.hamcrest.CoreMatchers; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import javax.annotation.Nonnull; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class MSQSelectTest extends MSQTestBase +{ + @Test + public void testCalculator() + { + RowSignature resultSignature = RowSignature.builder() + .add("EXPR$0", ColumnType.LONG) + .build(); + + testSelectQuery() + .setSql("select 1 + 1") + .setExpectedMSQSpec( + MSQSpec.builder() + .query( + newScanQueryBuilder() + .dataSource( + InlineDataSource.fromIterable( + ImmutableList.of(new Object[]{2L}), + resultSignature + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("EXPR$0") + .context(defaultScanQueryContext(resultSignature)) + .build() + ) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{2L})).verifyResults(); + } + + @Test + public void testSelectOnFoo() + { + RowSignature resultSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .build(); + + testSelectQuery() + .setSql("select cnt,dim1 from foo") + .setExpectedMSQSpec( + MSQSpec.builder() + .query( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("cnt", "dim1") + .context(defaultScanQueryContext(resultSignature)) + .build() + ) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1L, !useDefault ? "" : null}, + new Object[]{1L, "10.1"}, + new Object[]{1L, "2"}, + new Object[]{1L, "1"}, + new Object[]{1L, "def"}, + new Object[]{1L, "abc"} + )).verifyResults(); + } + + @Test + public void testSelectOnFoo2() + { + RowSignature resultSignature = RowSignature.builder() + .add("m1", ColumnType.LONG) + .add("dim2", ColumnType.STRING) + .build(); + + testSelectQuery() + .setSql("select m1,dim2 from foo2") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE2) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2", "m1") + .context(defaultScanQueryContext( + RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("m1", ColumnType.LONG) + .build() + )) + .build()) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1L, "en"}, + new Object[]{1L, "ru"}, + new Object[]{1L, "he"} + )).verifyResults(); + } + + @Test + public void testGroupByOnFoo() + { + RowSignature rowSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + testSelectQuery() + .setSql("select cnt,count(*) as cnt1 from foo group by cnt") + .setExpectedMSQSpec(MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration + .eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec( + "cnt", + "d0", + ColumnType.LONG + ) + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(DEFAULT_MSQ_CONTEXT) + .build()) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "cnt"), + new ColumnMapping("a0", "cnt1") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) + .verifyResults(); + } + + @Test + public void testGroupByOrderByDimension() + { + RowSignature rowSignature = RowSignature.builder() + .add("m1", ColumnType.FLOAT) + .add("cnt", ColumnType.LONG) + .build(); + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "d0", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + ) + ), + null + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql("select m1, count(*) as cnt from foo group by m1 order by m1 desc") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "m1"), + new ColumnMapping("a0", "cnt") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{6f, 1L}, + new Object[]{5f, 1L}, + new Object[]{4f, 1L}, + new Object[]{3f, 1L}, + new Object[]{2f, 1L}, + new Object[]{1f, 1L} + ) + ) + .verifyResults(); + } + + @Test + public void testSubquery() + { + RowSignature resultSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .build(); + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource("foo") + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT))) + .setContext(DEFAULT_MSQ_CONTEXT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql("select count(*) AS cnt from (select distinct m1 from foo)") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "cnt")))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{6L})) + .verifyResults(); + } + + @Test + public void testJoin() + { + final RowSignature resultSignature = RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("EXPR$1", ColumnType.DOUBLE) + .build(); + + final ImmutableList expectedResults; + + if (NullHandling.sqlCompatible()) { + expectedResults = ImmutableList.of( + new Object[]{null, 4.0}, + new Object[]{"", 3.0}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ); + } else { + expectedResults = ImmutableList.of( + new Object[]{null, 3.6666666666666665}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ); + } + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2", "m1", "m2") + .context( + defaultScanQueryContext( + RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("m1", ColumnType.FLOAT) + .add("m2", ColumnType.DOUBLE) + .build() + ) + ) + .limit(10) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns(ImmutableList.of("m1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context( + defaultScanQueryContext( + RowSignature.builder().add("m1", ColumnType.FLOAT).build() + ) + ) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.ofColumn(ColumnType.FLOAT, "m1"), + DruidExpression.ofColumn(ColumnType.FLOAT, "j0.m1") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING)) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + useDefault + ? aggregators( + new DoubleSumAggregatorFactory("a0:sum", "m2"), + new CountAggregatorFactory("a0:count") + ) + : aggregators( + new DoubleSumAggregatorFactory("a0:sum", "m2"), + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0:count"), + not(selector("m2", null, null)), + + // Not sure why the name is only set in SQL-compatible null mode. Seems strange. + // May be due to JSON serialization: name is set on the serialized aggregator even + // if it was originally created with no name. + NullHandling.sqlCompatible() ? "a0:count" : null + ) + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "a0:sum"), + new FieldAccessPostAggregator(null, "a0:count") + ) + ) + + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql( + "SELECT t1.dim2, AVG(t1.m2) FROM " + + "(SELECT * FROM foo LIMIT 10) AS t1 " + + "INNER JOIN foo AS t2 " + + "ON t1.m1 = t2.m1 " + + "GROUP BY t1.dim2" + ) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings(new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "dim2"), + new ColumnMapping("a0", "EXPR$1") + ))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(expectedResults) + .verifyResults(); + } + + @Test + public void testGroupByOrderByAggregation() + { + RowSignature rowSignature = RowSignature.builder() + .add("m1", ColumnType.FLOAT) + .add("sum_m1", ColumnType.DOUBLE) + .build(); + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT))) + .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "a0", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + ) + ), + null + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql("select m1, sum(m1) as sum_m1 from foo group by m1 order by sum_m1 desc") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings( + new ColumnMappings( + ImmutableList.of( + new ColumnMapping("d0", "m1"), + new ColumnMapping("a0", "sum_m1") + ) + ) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{6f, 6d}, + new Object[]{5f, 5d}, + new Object[]{4f, 4d}, + new Object[]{3f, 3d}, + new Object[]{2f, 2d}, + new Object[]{1f, 1d} + ) + ).verifyResults(); + } + + @Test + public void testGroupByOrderByAggregationWithLimit() + { + RowSignature rowSignature = RowSignature.builder() + .add("m1", ColumnType.FLOAT) + .add("sum_m1", ColumnType.DOUBLE) + .build(); + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT))) + .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "a0", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + ) + ), + 3 + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql("select m1, sum(m1) as sum_m1 from foo group by m1 order by sum_m1 desc limit 3") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings( + new ColumnMappings( + ImmutableList.of( + new ColumnMapping("d0", "m1"), + new ColumnMapping("a0", "sum_m1") + ) + ) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{6f, 6d}, + new Object[]{5f, 5d}, + new Object[]{4f, 4d} + ) + ).verifyResults(); + } + + @Test + public void testGroupByOrderByAggregationWithLimitAndOffset() + { + RowSignature rowSignature = RowSignature.builder() + .add("m1", ColumnType.FLOAT) + .add("sum_m1", ColumnType.DOUBLE) + .build(); + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT))) + .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "a0", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + ) + ), + 1, + 2 + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql("select m1, sum(m1) as sum_m1 from foo group by m1 order by sum_m1 desc limit 2 offset 1") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings( + new ColumnMappings( + ImmutableList.of( + new ColumnMapping("d0", "m1"), + new ColumnMapping("a0", "sum_m1") + ) + ) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{5f, 5d}, + new Object[]{4f, 4d} + ) + ).verifyResults(); + } + + @Test + public void testExternSelect1() throws IOException + { + final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); + final String toReadAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("cnt", ColumnType.LONG) + .build(); + + final GroupByQuery expectedQuery = + GroupByQuery.builder() + .setDataSource( + new ExternalDataSource( + new LocalInputSource(null, null, ImmutableList.of(toRead.getAbsoluteFile())), + new JsonInputFormat(null, null, null), + RowSignature.builder() + .add("timestamp", ColumnType.STRING) + .add("page", ColumnType.STRING) + .add("user", ColumnType.STRING) + .build() + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + new ExpressionVirtualColumn( + "v0", + "timestamp_floor(timestamp_parse(\"timestamp\",null,'UTC'),'P1D',null,'UTC')", + ColumnType.LONG, + CalciteTests.createExprMacroTable() + ) + ) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1466985600000L, 20L})) + .setExpectedMSQSpec( + MSQSpec + .builder() + .query(expectedQuery) + .columnMappings(new ColumnMappings( + ImmutableList.of( + new ColumnMapping("d0", "__time"), + new ColumnMapping("a0", "cnt") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .verifyResults(); + } + + @Test + public void testIncorrectSelectQuery() + { + testSelectQuery() + .setSql("select a from ") + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Encountered \"from \"")) + )) + .verifyPlanningErrors(); + } + + @Test + public void testSelectOnInformationSchemaSource() + { + testSelectQuery() + .setSql("SELECT * FROM INFORMATION_SCHEMA.SCHEMATA") + .setExpectedValidationErrorMatcher( + CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( + "Cannot query table [INFORMATION_SCHEMA.SCHEMATA] with SQL engine 'msq-task'.")) + ) + ) + .verifyPlanningErrors(); + } + + @Test + public void testSelectOnSysSource() + { + testSelectQuery() + .setSql("SELECT * FROM sys.segments") + .setExpectedValidationErrorMatcher( + CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( + "Cannot query table [sys.segments] with SQL engine 'msq-task'.")) + ) + ) + .verifyPlanningErrors(); + } + + @Test + public void testSelectOnSysSourceWithJoin() + { + testSelectQuery() + .setSql("select s.segment_id, s.num_rows, f.dim1 from sys.segments as s, foo as f") + .setExpectedValidationErrorMatcher( + CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( + "Cannot query table [sys.segments] with SQL engine 'msq-task'.")) + ) + ) + .verifyPlanningErrors(); + } + + @Test + public void testSelectOnSysSourceContainingWith() + { + testSelectQuery() + .setSql("with segment_source as (SELECT * FROM sys.segments) " + + "select segment_source.segment_id, segment_source.num_rows from segment_source") + .setExpectedValidationErrorMatcher( + CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( + "Cannot query table [sys.segments] with SQL engine 'msq-task'.")) + ) + ) + .verifyPlanningErrors(); + } + + + @Test + public void testSelectOnUserDefinedSourceContainingWith() + { + RowSignature resultSignature = RowSignature.builder() + .add("m1", ColumnType.LONG) + .add("dim2", ColumnType.STRING) + .build(); + + testSelectQuery() + .setSql("with sys as (SELECT * FROM foo2) " + + "select m1, dim2 from sys") + .setExpectedMSQSpec( + MSQSpec.builder() + .query( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE2) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2", "m1") + .context(defaultScanQueryContext( + RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("m1", ColumnType.LONG) + .build() + )) + .build() + ) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1L, "en"}, + new Object[]{1L, "ru"}, + new Object[]{1L, "he"} + )) + .verifyResults(); + } + + @Test + public void testScanWithMultiValueSelectQuery() + { + RowSignature resultSignature = RowSignature.builder() + .add("dim3", ColumnType.STRING) + .build(); + + testSelectQuery() + .setSql("select dim3 from foo") + .setExpectedMSQSpec(MSQSpec.builder() + .query(newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim3") + .context(defaultScanQueryContext(resultSignature)) + .build()) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{ImmutableList.of("a", "b")}, + new Object[]{ImmutableList.of("b", "c")}, + new Object[]{"d"}, + new Object[]{!useDefault ? "" : null}, + new Object[]{null}, + new Object[]{null} + )).verifyResults(); + } + + @Test + public void testGroupByWithMultiValue() + { + Map context = ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put("groupByEnableMultiValueUnnesting", true) + .build(); + RowSignature rowSignature = RowSignature.builder() + .add("dim3", ColumnType.STRING) + .add("cnt1", ColumnType.LONG) + .build(); + + testSelectQuery() + .setSql("select dim3, count(*) as cnt1 from foo group by dim3") + .setQueryContext(context) + .setExpectedMSQSpec( + MSQSpec.builder() + .query( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec( + "dim3", + "d0" + ) + ) + ) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(context) + .build() + ) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "dim3"), + new ColumnMapping("a0", "cnt1") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows( + expectedMultiValueFooRowsGroup() + ) + .verifyResults(); + } + + + @Test + public void testGroupByWithMultiValueWithoutGroupByEnable() + { + Map context = ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put("groupByEnableMultiValueUnnesting", false) + .build(); + + testSelectQuery() + .setSql("select dim3, count(*) as cnt1 from foo group by dim3") + .setQueryContext(context) + .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(ISE.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.")) + )) + .verifyExecutionError(); + } + + @Test + public void testGroupByWithMultiValueMvToArray() + { + Map context = ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put("groupByEnableMultiValueUnnesting", true) + .build(); + + RowSignature rowSignature = RowSignature.builder() + .add("EXPR$0", ColumnType.STRING_ARRAY) + .add("cnt1", ColumnType.LONG) + .build(); + + testSelectQuery() + .setSql("select MV_TO_ARRAY(dim3), count(*) as cnt1 from foo group by dim3") + .setQueryContext(context) + .setExpectedMSQSpec(MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec( + "dim3", + "d0" + ) + ) + ) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setPostAggregatorSpecs( + ImmutableList.of(new ExpressionPostAggregator( + "p0", + "mv_to_array(\"d0\")", + null, ExprMacroTable.nil() + ) + ) + ) + .setContext(context) + .build() + ) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("p0", "EXPR$0"), + new ColumnMapping("a0", "cnt1") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows( + expectedMultiValueFooRowsGroupByList() + ) + .verifyResults(); + } + + @Test + public void testGroupByWithMultiValueMvToArrayWithoutGroupByEnable() + { + Map context = ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put("groupByEnableMultiValueUnnesting", false) + .build(); + + testSelectQuery() + .setSql("select MV_TO_ARRAY(dim3), count(*) as cnt1 from foo group by dim3") + .setQueryContext(context) + .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(ISE.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.")) + )) + .verifyExecutionError(); + } + + @Test + public void testGroupByMultiValueMeasureQuery() + { + final RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + final GroupByQuery expectedQuery = + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ColumnType.LONG))) + .setAggregatorSpecs( + aggregators( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + new NotDimFilter(new SelectorDimFilter("dim3", null, null)), + "a0" + ) + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql("select __time, count(dim3) as cnt1 from foo group by __time") + .setQueryContext(DEFAULT_MSQ_CONTEXT) + .setExpectedMSQSpec(MSQSpec.builder() + .query(expectedQuery) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "__time"), + new ColumnMapping("a0", "cnt1") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, 1L}, + new Object[]{946771200000L, 1L}, + new Object[]{946857600000L, 1L}, + new Object[]{978307200000L, !useDefault ? 1L : 0L}, + new Object[]{978393600000L, 0L}, + new Object[]{978480000000L, 0L} + ) + ) + .verifyResults(); + } + + @Nonnull + private List expectedMultiValueFooRowsGroup() + { + ArrayList expected = new ArrayList<>(); + expected.add(new Object[]{null, !useDefault ? 2L : 3L}); + if (!useDefault) { + expected.add(new Object[]{"", 1L}); + } + expected.addAll(ImmutableList.of( + new Object[]{"a", 1L}, + new Object[]{"b", 2L}, + new Object[]{"c", 1L}, + new Object[]{"d", 1L} + )); + return expected; + } + + @Nonnull + private List expectedMultiValueFooRowsGroupByList() + { + ArrayList expected = new ArrayList<>(); + expected.add(new Object[]{Collections.singletonList(null), !useDefault ? 2L : 3L}); + if (!useDefault) { + expected.add(new Object[]{Collections.singletonList(""), 1L}); + } + expected.addAll(ImmutableList.of( + new Object[]{Collections.singletonList("a"), 1L}, + new Object[]{Collections.singletonList("b"), 2L}, + new Object[]{Collections.singletonList("c"), 1L}, + new Object[]{Collections.singletonList("d"), 1L} + )); + return expected; + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java new file mode 100644 index 00000000000..1321d1362f7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java @@ -0,0 +1,247 @@ +/* + * 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.druid.msq.exec; + +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.msq.indexing.MSQWorkerTask; +import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; +import org.apache.druid.msq.indexing.error.TooManyColumnsFault; +import org.apache.druid.msq.indexing.error.TooManyWorkersFault; +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class MSQTasksTest +{ + private static final String CONTROLLER_ID = "controller-id"; + private static final String WORKER_ID = "worker-id"; + private static final String CONTROLLER_HOST = "controller-host"; + private static final String WORKER_HOST = "worker-host"; + + @Test + public void test_makeErrorReport_allNull() + { + Assert.assertEquals( + MSQErrorReport.fromFault( + CONTROLLER_ID, + CONTROLLER_HOST, + null, + UnknownFault.forMessage(MSQTasks.GENERIC_QUERY_FAILED_MESSAGE) + ), + MSQTasks.makeErrorReport(CONTROLLER_ID, CONTROLLER_HOST, null, null) + ); + } + + @Test + public void test_makeErrorReport_controllerOnly() + { + final MSQErrorReport controllerReport = MSQTasks.makeErrorReport( + CONTROLLER_ID, + CONTROLLER_HOST, + MSQErrorReport.fromFault(CONTROLLER_ID, CONTROLLER_HOST, null, new TooManyColumnsFault(1, 10)), + null + ); + + Assert.assertEquals(controllerReport, MSQTasks.makeErrorReport(WORKER_ID, WORKER_HOST, controllerReport, null)); + } + + @Test + public void test_makeErrorReport_workerOnly() + { + final MSQErrorReport workerReport = MSQTasks.makeErrorReport( + WORKER_ID, + WORKER_HOST, + MSQErrorReport.fromFault(WORKER_ID, WORKER_HOST, null, new TooManyColumnsFault(1, 10)), + null + ); + + Assert.assertEquals(workerReport, MSQTasks.makeErrorReport(WORKER_ID, WORKER_HOST, null, workerReport)); + } + + @Test + public void test_makeErrorReport_controllerPreferred() + { + final MSQErrorReport controllerReport = MSQTasks.makeErrorReport( + WORKER_ID, + WORKER_HOST, + MSQErrorReport.fromFault(WORKER_ID, WORKER_HOST, null, new TooManyWorkersFault(2, 20)), + null + ); + + final MSQErrorReport workerReport = MSQTasks.makeErrorReport( + WORKER_ID, + WORKER_HOST, + MSQErrorReport.fromFault(WORKER_ID, WORKER_HOST, null, new TooManyColumnsFault(1, 10)), + null + ); + + Assert.assertEquals( + controllerReport, + MSQTasks.makeErrorReport(WORKER_ID, WORKER_HOST, controllerReport, workerReport) + ); + } + + @Test + public void test_makeErrorReport_workerPreferred() + { + final MSQErrorReport controllerReport = MSQTasks.makeErrorReport( + WORKER_ID, + WORKER_HOST, + MSQErrorReport.fromFault(WORKER_ID, WORKER_HOST, null, new WorkerRpcFailedFault(WORKER_ID)), + null + ); + + final MSQErrorReport workerReport = MSQTasks.makeErrorReport( + WORKER_ID, + WORKER_HOST, + MSQErrorReport.fromFault(WORKER_ID, WORKER_HOST, null, new TooManyColumnsFault(1, 10)), + null + ); + + Assert.assertEquals( + workerReport, + MSQTasks.makeErrorReport(WORKER_ID, WORKER_HOST, controllerReport, workerReport) + ); + } + + @Test + public void test_queryWithoutEnoughSlots_shouldThrowException() + { + final int numSlots = 5; + final int numTasks = 10; + + ControllerContext controllerContext = mock(ControllerContext.class); + when(controllerContext.workerManager()).thenReturn(new TasksTestWorkerManagerClient(numSlots)); + MSQWorkerTaskLauncher msqWorkerTaskLauncher = new MSQWorkerTaskLauncher( + CONTROLLER_ID, + "foo", + controllerContext, + false, + TimeUnit.SECONDS.toMillis(5) + ); + + try { + msqWorkerTaskLauncher.start(); + msqWorkerTaskLauncher.launchTasksIfNeeded(numTasks); + fail(); + } + catch (Exception e) { + Assert.assertEquals( + new TaskStartTimeoutFault(numTasks + 1).getCodeWithMessage(), + ((MSQException) e.getCause()).getFault().getCodeWithMessage() + ); + } + } + + static class TasksTestWorkerManagerClient implements WorkerManagerClient + { + // Num of slots available for tasks + final int numSlots; + + @GuardedBy("this") + final Set allTasks = new HashSet<>(); + + @GuardedBy("this") + final Set runningTasks = new HashSet<>(); + + @GuardedBy("this") + final Set canceledTasks = new HashSet<>(); + + public TasksTestWorkerManagerClient(final int numSlots) + { + this.numSlots = numSlots; + } + + @Override + public synchronized Map statuses(final Set taskIds) + { + final Map retVal = new HashMap<>(); + + for (final String taskId : taskIds) { + if (allTasks.contains(taskId)) { + retVal.put( + taskId, + new TaskStatus( + taskId, + canceledTasks.contains(taskId) ? TaskState.FAILED : TaskState.RUNNING, + 2, + null, + null + ) + ); + } + } + + return retVal; + } + + @Override + public synchronized TaskLocation location(String workerId) + { + if (runningTasks.contains(workerId)) { + return TaskLocation.create("host-" + workerId, 1, -1); + } else { + return TaskLocation.unknown(); + } + } + + @Override + public synchronized String run(String controllerId, MSQWorkerTask task) + { + allTasks.add(task.getId()); + + if (runningTasks.size() < numSlots) { + runningTasks.add(task.getId()); + } + + return task.getId(); + } + + @Override + public synchronized void cancel(String workerId) + { + runningTasks.remove(workerId); + canceledTasks.add(workerId); + } + + @Override + public void close() + { + // do nothing + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java new file mode 100644 index 00000000000..d9cbb48d986 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java @@ -0,0 +1,144 @@ +/* + * 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.druid.msq.exec; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.QueryDefinitionBuilder; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageDefinitionBuilder; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; +import java.util.UUID; +import java.util.stream.IntStream; + +public class QueryValidatorTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testValidQueryDefination() + { + QueryValidator.validateQueryDef(createQueryDefinition(1, 1)); + QueryValidator.validateQueryDef(createQueryDefinition( + Limits.MAX_FRAME_COLUMNS, + Limits.MAX_WORKERS + )); + } + + @Test + public void testNegativeWorkers() + { + expectedException.expect(ISE.class); + expectedException.expectMessage("Number of workers must be greater than 0"); + QueryValidator.validateQueryDef(createQueryDefinition(1, -1)); + } + + @Test + public void testZeroWorkers() + { + expectedException.expect(ISE.class); + expectedException.expectMessage("Number of workers must be greater than 0"); + QueryValidator.validateQueryDef(createQueryDefinition(1, 0)); + } + + @Test + public void testGreaterThanMaxWorkers() + { + expectedException.expect(MSQException.class); + expectedException.expectMessage( + StringUtils.format( + "Too many workers (current = %d; max = %d)", + Limits.MAX_WORKERS + 1, + Limits.MAX_WORKERS + )); + QueryValidator.validateQueryDef(createQueryDefinition(1, Limits.MAX_WORKERS + 1)); + } + + @Test + public void testGreaterThanMaxColumns() + { + expectedException.expect(MSQException.class); + expectedException.expectMessage(StringUtils.format( + "Too many output columns (requested = %d, max = %d)", + Limits.MAX_FRAME_COLUMNS + 1, + Limits.MAX_FRAME_COLUMNS + )); + QueryValidator.validateQueryDef(createQueryDefinition(Limits.MAX_FRAME_COLUMNS + 1, 1)); + } + + @Test + public void testMoreInputFiles() + { + int numWorkers = 3; + int inputFiles = numWorkers * Limits.MAX_INPUT_FILES_PER_WORKER + 1; + + final WorkOrder workOrder = new WorkOrder( + createQueryDefinition(inputFiles, numWorkers), + 0, + 0, + Collections.singletonList(() -> inputFiles), // Slice with a large number of inputFiles + null + ); + + expectedException.expect(MSQException.class); + expectedException.expectMessage(StringUtils.format( + "Too many input files/segments [%d] encountered. Maximum input files/segments per worker is set to [%d]. Try" + + " breaking your query up into smaller queries, or increasing the number of workers to at least [%d] by" + + " setting %s in your query context", + inputFiles, + Limits.MAX_INPUT_FILES_PER_WORKER, + numWorkers + 1, + MultiStageQueryContext.CTX_MAX_NUM_TASKS + )); + + QueryValidator.validateWorkOrder(workOrder); + } + + private static QueryDefinition createQueryDefinition(int numColumns, int numWorkers) + { + QueryDefinitionBuilder builder = QueryDefinition.builder(); + builder.queryId(UUID.randomUUID().toString()); + + StageDefinitionBuilder stageBuilder = StageDefinition.builder(0); + builder.add(stageBuilder); + stageBuilder.maxWorkerCount(numWorkers); + + // Need to have *some* processorFactory. + stageBuilder.processorFactory(new OffsetLimitFrameProcessorFactory(1, 1L)); + + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + IntStream.range(0, numColumns).forEach(col -> rowSignatureBuilder.add("col_" + col, ColumnType.STRING)); + stageBuilder.signature(rowSignatureBuilder.build()); + + return builder.build(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java new file mode 100644 index 00000000000..8df168b1384 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java @@ -0,0 +1,116 @@ +/* + * 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.druid.msq.exec; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault; +import org.apache.druid.msq.indexing.error.TooManyWorkersFault; +import org.junit.Assert; +import org.junit.Test; + +public class WorkerMemoryParametersTest +{ + @Test + public void test_oneWorkerInJvm_alone() + { + Assert.assertEquals(parameters(1, 45, 373_000_000), compute(1_000_000_000, 1, 1, 1)); + Assert.assertEquals(parameters(2, 14, 248_000_000), compute(1_000_000_000, 1, 2, 1)); + Assert.assertEquals(parameters(4, 3, 148_000_000), compute(1_000_000_000, 1, 4, 1)); + Assert.assertEquals(parameters(3, 2, 81_333_333), compute(1_000_000_000, 1, 8, 1)); + Assert.assertEquals(parameters(1, 4, 42_117_647), compute(1_000_000_000, 1, 16, 1)); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> compute(1_000_000_000, 1, 32, 1) + ); + + Assert.assertEquals(new NotEnoughMemoryFault(1_000_000_000, 1, 32), e.getFault()); + } + + @Test + public void test_oneWorkerInJvm_twoHundredWorkersInCluster() + { + Assert.assertEquals(parameters(1, 45, 174_000_000), compute(1_000_000_000, 1, 1, 200)); + Assert.assertEquals(parameters(2, 14, 49_000_000), compute(1_000_000_000, 1, 2, 200)); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> compute(1_000_000_000, 1, 4, 200) + ); + + Assert.assertEquals(new TooManyWorkersFault(200, 124), e.getFault()); + } + + @Test + public void test_fourWorkersInJvm_twoHundredWorkersInCluster() + { + Assert.assertEquals(parameters(1, 149, 999_000_000), compute(8_000_000_000L, 4, 1, 200)); + Assert.assertEquals(parameters(2, 61, 799_000_000), compute(8_000_000_000L, 4, 2, 200)); + Assert.assertEquals(parameters(4, 22, 549_000_000), compute(8_000_000_000L, 4, 4, 200)); + Assert.assertEquals(parameters(4, 14, 299_000_000), compute(8_000_000_000L, 4, 8, 200)); + Assert.assertEquals(parameters(4, 8, 99_000_000), compute(8_000_000_000L, 4, 16, 200)); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> compute(8_000_000_000L, 4, 32, 200) + ); + + Assert.assertEquals(new TooManyWorkersFault(200, 140), e.getFault()); + + // Make sure 140 actually works. (Verify the error message above.) + Assert.assertEquals(parameters(4, 4, 25_666_666), compute(8_000_000_000L, 4, 32, 140)); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(WorkerMemoryParameters.class).usingGetClass().verify(); + } + + private static WorkerMemoryParameters parameters( + final int superSorterMaxActiveProcessors, + final int superSorterMaxChannelsPerProcessor, + final long bundleMemory + ) + { + return new WorkerMemoryParameters( + superSorterMaxActiveProcessors, + superSorterMaxChannelsPerProcessor, + (long) (bundleMemory * WorkerMemoryParameters.APPENDERATOR_MEMORY_FRACTION), + (long) (bundleMemory * WorkerMemoryParameters.BROADCAST_JOIN_MEMORY_FRACTION) + ); + } + + private static WorkerMemoryParameters compute( + final long maxMemoryInJvm, + final int numWorkersInJvm, + final int numProcessingThreadsInJvm, + final int numInputWorkers + ) + { + return WorkerMemoryParameters.compute( + maxMemoryInJvm, + numWorkersInJvm, + numProcessingThreadsInJvm, + numInputWorkers + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/ColumnMappingTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/ColumnMappingTest.java new file mode 100644 index 00000000000..54d740fb73a --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/ColumnMappingTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class ColumnMappingTest +{ + + @Test + public void testEquals() + { + EqualsVerifier.forClass(ColumnMapping.class) + .withNonnullFields("queryColumn", "outputColumn") + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DataSourceMSQDestinationTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DataSourceMSQDestinationTest.java new file mode 100644 index 00000000000..5063c054777 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DataSourceMSQDestinationTest.java @@ -0,0 +1,37 @@ +/* + * 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.druid.msq.indexing; + + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class DataSourceMSQDestinationTest +{ + + @Test + public void testEquals() + { + EqualsVerifier.forClass(DataSourceMSQDestination.class) + .withNonnullFields("dataSource", "segmentGranularity", "segmentSortOrder") + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java new file mode 100644 index 00000000000..257d9b6aafe --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java @@ -0,0 +1,87 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.util.concurrent.Futures; +import com.google.inject.Injector; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.rpc.ServiceLocations; +import org.apache.druid.rpc.ServiceLocator; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Collections; + +public class IndexerWorkerContextTest +{ + + private IndexerWorkerContext indexerWorkerContext = null; + + @Before + public void setup() + { + final Injector injectorMock = Mockito.mock(Injector.class); + Mockito.when(injectorMock.getInstance(SegmentCacheManagerFactory.class)) + .thenReturn(Mockito.mock(SegmentCacheManagerFactory.class)); + + indexerWorkerContext = new IndexerWorkerContext( + Mockito.mock(TaskToolbox.class), + injectorMock, + null, + null, + null + ); + } + + @Test + public void testControllerCheckerRunnableExitsWhenEmptyStatus() + { + final ServiceLocator controllerLocatorMock = Mockito.mock(ServiceLocator.class); + Mockito.when(controllerLocatorMock.locate()) + .thenReturn(Futures.immediateFuture(ServiceLocations.forLocations(Collections.emptySet()))); + + final Worker workerMock = Mockito.mock(Worker.class); + + indexerWorkerContext.controllerCheckerRunnable(controllerLocatorMock, workerMock); + Mockito.verify(controllerLocatorMock, Mockito.times(1)).locate(); + Mockito.verify(workerMock, Mockito.times(1)).controllerFailed(); + } + + @Test + public void testControllerCheckerRunnableExitsOnlyWhenClosedStatus() + { + final ServiceLocator controllerLocatorMock = Mockito.mock(ServiceLocator.class); + Mockito.when(controllerLocatorMock.locate()) + .thenReturn(Futures.immediateFuture(ServiceLocations.forLocation(new ServiceLocation("h", 1, -1, "/")))) + // Done to check the behavior of the runnable, the situation of exiting after success might not occur actually + .thenReturn(Futures.immediateFuture(ServiceLocations.forLocation(new ServiceLocation("h", 1, -1, "/")))) + .thenReturn(Futures.immediateFuture(ServiceLocations.closed())); + + final Worker workerMock = Mockito.mock(Worker.class); + + indexerWorkerContext.controllerCheckerRunnable(controllerLocatorMock, workerMock); + Mockito.verify(controllerLocatorMock, Mockito.times(3)).locate(); + Mockito.verify(workerMock, Mockito.times(1)).controllerFailed(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQPartitionAssignmentTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQPartitionAssignmentTest.java new file mode 100644 index 00000000000..e6aebd5c9e3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQPartitionAssignmentTest.java @@ -0,0 +1,57 @@ +/* + * 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.druid.msq.indexing; + +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; + +import static org.easymock.EasyMock.mock; + +public class MSQPartitionAssignmentTest +{ + + @Test(expected = NullPointerException.class) + public void testNullPartition() + { + new MSQPartitionAssignment(null, Collections.emptyMap()); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidPartition() + { + Map allocations = ImmutableMap.of(-1, mock(SegmentIdWithShardSpec.class)); + new MSQPartitionAssignment(ClusterByPartitions.oneUniversalPartition(), allocations); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(MSQPartitionAssignment.class) + .withNonnullFields("partitions", "allocations") + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java new file mode 100644 index 00000000000..a8f806fb1ca --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQSpecTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class MSQSpecTest +{ + + @Test + public void testEquals() + { + EqualsVerifier.forClass(MSQSpec.class) + .withNonnullFields("query", "destination", "tuningConfig") + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.java new file mode 100644 index 00000000000..82ed89306e9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTuningConfigTest.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.druid.msq.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class MSQTuningConfigTest +{ + @Test + public void testSerdeDefault() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + + Assert.assertEquals( + MSQTuningConfig.defaultConfig(), + mapper.readValue( + mapper.writeValueAsString(MSQTuningConfig.defaultConfig()), + MSQTuningConfig.class + ) + ); + } + + @Test + public void testSerdeNonDefault() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + final MSQTuningConfig config = new MSQTuningConfig(2, 3, 4); + + Assert.assertEquals( + config, + mapper.readValue( + mapper.writeValueAsString(config), + MSQTuningConfig.class + ) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(MSQTuningConfig.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFaultTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFaultTest.java new file mode 100644 index 00000000000..807a86b7775 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/InsertLockPreemptedFaultTest.java @@ -0,0 +1,74 @@ +/* + * 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.druid.msq.indexing.error; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.msq.test.MSQTestTaskActionClient; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Test; + +public class InsertLockPreemptedFaultTest extends MSQTestBase +{ + + @Test + public void testThrowsInsertLockPreemptedFault() + { + LockPreemptedHelper.preempt(true); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + testIngestQuery().setSql( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null " + + "group by 1, 2 PARTITIONED by day clustered by dim1") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedMSQFault(InsertLockPreemptedFault.instance()) + .verifyResults(); + + } + + /** + * Dummy class for {@link MSQTestTaskActionClient} to determine whether + * to grant or preempt the lock. This should be ideally done via injectors + */ + public static class LockPreemptedHelper + { + private static boolean preempted = false; + + public static void preempt(final boolean preempted) + { + LockPreemptedHelper.preempted = preempted; + } + + public static void throwIfPreempted() + { + if (preempted) { + throw new ISE( + "Segments[dummySegment] are not covered by locks[dummyLock] for task[dummyTask]" + ); + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java new file mode 100644 index 00000000000..77abdfc8356 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -0,0 +1,97 @@ +/* + * 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.druid.msq.indexing.error; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.timeline.SegmentId; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +public class MSQFaultSerdeTest +{ + private ObjectMapper objectMapper; + + @Before + public void setUp() + { + objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerModules(new MSQIndexingModule().getJacksonModules()); + objectMapper.enable(JsonParser.Feature.STRICT_DUPLICATE_DETECTION); + } + + @Test + public void testFaultSerde() throws IOException + { + assertFaultSerde(new BroadcastTablesTooLargeFault(10)); + assertFaultSerde(CanceledFault.INSTANCE); + assertFaultSerde(new CannotParseExternalDataFault("the message")); + assertFaultSerde(new ColumnTypeNotSupportedFault("the column", null)); + assertFaultSerde(new ColumnTypeNotSupportedFault("the column", ColumnType.STRING_ARRAY)); + assertFaultSerde(new ColumnNameRestrictedFault("the column")); + assertFaultSerde(new InsertCannotAllocateSegmentFault("the datasource", Intervals.ETERNITY)); + assertFaultSerde(new InsertCannotBeEmptyFault("the datasource")); + assertFaultSerde(new InsertCannotOrderByDescendingFault("the column")); + assertFaultSerde( + new InsertCannotReplaceExistingSegmentFault(SegmentId.of("the datasource", Intervals.ETERNITY, "v1", 1)) + ); + assertFaultSerde(InsertLockPreemptedFault.INSTANCE); + assertFaultSerde(InsertTimeNullFault.INSTANCE); + assertFaultSerde(new InsertTimeOutOfBoundsFault(Intervals.ETERNITY)); + assertFaultSerde(new InvalidNullByteFault("the column")); + assertFaultSerde(new NotEnoughMemoryFault(1000, 1, 2)); + assertFaultSerde(QueryNotSupportedFault.INSTANCE); + assertFaultSerde(new RowTooLargeFault(1000)); + assertFaultSerde(new TaskStartTimeoutFault(10)); + assertFaultSerde(new TooManyBucketsFault(10)); + assertFaultSerde(new TooManyColumnsFault(10, 8)); + assertFaultSerde(new TooManyInputFilesFault(15, 10, 5)); + assertFaultSerde(new TooManyPartitionsFault(10)); + assertFaultSerde(new TooManyWarningsFault(10, "the error")); + assertFaultSerde(new TooManyWorkersFault(10, 5)); + assertFaultSerde(UnknownFault.forMessage(null)); + assertFaultSerde(UnknownFault.forMessage("the message")); + assertFaultSerde(new WorkerFailedFault("the worker task", "the error msg")); + assertFaultSerde(new WorkerRpcFailedFault("the worker task")); + } + + @Test + public void testFaultEqualsAndHashCode() + { + for (Class faultClass : MSQIndexingModule.FAULT_CLASSES) { + EqualsVerifier.forClass(faultClass).usingGetClass().verify(); + } + } + + private void assertFaultSerde(final MSQFault fault) throws IOException + { + final String json = objectMapper.writeValueAsString(fault); + final MSQFault fault2 = objectMapper.readValue(json, MSQFault.class); + Assert.assertEquals(json, fault, fault2); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java new file mode 100644 index 00000000000..e25867b059b --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java @@ -0,0 +1,404 @@ +/* + * 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.druid.msq.indexing.error; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.msq.indexing.ColumnMapping; +import org.apache.druid.msq.indexing.ColumnMappings; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.Query; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.sql.calcite.external.ExternalDataSource; +import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +/** + * unparseable.gz is a file containing 10 valid and 9 invalid records + */ +public class MSQWarningsTest extends MSQTestBase +{ + + private File toRead; + private RowSignature rowSignature; + private String toReadFileNameAsJson; + + private Query defaultQuery; + private ColumnMappings defaultColumnMappings; + + @Before + public void setUp3() throws IOException + { + toRead = getResourceAsTemporaryFile("/unparseable.gz"); + toReadFileNameAsJson = queryJsonMapper.writeValueAsString(toRead.getAbsolutePath()); + + rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("cnt", ColumnType.LONG) + .build(); + + defaultQuery = GroupByQuery.builder() + .setDataSource(new ExternalDataSource( + new LocalInputSource( + null, + null, + ImmutableList.of( + toRead.getAbsoluteFile() + ) + ), + new JsonInputFormat(null, null, null), + RowSignature.builder() + .add("timestamp", ColumnType.STRING) + .add("page", ColumnType.STRING) + .add("user", ColumnType.STRING) + .build() + )) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns(new ExpressionVirtualColumn( + "v0", + "timestamp_floor(timestamp_parse(\"timestamp\",null,'UTC'),'P1D',null,'UTC')", + ColumnType.LONG, + CalciteTests.createExprMacroTable() + )) + .setDimensions(dimensions(new DefaultDimensionSpec( + "v0", + "d0", + ColumnType.LONG + ) + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + defaultColumnMappings = new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "__time"), + new ColumnMapping("a0", "cnt") + ) + ); + } + + + @Test + public void testThrowExceptionWhenParseExceptionsExceedLimit() + { + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext(ImmutableMap.of( + MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0 + )) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1466985600000L, 20L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedMSQFault(new TooManyWarningsFault(0, CannotParseExternalDataFault.CODE)) + .verifyResults(); + } + + @Test + public void testSuccessWhenNoLimitEnforced() + { + final Map userContext = ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, -1); + + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext( + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(userContext) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1566172800000L, 10L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery.withOverriddenContext(userContext)) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .verifyResults(); + } + + @Test + public void testInvalidMaxParseExceptionsPassed() + { + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext(ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, -2)) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1566172800000L, 10L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedMSQFault(UnknownFault.forMessage( + "java.lang.IllegalArgumentException: " + + "Invalid limit of -2 supplied for warnings of type CannotParseExternalData. " + + "Limit can be greater than or equal to -1.")) + .verifyResults(); + } + + @Test + public void testFailureWhenParseExceptionsExceedPositiveLimit() + { + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext(ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 4)) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1466985600000L, 20L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedMSQFault(new TooManyWarningsFault(4, CannotParseExternalDataFault.CODE)) + .verifyResults(); + } + + + @Test + public void testSuccessWhenParseExceptionsOnLimit() + { + final Map userContext = ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 10); + + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext( + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(userContext) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1566172800000L, 10L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery.withOverriddenContext(userContext)) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .verifyResults(); + } + + @Test + public void testSuccessInNonStrictMode() + { + final Map userContext = ImmutableMap.of(MultiStageQueryContext.CTX_MSQ_MODE, "nonStrict"); + + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext( + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(userContext) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1566172800000L, 10L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery.withOverriddenContext(userContext)) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .verifyResults(); + } + + + @Test + public void testFailureInStrictMode() + { + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext(ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0)) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1466985600000L, 20L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedMSQFault(new TooManyWarningsFault(0, CannotParseExternalDataFault.CODE)) + .verifyResults(); + } + + @Test + public void testDefaultStrictMode() + { + testIngestQuery().setSql(" insert into foo1 SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1 PARTITIONED by day ") + .setQueryContext(ROLLUP_CONTEXT) + .setExpectedRollUp(true) + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) + .setExpectedMSQFault(new TooManyWarningsFault(0, CannotParseExternalDataFault.CODE)) + .verifyResults(); + } + + @Test + public void testControllerTemporaryFileCleanup() + { + testIngestQuery().setSql(" insert into foo1 SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1 PARTITIONED by day ") + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedMSQFault(new TooManyWarningsFault(0, CannotParseExternalDataFault.CODE)) + .verifyResults(); + + // Temporary directory should not contain any controller-related folders + Assert.assertEquals(0, localFileStorageDir.listFiles().length); + } + + @Test + public void testSuccessWhenModeIsOverridden() + { + final Map userContext = + ImmutableMap.builder() + .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, -1) + .put(MultiStageQueryContext.CTX_MSQ_MODE, "strict") + .build(); + + testSelectQuery().setSql("SELECT\n" + + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" + + " count(*) as cnt\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [\"" + toRead.getAbsolutePath() + "\"],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") group by 1") + .setQueryContext( + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(userContext) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1566172800000L, 10L})) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(defaultQuery.withOverriddenContext(userContext)) + .columnMappings(defaultColumnMappings) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .verifyResults(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java new file mode 100644 index 00000000000..2e4fc794cf9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -0,0 +1,214 @@ +/* + * 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.druid.msq.indexing.report; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.error.TooManyColumnsFault; +import org.apache.druid.msq.kernel.MaxCountShuffleSpec; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class MSQTaskReportTest +{ + private static final String TASK_ID = "mytask"; + private static final String HOST = "example.com:1234"; + private static final QueryDefinition QUERY_DEFINITION = + QueryDefinition + .builder() + .add( + StageDefinition + .builder(0) + .processorFactory(new OffsetLimitFrameProcessorFactory(0, 1L)) + .shuffleSpec( + new MaxCountShuffleSpec( + new ClusterBy(ImmutableList.of(new SortColumn("s", false)), 0), + 2, + false + ) + ) + .maxWorkerCount(3) + .signature(RowSignature.builder().add("s", ColumnType.STRING).build()) + ) + .build(); + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void testSerdeResultsReport() throws Exception + { + final List results = ImmutableList.of( + new Object[]{"foo"}, + new Object[]{"bar"} + ); + + final MSQTaskReport report = new MSQTaskReport( + TASK_ID, + new MSQTaskReportPayload( + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0), + MSQStagesReport.create( + QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + new MSQResultsReport( + RowSignature.builder().add("s", ColumnType.STRING).build(), + ImmutableList.of("VARCHAR"), + Yielders.each(Sequences.simple(results)) + ) + ) + ); + + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final MSQTaskReport report2 = (MSQTaskReport) mapper.readValue( + mapper.writeValueAsString(report), + TaskReport.class + ); + + Assert.assertEquals(TASK_ID, report2.getTaskId()); + Assert.assertEquals(report.getPayload().getStatus().getStatus(), report2.getPayload().getStatus().getStatus()); + Assert.assertNull(report2.getPayload().getStatus().getErrorReport()); + Assert.assertEquals(report.getPayload().getStages(), report2.getPayload().getStages()); + + Yielder yielder = report2.getPayload().getResults().getResultYielder(); + final List results2 = new ArrayList<>(); + + while (!yielder.isDone()) { + results2.add(yielder.get()); + yielder = yielder.next(null); + } + + Assert.assertEquals(results.size(), results2.size()); + for (int i = 0; i < results.size(); i++) { + Assert.assertArrayEquals(results.get(i), results2.get(i)); + } + } + + @Test + public void testSerdeErrorReport() throws Exception + { + final MSQErrorReport errorReport = MSQErrorReport.fromFault(TASK_ID, HOST, 0, new TooManyColumnsFault(10, 5)); + final MSQTaskReport report = new MSQTaskReport( + TASK_ID, + new MSQTaskReportPayload( + new MSQStatusReport(TaskState.FAILED, errorReport, new ArrayDeque<>(), null, 0), + MSQStagesReport.create( + QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + null + ) + ); + + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final MSQTaskReport report2 = (MSQTaskReport) mapper.readValue( + mapper.writeValueAsString(report), + TaskReport.class + ); + + Assert.assertEquals(TASK_ID, report2.getTaskId()); + Assert.assertEquals(report.getPayload().getStatus().getStatus(), report2.getPayload().getStatus().getStatus()); + Assert.assertEquals( + report.getPayload().getStatus().getErrorReport(), + report2.getPayload().getStatus().getErrorReport() + ); + Assert.assertEquals(report.getPayload().getStages(), report2.getPayload().getStages()); + } + + @Test + @Ignore("requires https://github.com/apache/druid/pull/12938") + public void testWriteTaskReport() throws Exception + { + final MSQTaskReport report = new MSQTaskReport( + TASK_ID, + new MSQTaskReportPayload( + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0), + MSQStagesReport.create( + QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + null + ) + ); + + final File reportFile = temporaryFolder.newFile(); + final SingleFileTaskReportFileWriter writer = new SingleFileTaskReportFileWriter(reportFile); + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + writer.setObjectMapper(mapper); + writer.write(TASK_ID, TaskReport.buildTaskReports(report)); + + final Map reportMap = mapper.readValue( + reportFile, + new TypeReference>() {} + ); + + final MSQTaskReport report2 = (MSQTaskReport) reportMap.get(MSQTaskReport.REPORT_KEY); + + Assert.assertEquals(TASK_ID, report2.getTaskId()); + Assert.assertEquals(report.getPayload().getStatus().getStatus(), report2.getPayload().getStatus().getStatus()); + Assert.assertEquals(report.getPayload().getStages(), report2.getPayload().getStages()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java new file mode 100644 index 00000000000..94fb76f8064 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java @@ -0,0 +1,43 @@ +/* + * 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.druid.msq.input; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.junit.Assert; +import org.junit.Test; + +public class InputSpecsTest +{ + @Test + public void test_getStageNumbers() + { + Assert.assertEquals( + ImmutableSet.of(1, 2), + InputSpecs.getStageNumbers( + ImmutableList.of( + new StageInputSpec(1), + new StageInputSpec(2) + ) + ) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/NilInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/NilInputSliceTest.java new file mode 100644 index 00000000000..f7ff529f8ee --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/NilInputSliceTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class NilInputSliceTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final NilInputSlice slice = NilInputSlice.INSTANCE; + + Assert.assertEquals( + slice, + mapper.readValue(mapper.writeValueAsString(slice), InputSlice.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(NilInputSlice.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSliceTest.java new file mode 100644 index 00000000000..b99eed1ca52 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSliceTest.java @@ -0,0 +1,65 @@ +/* + * 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.druid.msq.input.external; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.util.Collections; + +public class ExternalInputSliceTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final ExternalInputSlice slice = new ExternalInputSlice( + Collections.singletonList( + new LocalInputSource( + null, + null, + Collections.singletonList(new File("/nonexistent/file")) + ) + ), + ExternalInputSpecSlicerTest.INPUT_FORMAT, + ExternalInputSpecSlicerTest.SIGNATURE + ); + + Assert.assertEquals( + slice, + mapper.readValue(mapper.writeValueAsString(slice), InputSlice.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(ExternalInputSlice.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicerTest.java new file mode 100644 index 00000000000..186cb0b9082 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicerTest.java @@ -0,0 +1,320 @@ +/* + * 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.druid.msq.input.external; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputFileAttribute; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.utils.Streams; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ExternalInputSpecSlicerTest +{ + static final InputFormat INPUT_FORMAT = new JsonInputFormat(null, null, null); + static final RowSignature SIGNATURE = RowSignature.builder().add("s", ColumnType.STRING).build(); + + private ExternalInputSpecSlicer slicer; + + @Before + public void setUp() + { + slicer = new ExternalInputSpecSlicer(); + } + + @Test + public void test_canSliceDynamic_splittable() + { + Assert.assertTrue(slicer.canSliceDynamic(splittableSpec())); + } + + @Test + public void test_canSliceDynamic_unsplittable() + { + Assert.assertFalse(slicer.canSliceDynamic(unsplittableSpec())); + } + + @Test + public void test_sliceStatic_unsplittable() + { + Assert.assertEquals( + ImmutableList.of( + unsplittableSlice("foo", "bar", "baz"), + NilInputSlice.INSTANCE + ), + slicer.sliceStatic(unsplittableSpec("foo", "bar", "baz"), 2) + ); + } + + @Test + public void test_sliceStatic_splittable() + { + Assert.assertEquals( + ImmutableList.of( + splittableSlice("foo", "baz"), + splittableSlice("bar") + ), + slicer.sliceStatic(splittableSpec("foo", "bar", "baz"), 2) + ); + } + + @Test + public void test_sliceDynamic_unsplittable() + { + Assert.assertEquals( + ImmutableList.of( + unsplittableSlice("foo", "bar", "baz") + ), + slicer.sliceDynamic(unsplittableSpec("foo", "bar", "baz"), 100, 1, 1) + ); + } + + @Test + public void test_sliceDynamic_splittable_needOne() + { + Assert.assertEquals( + ImmutableList.of( + splittableSlice("foo", "bar", "baz") + ), + slicer.sliceDynamic(splittableSpec("foo", "bar", "baz"), 100, 5, Long.MAX_VALUE) + ); + } + + @Test + public void test_sliceDynamic_splittable_needTwoDueToFiles() + { + Assert.assertEquals( + ImmutableList.of( + splittableSlice("foo", "baz"), + splittableSlice("bar") + ), + slicer.sliceDynamic(splittableSpec("foo", "bar", "baz"), 100, 2, Long.MAX_VALUE) + ); + } + + @Test + public void test_sliceDynamic_splittable_needTwoDueToBytes() + { + Assert.assertEquals( + ImmutableList.of( + splittableSlice("foo", "baz"), + splittableSlice("bar") + ), + slicer.sliceDynamic(splittableSpec("foo", "bar", "baz"), 100, 5, 7) + ); + } + + static ExternalInputSpec splittableSpec(final String... strings) + { + return new ExternalInputSpec( + new TestSplittableInputSource(Arrays.asList(strings)), + INPUT_FORMAT, + SIGNATURE + ); + } + + static ExternalInputSpec unsplittableSpec(final String... strings) + { + return new ExternalInputSpec( + new TestUnsplittableInputSource(Arrays.asList(strings)), + INPUT_FORMAT, + SIGNATURE + ); + } + + static ExternalInputSlice splittableSlice(final String... strings) + { + return new ExternalInputSlice( + Stream.of(strings) + .map(s -> new TestSplittableInputSource(Collections.singletonList(s))) + .collect(Collectors.toList()), + INPUT_FORMAT, + SIGNATURE + ); + } + + static ExternalInputSlice unsplittableSlice(final String... strings) + { + return new ExternalInputSlice( + Collections.singletonList(new TestUnsplittableInputSource(Arrays.asList(strings))), + INPUT_FORMAT, + SIGNATURE + ); + } + + private static class TestUnsplittableInputSource implements InputSource + { + private final List strings; + + public TestUnsplittableInputSource(final List strings) + { + this.strings = strings; + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + File temporaryDirectory + ) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestUnsplittableInputSource that = (TestUnsplittableInputSource) o; + return Objects.equals(strings, that.strings); + } + + @Override + public int hashCode() + { + return Objects.hash(strings); + } + + @Override + public String toString() + { + return "TestUnsplittableInputSource{" + + "strings=" + strings + + '}'; + } + } + + private static class TestSplittableInputSource implements SplittableInputSource> + { + private final List strings; + + public TestSplittableInputSource(final List strings) + { + this.strings = strings; + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + File temporaryDirectory + ) + { + throw new UnsupportedOperationException(); + } + + @Override + public Stream>> createSplits( + InputFormat inputFormat, + @Nullable SplitHintSpec splitHintSpec + ) + { + final Iterator> splits = splitHintSpec.split( + strings.iterator(), + s -> new InputFileAttribute(s.length()) + ); + + return Streams.sequentialStreamFrom(splits).map(InputSplit::new); + } + + @Override + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public InputSource withSplit(InputSplit> split) + { + return new TestSplittableInputSource(split.get()); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestSplittableInputSource that = (TestSplittableInputSource) o; + return Objects.equals(strings, that.strings); + } + + @Override + public int hashCode() + { + return Objects.hash(strings); + } + + @Override + public String toString() + { + return "TestSplittableInputSource{" + + "strings=" + strings + + '}'; + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecTest.java new file mode 100644 index 00000000000..878e29ecc3a --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/external/ExternalInputSpecTest.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.druid.msq.input.external; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.util.Collections; + +public class ExternalInputSpecTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final ExternalInputSpec spec = new ExternalInputSpec( + new LocalInputSource( + null, + null, + Collections.singletonList(new File("/nonexistent/file")) + ), + ExternalInputSpecSlicerTest.INPUT_FORMAT, + ExternalInputSpecSlicerTest.SIGNATURE + ); + + Assert.assertEquals( + spec, + mapper.readValue(mapper.writeValueAsString(spec), InputSpec.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(ExternalInputSpec.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CollectedReadablePartitionsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CollectedReadablePartitionsTest.java new file mode 100644 index 00000000000..6ed7d2d43d4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CollectedReadablePartitionsTest.java @@ -0,0 +1,83 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class CollectedReadablePartitionsTest +{ + @Test + public void testPartitionToWorkerMap() + { + final CollectedReadablePartitions partitions = ReadablePartitions.collected(1, ImmutableMap.of(0, 1, 1, 2, 2, 1)); + Assert.assertEquals(ImmutableMap.of(0, 1, 1, 2, 2, 1), partitions.getPartitionToWorkerMap()); + } + + @Test + public void testStageNumber() + { + final CollectedReadablePartitions partitions = ReadablePartitions.collected(1, ImmutableMap.of(0, 1, 1, 2, 2, 1)); + Assert.assertEquals(1, partitions.getStageNumber()); + } + + @Test + public void testSplit() + { + final CollectedReadablePartitions partitions = ReadablePartitions.collected(1, ImmutableMap.of(0, 1, 1, 2, 2, 1)); + + Assert.assertEquals( + ImmutableList.of( + ReadablePartitions.collected(1, ImmutableMap.of(0, 1, 2, 1)), + ReadablePartitions.collected(1, ImmutableMap.of(1, 2)) + ), + partitions.split(2) + ); + } + + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final CollectedReadablePartitions partitions = ReadablePartitions.collected(1, ImmutableMap.of(0, 1, 1, 2, 2, 1)); + + Assert.assertEquals( + partitions, + mapper.readValue( + mapper.writeValueAsString(partitions), + ReadablePartitions.class + ) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(CollectedReadablePartitions.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CombinedReadablePartitionsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CombinedReadablePartitionsTest.java new file mode 100644 index 00000000000..685f4ff7a8a --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/CombinedReadablePartitionsTest.java @@ -0,0 +1,95 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class CombinedReadablePartitionsTest +{ + private static final CombinedReadablePartitions PARTITIONS = ReadablePartitions.combine( + ImmutableList.of( + ReadablePartitions.striped(0, 2, 2), + ReadablePartitions.striped(1, 2, 4) + ) + ); + + @Test + public void testEmpty() + { + Assert.assertEquals(0, Iterables.size(ReadablePartitions.empty())); + } + + @Test + public void testSplitOne() + { + Assert.assertEquals(ImmutableList.of(PARTITIONS), PARTITIONS.split(1)); + } + + @Test + public void testSplitTwo() + { + Assert.assertEquals( + ImmutableList.of( + ReadablePartitions.combine( + ImmutableList.of( + new StripedReadablePartitions(0, 2, new IntAVLTreeSet(new int[]{0})), + new StripedReadablePartitions(1, 2, new IntAVLTreeSet(new int[]{0, 2})) + ) + ), + ReadablePartitions.combine( + ImmutableList.of( + new StripedReadablePartitions(0, 2, new IntAVLTreeSet(new int[]{1})), + new StripedReadablePartitions(1, 2, new IntAVLTreeSet(new int[]{1, 3})) + ) + ) + ), + PARTITIONS.split(2) + ); + } + + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + Assert.assertEquals( + PARTITIONS, + mapper.readValue( + mapper.writeValueAsString(PARTITIONS), + ReadablePartitions.class + ) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(CombinedReadablePartitions.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/ReadablePartitionTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/ReadablePartitionTest.java new file mode 100644 index 00000000000..659fcaa9db1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/ReadablePartitionTest.java @@ -0,0 +1,32 @@ +/* + * 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.druid.msq.input.stage; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class ReadablePartitionTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(ReadablePartition.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSliceTest.java new file mode 100644 index 00000000000..3b2705c8ba6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSliceTest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.input.stage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class StageInputSliceTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final StageInputSlice slice = new StageInputSlice( + 2, + ReadablePartitions.striped(2, 3, 4) + ); + + Assert.assertEquals( + slice, + mapper.readValue(mapper.writeValueAsString(slice), InputSlice.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(StageInputSlice.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecSlicerTest.java new file mode 100644 index 00000000000..43d89e7fc69 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecSlicerTest.java @@ -0,0 +1,120 @@ +/* + * 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.druid.msq.input.stage; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +public class StageInputSpecSlicerTest +{ + private static final Int2ObjectMap STAGE_PARTITIONS_MAP = + new Int2ObjectOpenHashMap<>( + ImmutableMap.builder() + .put(0, ReadablePartitions.striped(0, 2, 2)) + .put(1, ReadablePartitions.striped(1, 2, 4)) + .put(2, ReadablePartitions.striped(2, 2, 4)) + .build() + ); + + private StageInputSpecSlicer slicer; + + @Before + public void setUp() + { + slicer = new StageInputSpecSlicer(STAGE_PARTITIONS_MAP); + } + + @Test + public void test_canSliceDynamic() + { + Assert.assertFalse(slicer.canSliceDynamic(new StageInputSpec(0))); + } + + @Test + public void test_sliceStatic_stageZeroOneSlice() + { + Assert.assertEquals( + Collections.singletonList( + new StageInputSlice( + 0, + ReadablePartitions.striped(0, 2, 2) + ) + ), + slicer.sliceStatic(new StageInputSpec(0), 1) + ); + } + + @Test + public void test_sliceStatic_stageZeroTwoSlices() + { + Assert.assertEquals( + ImmutableList.of( + new StageInputSlice( + 0, + new StripedReadablePartitions(0, 2, new IntAVLTreeSet(new int[]{0})) + ), + new StageInputSlice( + 0, + new StripedReadablePartitions(0, 2, new IntAVLTreeSet(new int[]{1})) + ) + ), + slicer.sliceStatic(new StageInputSpec(0), 2) + ); + } + + @Test + public void test_sliceStatic_stageOneTwoSlices() + { + Assert.assertEquals( + ImmutableList.of( + new StageInputSlice( + 1, + new StripedReadablePartitions(1, 2, new IntAVLTreeSet(new int[]{0, 2})) + ), + new StageInputSlice( + 1, + new StripedReadablePartitions(1, 2, new IntAVLTreeSet(new int[]{1, 3})) + ) + ), + slicer.sliceStatic(new StageInputSpec(1), 2) + ); + } + + @Test + public void test_sliceStatic_notAvailable() + { + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> slicer.sliceStatic(new StageInputSpec(3), 1) + ); + + MatcherAssert.assertThat(e.getMessage(), CoreMatchers.equalTo("Stage [3] not available")); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecTest.java new file mode 100644 index 00000000000..3f3bc6f7911 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StageInputSpecTest.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.druid.msq.input.stage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class StageInputSpecTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final StageInputSpec spec = new StageInputSpec(2); + + Assert.assertEquals( + spec, + mapper.readValue(mapper.writeValueAsString(spec), InputSpec.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(StageInputSpec.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StripedReadablePartitionsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StripedReadablePartitionsTest.java new file mode 100644 index 00000000000..38e0707f5d0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/stage/StripedReadablePartitionsTest.java @@ -0,0 +1,91 @@ +/* + * 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.druid.msq.input.stage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class StripedReadablePartitionsTest +{ + @Test + public void testPartitionNumbers() + { + final StripedReadablePartitions partitions = ReadablePartitions.striped(1, 2, 3); + Assert.assertEquals(ImmutableSet.of(0, 1, 2), partitions.getPartitionNumbers()); + } + + @Test + public void testNumWorkers() + { + final StripedReadablePartitions partitions = ReadablePartitions.striped(1, 2, 3); + Assert.assertEquals(2, partitions.getNumWorkers()); + } + + @Test + public void testStageNumber() + { + final StripedReadablePartitions partitions = ReadablePartitions.striped(1, 2, 3); + Assert.assertEquals(1, partitions.getStageNumber()); + } + + @Test + public void testSplit() + { + final StripedReadablePartitions partitions = ReadablePartitions.striped(1, 2, 3); + + Assert.assertEquals( + ImmutableList.of( + new StripedReadablePartitions(1, 2, new IntAVLTreeSet(new int[]{0, 2})), + new StripedReadablePartitions(1, 2, new IntAVLTreeSet(new int[]{1})) + ), + partitions.split(2) + ); + } + + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final StripedReadablePartitions partitions = ReadablePartitions.striped(1, 2, 3); + + Assert.assertEquals( + partitions, + mapper.readValue( + mapper.writeValueAsString(partitions), + ReadablePartitions.class + ) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(StripedReadablePartitions.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java new file mode 100644 index 00000000000..8884a92a665 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java @@ -0,0 +1,88 @@ +/* + * 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.druid.msq.input.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class RichSegmentDescriptorTest +{ + @Test + public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + final RichSegmentDescriptor descriptor = new RichSegmentDescriptor( + Intervals.of("2000/2002"), + Intervals.of("2000/2001"), + "2", + 3 + ); + + Assert.assertEquals( + descriptor, + mapper.readValue(mapper.writeValueAsString(descriptor), RichSegmentDescriptor.class) + ); + } + + @Test + public void testSerdeWithFullIntervalSameAsInterval() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + final RichSegmentDescriptor descriptor = new RichSegmentDescriptor( + Intervals.of("2000/2001"), + Intervals.of("2000/2001"), + "2", + 3 + ); + + Assert.assertEquals( + descriptor, + mapper.readValue(mapper.writeValueAsString(descriptor), RichSegmentDescriptor.class) + ); + } + + @Test + public void testDeserializeRichSegmentDescriptorAsSegmentDescriptor() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + final RichSegmentDescriptor descriptor = new RichSegmentDescriptor( + Intervals.of("2000/2002"), + Intervals.of("2000/2001"), + "2", + 3 + ); + + Assert.assertEquals( + new SegmentDescriptor(Intervals.of("2000/2001"), "2", 3), + mapper.readValue(mapper.writeValueAsString(descriptor), SegmentDescriptor.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(RichSegmentDescriptor.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java new file mode 100644 index 00000000000..29a0ebef4ba --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java @@ -0,0 +1,32 @@ +/* + * 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.druid.msq.input.table; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class SegmentWithDescriptorTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(SegmentWithDescriptor.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java new file mode 100644 index 00000000000..df2937f3003 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.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.druid.msq.input.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class SegmentsInputSliceTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final SegmentsInputSlice slice = new SegmentsInputSlice( + "myds", + ImmutableList.of( + new RichSegmentDescriptor( + Intervals.of("2000/P1M"), + Intervals.of("2000/P1M"), + "1", + 0 + ) + ) + ); + + Assert.assertEquals( + slice, + mapper.readValue(mapper.writeValueAsString(slice), InputSlice.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(SegmentsInputSlice.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java new file mode 100644 index 00000000000..fe55fa1327b --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java @@ -0,0 +1,493 @@ +/* + * 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.druid.msq.input.table; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.StringTuple; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.querykit.DataSegmentTimelineView; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.DimensionRangeShardSpec; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.Optional; + +public class TableInputSpecSlicerTest extends InitializedNullHandlingTest +{ + private static final String DATASOURCE = "test-ds"; + private static final long BYTES_PER_SEGMENT = 1000; + + private static final DataSegment SEGMENT1 = new DataSegment( + DATASOURCE, + Intervals.of("2000/2001"), + "1", + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + new DimensionRangeShardSpec( + ImmutableList.of("dim"), + null, + new StringTuple(new String[]{"foo"}), + 0, + 2 + ), + null, + null, + BYTES_PER_SEGMENT + ); + + private static final DataSegment SEGMENT2 = new DataSegment( + DATASOURCE, + Intervals.of("2000/2001"), + "1", + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + new DimensionRangeShardSpec( + ImmutableList.of("dim"), + new StringTuple(new String[]{"foo"}), + null, + 1, + 2 + ), + null, + null, + BYTES_PER_SEGMENT + ); + + private VersionedIntervalTimeline timeline; + private DataSegmentTimelineView timelineView; + private TableInputSpecSlicer slicer; + + @Before + public void setUp() + { + timeline = VersionedIntervalTimeline.forSegments(ImmutableList.of(SEGMENT1, SEGMENT2)); + timelineView = (dataSource, intervals) -> { + if (DATASOURCE.equals(dataSource)) { + return Optional.of(timeline); + } else { + return Optional.empty(); + } + }; + slicer = new TableInputSpecSlicer(timelineView); + } + + @Test + public void test_canSliceDynamic() + { + Assert.assertTrue(slicer.canSliceDynamic(new TableInputSpec(DATASOURCE, null, null))); + } + + @Test + public void test_sliceStatic_noDataSource() + { + final TableInputSpec spec = new TableInputSpec("no such datasource", null, null); + Assert.assertEquals(Collections.emptyList(), slicer.sliceStatic(spec, 2)); + } + + @Test + public void test_sliceStatic_intervalFilter() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + ImmutableList.of( + Intervals.of("2000/P1M"), + Intervals.of("2000-06-01/P1M") + ), + null + ); + + Assert.assertEquals( + Collections.singletonList( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ), + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ), + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000-06-01/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ), + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + Intervals.of("2000-06-01/P1M"), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceStatic(spec, 1) + ); + } + + @Test + public void test_sliceStatic_intervalFilterMatchesNothing() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + Collections.singletonList(Intervals.of("2002/P1M")), + null + ); + + Assert.assertEquals(Collections.emptyList(), slicer.sliceStatic(spec, 2)); + } + + @Test + public void test_sliceStatic_dimFilter() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + null, + new SelectorDimFilter("dim", "bar", null) + ); + + Assert.assertEquals( + ImmutableList.of( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + SEGMENT1.getInterval(), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ) + ) + ), + NilInputSlice.INSTANCE + ), + slicer.sliceStatic(spec, 2) + ); + } + + @Test + public void test_sliceStatic_intervalAndDimFilter() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + ImmutableList.of( + Intervals.of("2000/P1M"), + Intervals.of("2000-06-01/P1M") + ), + new SelectorDimFilter("dim", "bar", null) + ); + + Assert.assertEquals( + ImmutableList.of( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ) + ) + ), + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000-06-01/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceStatic(spec, 2) + ); + } + + @Test + public void test_sliceStatic_oneSlice() + { + final TableInputSpec spec = new TableInputSpec(DATASOURCE, null, null); + Assert.assertEquals( + Collections.singletonList( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + SEGMENT1.getInterval(), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ), + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + SEGMENT2.getInterval(), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceStatic(spec, 1) + ); + } + + @Test + public void test_sliceStatic_needTwoSlices() + { + final TableInputSpec spec = new TableInputSpec(DATASOURCE, null, null); + Assert.assertEquals( + ImmutableList.of( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + SEGMENT1.getInterval(), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ) + ) + ), + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + SEGMENT2.getInterval(), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceStatic(spec, 2) + ); + } + + @Test + public void test_sliceStatic_threeSlices() + { + final TableInputSpec spec = new TableInputSpec(DATASOURCE, null, null); + Assert.assertEquals( + ImmutableList.of( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + SEGMENT1.getInterval(), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ) + ) + ), + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + SEGMENT2.getInterval(), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ), + NilInputSlice.INSTANCE + ), + slicer.sliceStatic(spec, 3) + ); + } + + @Test + public void test_sliceDynamic_none() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + ImmutableList.of(Intervals.of("2002/P1M")), + null + ); + + Assert.assertEquals( + Collections.emptyList(), + slicer.sliceDynamic(spec, 1, 1, 1) + ); + } + + @Test + public void test_sliceDynamic_maxOneSlice() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + ImmutableList.of(Intervals.of("2000/P1M")), + null + ); + + Assert.assertEquals( + Collections.singletonList( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ), + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceDynamic(spec, 1, 1, 1) + ); + } + + @Test + public void test_sliceDynamic_needOne() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + ImmutableList.of(Intervals.of("2000/P1M")), + null + ); + + Assert.assertEquals( + Collections.singletonList( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ), + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceDynamic(spec, 100, 5, BYTES_PER_SEGMENT * 5) + ); + } + + @Test + public void test_sliceDynamic_needTwoDueToFiles() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + ImmutableList.of(Intervals.of("2000/P1M")), + null + ); + + Assert.assertEquals( + ImmutableList.of( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ) + ) + ), + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceDynamic(spec, 100, 1, BYTES_PER_SEGMENT * 5) + ); + } + + @Test + public void test_sliceDynamic_needTwoDueToBytes() + { + final TableInputSpec spec = new TableInputSpec( + DATASOURCE, + ImmutableList.of(Intervals.of("2000/P1M")), + null + ); + + Assert.assertEquals( + ImmutableList.of( + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT1.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT1.getVersion(), + SEGMENT1.getShardSpec().getPartitionNum() + ) + ) + ), + new SegmentsInputSlice( + DATASOURCE, + ImmutableList.of( + new RichSegmentDescriptor( + SEGMENT2.getInterval(), + Intervals.of("2000/P1M"), + SEGMENT2.getVersion(), + SEGMENT2.getShardSpec().getPartitionNum() + ) + ) + ) + ), + slicer.sliceDynamic(spec, 100, 5, BYTES_PER_SEGMENT) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecTest.java new file mode 100644 index 00000000000..10aba22939b --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecTest.java @@ -0,0 +1,78 @@ +/* + * 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.druid.msq.input.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class TableInputSpecTest extends InitializedNullHandlingTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final TableInputSpec spec = new TableInputSpec( + "myds", + Collections.singletonList(Intervals.of("2000/P1M")), + new SelectorDimFilter("dim", "val", null) + ); + + Assert.assertEquals( + spec, + mapper.readValue(mapper.writeValueAsString(spec), InputSpec.class) + ); + } + + @Test + public void testSerdeEternityInterval() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + final TableInputSpec spec = new TableInputSpec( + "myds", + Intervals.ONLY_ETERNITY, + new SelectorDimFilter("dim", "val", null) + ); + + Assert.assertEquals( + spec, + mapper.readValue(mapper.writeValueAsString(spec), InputSpec.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(TableInputSpec.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/QueryDefinitionTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/QueryDefinitionTest.java new file mode 100644 index 00000000000..fc97fcd708d --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/QueryDefinitionTest.java @@ -0,0 +1,76 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Assert; +import org.junit.Test; + +public class QueryDefinitionTest +{ + @Test + public void testSerde() throws Exception + { + final QueryDefinition queryDef = + QueryDefinition + .builder() + .add( + StageDefinition + .builder(0) + .processorFactory(new OffsetLimitFrameProcessorFactory(0, 1L)) + .shuffleSpec( + new MaxCountShuffleSpec( + new ClusterBy(ImmutableList.of(new SortColumn("s", false)), 0), + 2, + false + ) + ) + .maxWorkerCount(3) + .signature(RowSignature.builder().add("s", ColumnType.STRING).build()) + ) + .build(); + + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + Assert.assertEquals( + queryDef, + mapper.readValue(mapper.writeValueAsString(queryDef), QueryDefinition.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(QueryDefinition.class) + .withNonnullFields("stageDefinitions", "finalStage") + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.java new file mode 100644 index 00000000000..b93cebd6fce --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.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.druid.msq.kernel; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class StageDefinitionTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(StageDefinition.class) + .withIgnoredFields("frameReader") + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageIdTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageIdTest.java new file mode 100644 index 00000000000..aec742f3972 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageIdTest.java @@ -0,0 +1,76 @@ +/* + * 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.druid.msq.kernel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Ordering; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class StageIdTest +{ + @Test + public void testCompareTo() + { + final List sortedStageIds = Ordering.natural().sortedCopy( + ImmutableList.of( + new StageId("xyz", 3), + new StageId("xyz", 1), + new StageId("xyz", 2), + new StageId("abc", 2), + new StageId("abc", 1) + ) + ); + + Assert.assertEquals( + ImmutableList.of( + new StageId("abc", 1), + new StageId("abc", 2), + new StageId("xyz", 1), + new StageId("xyz", 2), + new StageId("xyz", 3) + ), + sortedStageIds + ); + } + + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + final StageId stageId = new StageId("abc", 1); + + Assert.assertEquals( + stageId, + mapper.readValue(mapper.writeValueAsString(stageId), StageId.class) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(StageId.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/WorkOrderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/WorkOrderTest.java new file mode 100644 index 00000000000..b57dca63353 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/WorkOrderTest.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.druid.msq.kernel; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class WorkOrderTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(WorkOrder.class) + .withNonnullFields("queryDefinition", "stageNumber", "workerNumber", "workerInputs") + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java new file mode 100644 index 00000000000..55fd6b75d09 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -0,0 +1,320 @@ +/* + * 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.druid.msq.kernel.controller; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.frame.key.KeyTestUtils; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.input.InputSpecSlicerFactory; +import org.apache.druid.msq.input.MapInputSpecSlicer; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.input.stage.StageInputSpecSlicer; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.testing.InitializedNullHandlingTest; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +public class BaseControllerQueryKernelTest extends InitializedNullHandlingTest +{ + + public ControllerQueryKernelTester testControllerQueryKernel(int numWorkers) + { + return new ControllerQueryKernelTester(numWorkers); + } + + /** + * A tester is broken into 2 phases + * 1. Before calling the init() - The unit tests can set up the controller DAG and the initial stages arbitrarily + * 2. After calling the init() - The unit tests must use the public interface of {@link ControllerQueryKernel} to drive + * the state machine forward and make assertions on the expected vs the actual state + */ + public static class ControllerQueryKernelTester + { + private boolean initialized = false; + private QueryDefinition queryDefinition = null; + private ControllerQueryKernel controllerQueryKernel = null; + private InputSpecSlicerFactory inputSlicerFactory = + stagePartitionsMap -> + new MapInputSpecSlicer( + ImmutableMap.of( + StageInputSpec.class, new StageInputSpecSlicer(stagePartitionsMap), + ControllerTestInputSpec.class, new ControllerTestInputSpecSlicer() + ) + ); + private final int numWorkers; + Set setupStages = new HashSet<>(); + + private ControllerQueryKernelTester(int numWorkers) + { + this.numWorkers = numWorkers; + } + + public ControllerQueryKernelTester queryDefinition(QueryDefinition queryDefinition) + { + this.queryDefinition = Preconditions.checkNotNull(queryDefinition); + this.controllerQueryKernel = new ControllerQueryKernel(queryDefinition); + return this; + } + + + public ControllerQueryKernelTester setupStage( + int stageNumber, + ControllerStagePhase controllerStagePhase + ) + { + return setupStage(stageNumber, controllerStagePhase, false); + } + + public ControllerQueryKernelTester setupStage( + int stageNumber, + ControllerStagePhase controllerStagePhase, + boolean recursiveCall + ) + { + Preconditions.checkNotNull(queryDefinition, "queryDefinition must be supplied before setting up stage"); + Preconditions.checkArgument(!initialized, "setupStage() can only be called pre init()"); + if (setupStages.contains(stageNumber)) { + throw new ISE("A stage can only be setup once"); + } + // Iniitalize the kernels that maybe necessary + createAndGetNewStageNumbers(false); + + // Initial phase would always be new as we can call this method only once for each + switch (controllerStagePhase) { + case NEW: + break; + + case READING_INPUT: + controllerQueryKernel.startStage(new StageId(queryDefinition.getQueryId(), stageNumber)); + break; + + case POST_READING: + setupStage(stageNumber, ControllerStagePhase.READING_INPUT, true); + + if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { + for (int i = 0; i < numWorkers; ++i) { + controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( + new StageId(queryDefinition.getQueryId(), stageNumber), + i, + ClusterByStatisticsSnapshot.empty() + ); + } + } else { + throw new IAE("Stage %d doesn't gather key result statistics", stageNumber); + } + + break; + + case RESULTS_READY: + if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { + setupStage(stageNumber, ControllerStagePhase.POST_READING, true); + } else { + setupStage(stageNumber, ControllerStagePhase.READING_INPUT, true); + } + for (int i = 0; i < numWorkers; ++i) { + controllerQueryKernel.setResultsCompleteForStageAndWorker( + new StageId(queryDefinition.getQueryId(), stageNumber), + i, + new Object() + ); + } + break; + + case FINISHED: + setupStage(stageNumber, ControllerStagePhase.RESULTS_READY, true); + controllerQueryKernel.finishStage(new StageId(queryDefinition.getQueryId(), stageNumber), false); + break; + + case FAILED: + controllerQueryKernel.failStage(new StageId(queryDefinition.getQueryId(), stageNumber)); + break; + } + if (!recursiveCall) { + setupStages.add(stageNumber); + } + return this; + } + + public ControllerQueryKernelTester init() + { + + Preconditions.checkNotNull(queryDefinition, "queryDefinition must be supplied"); + + if (!isValidInitState()) { + throw new ISE("The stages and their phases are not initialized correctly"); + } + initialized = true; + return this; + } + + /** + * For use by external callers. For internal purpose we can skip the "initialized" check + */ + public Set createAndGetNewStageNumbers() + { + return createAndGetNewStageNumbers(true); + } + + private Set createAndGetNewStageNumbers(boolean checkInitialized) + { + if (checkInitialized) { + Preconditions.checkArgument(initialized); + } + return mapStageIdsToStageNumbers( + controllerQueryKernel.createAndGetNewStageIds( + inputSlicerFactory, + WorkerAssignmentStrategy.MAX + ) + ); + } + + public Set getEffectivelyFinishedStageNumbers() + { + Preconditions.checkArgument(initialized); + return mapStageIdsToStageNumbers(controllerQueryKernel.getEffectivelyFinishedStageIds()); + } + + public boolean isDone() + { + Preconditions.checkArgument(initialized); + return controllerQueryKernel.isDone(); + } + + public void markSuccessfulTerminalStagesAsFinished() + { + Preconditions.checkArgument(initialized); + controllerQueryKernel.markSuccessfulTerminalStagesAsFinished(); + } + + public boolean isSuccess() + { + Preconditions.checkArgument(initialized); + return controllerQueryKernel.isSuccess(); + } + + public ControllerStagePhase getStagePhase(int stageNumber) + { + Preconditions.checkArgument(initialized); + return controllerQueryKernel.getStagePhase(new StageId(queryDefinition.getQueryId(), stageNumber)); + } + + public void startStage(int stageNumber) + { + Preconditions.checkArgument(initialized); + controllerQueryKernel.startStage(new StageId(queryDefinition.getQueryId(), stageNumber)); + } + + + public void finishStage(int stageNumber) + { + finishStage(stageNumber, true); + } + + public void finishStage(int stageNumber, boolean strict) + { + Preconditions.checkArgument(initialized); + controllerQueryKernel.finishStage(new StageId(queryDefinition.getQueryId(), stageNumber), strict); + } + + public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerNumber) + { + Preconditions.checkArgument(initialized); + + // Simulate 1000 keys being encountered in the data, so the kernel can generate some partitions. + final ClusterByStatisticsCollector keyStatsCollector = + queryDefinition.getStageDefinition(stageNumber).createResultKeyStatisticsCollector(); + for (int i = 0; i < 1000; i++) { + final RowKey key = KeyTestUtils.createKey( + MockQueryDefinitionBuilder.STAGE_SIGNATURE, + String.valueOf(i) + ); + + keyStatsCollector.add(key, 1); + } + + controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( + new StageId(queryDefinition.getQueryId(), stageNumber), + workerNumber, + keyStatsCollector.snapshot() + ); + } + + public void setResultsCompleteForStageAndWorker(int stageNumber, int workerNumber) + { + Preconditions.checkArgument(initialized); + controllerQueryKernel.setResultsCompleteForStageAndWorker( + new StageId(queryDefinition.getQueryId(), stageNumber), + workerNumber, + new Object() + ); + } + + public void failStage(int stageNumber) + { + Preconditions.checkArgument(initialized); + controllerQueryKernel.failStage(new StageId(queryDefinition.getQueryId(), stageNumber)); + } + + public void assertStagePhase(int stageNumber, ControllerStagePhase expectedControllerStagePhase) + { + Preconditions.checkArgument(initialized); + ControllerStageTracker controllerStageTracker = Preconditions.checkNotNull( + controllerQueryKernel.getControllerStageKernel(stageNumber), + StringUtils.format("Stage kernel for stage number %d is not initialized yet", stageNumber) + ); + if (controllerStageTracker.getPhase() != expectedControllerStagePhase) { + throw new ISE( + StringUtils.format( + "Stage kernel for stage number %d is in %s phase which is different from the expected phase", + stageNumber, + controllerStageTracker.getPhase() + ) + ); + } + } + + /** + * Checks if the state of the BaseControllerQueryKernel is initialized properly. Currently this is just stubbed to + * return true irrespective of the actual state + */ + private boolean isValidInitState() + { + return true; + } + + private Set mapStageIdsToStageNumbers(List stageIds) + { + return stageIds.stream() + .map(StageId::getStageNumber) + .collect(Collectors.toSet()); + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java new file mode 100644 index 00000000000..3094bf344d2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java @@ -0,0 +1,383 @@ +/* + * 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.druid.msq.kernel.controller; + +import com.google.common.collect.ImmutableSet; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Set; + +public class ControllerQueryKernelTests extends BaseControllerQueryKernelTest +{ + + @Test + public void testCompleteDAGExecutionForSingleWorker() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(1); + // 0 1 + // | / | + // 2 / 3 + // | | + // 4 5 + // \ / + // 6 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(7) + .addVertex(0, 2) + .addVertex(1, 2) + .addVertex(1, 3) + .addVertex(2, 4) + .addVertex(3, 5) + .addVertex(4, 6) + .addVertex(5, 6) + .getQueryDefinitionBuilder() + .build() + ); + + controllerQueryKernelTester.init(); + + Set newStageNumbers; + Set effectivelyFinishedStageNumbers; + + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(0, 1), newStageNumbers); + Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); + + + transitionNewToResultsComplete(controllerQueryKernelTester, 1); + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + Assert.assertEquals(ImmutableSet.of(0, 3), newStageNumbers); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); + + + // Mark 3 as done and fetch the new kernels. 5 should be unblocked along with 0. + transitionNewToResultsComplete(controllerQueryKernelTester, 3); + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + Assert.assertEquals(ImmutableSet.of(0, 5), newStageNumbers); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); + + + // Mark 5 as done and fetch the new kernels. Only 0 is still unblocked, but 3 can now be cleaned + transitionNewToResultsComplete(controllerQueryKernelTester, 5); + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + Assert.assertEquals(ImmutableSet.of(0), newStageNumbers); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(3), effectivelyFinishedStageNumbers); + + // Mark 0 as done and fetch the new kernels. This should unblock 2 + transitionNewToResultsComplete(controllerQueryKernelTester, 0); + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + Assert.assertEquals(ImmutableSet.of(2), newStageNumbers); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(3), effectivelyFinishedStageNumbers); + + // Mark 2 as done and fetch new kernels. This should clear up 0 and 1 alongside 3 (which is not marked as FINISHED yet) + transitionNewToResultsComplete(controllerQueryKernelTester, 2); + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + Assert.assertEquals(ImmutableSet.of(4), newStageNumbers); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(0, 1, 3), effectivelyFinishedStageNumbers); + + // Mark 0, 1, 3 finished together + effectivelyFinishedStageNumbers.forEach(controllerQueryKernelTester::finishStage); + + // Mark 4 as done and fetch new kernels. This should unblock 6 and clear up 2 + transitionNewToResultsComplete(controllerQueryKernelTester, 4); + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + Assert.assertEquals(ImmutableSet.of(6), newStageNumbers); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(2), effectivelyFinishedStageNumbers); + + // Mark 6 as done. No more kernels left, but we can clean up 4 and 5 alongwith 2 + transitionNewToResultsComplete(controllerQueryKernelTester, 6); + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + Assert.assertEquals(ImmutableSet.of(), newStageNumbers); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(2, 4, 5), effectivelyFinishedStageNumbers); + effectivelyFinishedStageNumbers.forEach(controllerQueryKernelTester::finishStage); + } + + @Test + public void testCompleteDAGExecutionForMultipleWorkers() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(2); + // 0 -> 1 -> 2 -> 3 + + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(4) + .addVertex(0, 1) + .addVertex(1, 2) + .addVertex(2, 3) + .defineStage(0, true, 1) // Ingestion only on one worker + .defineStage(1, true, 2) + .defineStage(3, true, 2) + .getQueryDefinitionBuilder() + .build() + ); + + controllerQueryKernelTester.init(); + + Set newStageNumbers; + Set effectivelyFinishedStageNumbers; + + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(0), newStageNumbers); + Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); + controllerQueryKernelTester.startStage(0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(1), newStageNumbers); + Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); + controllerQueryKernelTester.startStage(1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 0); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 1); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 0); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 1); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.RESULTS_READY); + + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(2), newStageNumbers); + Assert.assertEquals(ImmutableSet.of(0), effectivelyFinishedStageNumbers); + controllerQueryKernelTester.startStage(2); + controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(2, 0); + controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.RESULTS_READY); + controllerQueryKernelTester.finishStage(0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); + + newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); + effectivelyFinishedStageNumbers = controllerQueryKernelTester.getEffectivelyFinishedStageNumbers(); + Assert.assertEquals(ImmutableSet.of(3), newStageNumbers); + Assert.assertEquals(ImmutableSet.of(1), effectivelyFinishedStageNumbers); + controllerQueryKernelTester.startStage(3); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 0); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 1); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 0); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 1); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.RESULTS_READY); + controllerQueryKernelTester.finishStage(1); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.FINISHED); + + controllerQueryKernelTester.markSuccessfulTerminalStagesAsFinished(); + controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.FINISHED); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.FINISHED); + } + + @Test + public void testTransitionsInShufflingStagesAndMultipleWorkers() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(2); + + // Single stage query definition + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(1) + .defineStage(0, true, 2) + .getQueryDefinitionBuilder() + .build() + ); + + controllerQueryKernelTester.init(); + + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + controllerQueryKernelTester.finishStage(0, false); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); + } + + @Test + public void testPrematureResultsComplete() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(2); + + // Single stage query definition + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(1) + .defineStage(0, true, 2) + .getQueryDefinitionBuilder() + .build() + ); + + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + } + + @Test + public void testKernelFailed() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(1); + + // 0 1 + // \ / + // 2 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(3) + .addVertex(0, 2) + .addVertex(1, 2) + .getQueryDefinitionBuilder() + .build() + ); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setupStage(1, ControllerStagePhase.RESULTS_READY); + + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.failStage(0); + + Assert.assertTrue(controllerQueryKernelTester.isDone()); + Assert.assertFalse(controllerQueryKernelTester.isSuccess()); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FAILED); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.RESULTS_READY); + } + + @Test(expected = IllegalStateException.class) + public void testCycleInvalidQueryThrowsException() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(1); + + // 0 - 1 + // \ / + // 2 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(3) + .addVertex(0, 1) + .addVertex(1, 2) + .addVertex(2, 0) + .getQueryDefinitionBuilder() + .build() + ); + } + + @Test(expected = IllegalStateException.class) + public void testSelfLoopInvalidQueryThrowsException() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(1); + + // 0 _ + // |__| + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(1) + .addVertex(0, 0) + .getQueryDefinitionBuilder() + .build() + ); + } + + @Test(expected = IllegalStateException.class) + public void testLoopInvalidQueryThrowsException() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(1); + + // 0 - 1 + // | | + // --- + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(2) + .addVertex(0, 1) + .addVertex(1, 0) + .getQueryDefinitionBuilder() + .build() + ); + } + + @Test + public void testMarkSuccessfulTerminalStagesAsFinished() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(1); + + // 0 1 + // \ / + // 2 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(3) + .addVertex(0, 2) + .addVertex(1, 2) + .getQueryDefinitionBuilder() + .build() + ); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.FINISHED); + controllerQueryKernelTester.setupStage(1, ControllerStagePhase.RESULTS_READY); + controllerQueryKernelTester.setupStage(2, ControllerStagePhase.RESULTS_READY); + + controllerQueryKernelTester.init(); + + Assert.assertTrue(controllerQueryKernelTester.isDone()); + Assert.assertTrue(controllerQueryKernelTester.isSuccess()); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.RESULTS_READY); + controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.RESULTS_READY); + + controllerQueryKernelTester.markSuccessfulTerminalStagesAsFinished(); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.FINISHED); + controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.FINISHED); + } + + private static void transitionNewToResultsComplete(ControllerQueryKernelTester queryKernelTester, int stageNumber) + { + queryKernelTester.startStage(stageNumber); + queryKernelTester.setResultsCompleteForStageAndWorker(stageNumber, 0); + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSlice.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSlice.java new file mode 100644 index 00000000000..f9473498ce2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSlice.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.druid.msq.kernel.controller; + +import org.apache.druid.msq.input.InputSlice; + +public class ControllerTestInputSlice implements InputSlice +{ + @Override + public int fileCount() + { + return 0; + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpec.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpec.java new file mode 100644 index 00000000000..80a04bdb405 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpec.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.druid.msq.kernel.controller; + +import org.apache.druid.msq.input.InputSpec; + +public class ControllerTestInputSpec implements InputSpec +{ +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpecSlicer.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpecSlicer.java new file mode 100644 index 00000000000..4d4cd9d9fc5 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerTestInputSpecSlicer.java @@ -0,0 +1,57 @@ +/* + * 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.druid.msq.kernel.controller; + +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; + +import java.util.ArrayList; +import java.util.List; + +public class ControllerTestInputSpecSlicer implements InputSpecSlicer +{ + @Override + public boolean canSliceDynamic(InputSpec inputSpec) + { + return false; + } + + @Override + public List sliceStatic(InputSpec inputSpec, int maxNumSlices) + { + final List slices = new ArrayList<>(maxNumSlices); + for (int i = 0; i < maxNumSlices; i++) { + slices.add(new ControllerTestInputSlice()); + } + return slices; + } + + @Override + public List sliceDynamic( + InputSpec inputSpec, + int maxNumSlices, + int maxFilesPerSlice, + long maxBytesPerSlice + ) + { + throw new UnsupportedOperationException(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java new file mode 100644 index 00000000000..f6046c84770 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java @@ -0,0 +1,221 @@ +/* + * 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.druid.msq.kernel.controller; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.kernel.FrameProcessorFactory; +import org.apache.druid.msq.kernel.MaxCountShuffleSpec; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.QueryDefinitionBuilder; +import org.apache.druid.msq.kernel.ShuffleSpec; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class MockQueryDefinitionBuilder +{ + static final String SHUFFLE_KEY_COLUMN = "shuffleKey"; + static final RowSignature STAGE_SIGNATURE = RowSignature.builder().add(SHUFFLE_KEY_COLUMN, ColumnType.STRING).build(); + + private static final int MAX_NUM_PARTITIONS = 32; + + private final int numStages; + + // Maps a stage to all the other stages on which it has dependency, i.e. for an edge like A -> B, the adjacency list + // would have an entry like B : [ A, ... ] + private final Map> adjacencyList = new HashMap<>(); + + // Keeps a collection of those stages that have been already defined + private final Set definedStages = new HashSet<>(); + + // Query definition builder corresponding to this mock builder + private final QueryDefinitionBuilder queryDefinitionBuilder = QueryDefinition.builder(); + + + public MockQueryDefinitionBuilder(final int numStages) + { + this.numStages = numStages; + } + + public MockQueryDefinitionBuilder addVertex(final int outEdge, final int inEdge) + { + Preconditions.checkArgument( + outEdge < numStages, + "vertex number can only be from 0 to one less than the total number of stages" + ); + + Preconditions.checkArgument( + inEdge < numStages, + "vertex number can only be from 0 to one less than the total number of stages" + ); + + Preconditions.checkArgument( + !definedStages.contains(inEdge), + StringUtils.format("%s is already defined, cannot create more connections from it", inEdge) + ); + + Preconditions.checkArgument( + !definedStages.contains(outEdge), + StringUtils.format("%s is already defined, cannot create more connections to it", outEdge) + ); + + adjacencyList.computeIfAbsent(inEdge, k -> new HashSet<>()).add(outEdge); + return this; + } + + public MockQueryDefinitionBuilder defineStage( + int stageNumber, + boolean shuffling, + int maxWorkers + ) + { + Preconditions.checkArgument( + stageNumber < numStages, + "stageNumber should be between 0 and total stages - 1" + ); + Preconditions.checkArgument( + !definedStages.contains(stageNumber), + StringUtils.format("%d is already defined", stageNumber) + ); + definedStages.add(stageNumber); + + ShuffleSpec shuffleSpec; + + if (shuffling) { + shuffleSpec = new MaxCountShuffleSpec( + new ClusterBy( + ImmutableList.of( + new SortColumn(SHUFFLE_KEY_COLUMN, false) + ), + 0 + ), + MAX_NUM_PARTITIONS, + false + ); + } else { + shuffleSpec = null; + } + + final List inputSpecs = + adjacencyList.getOrDefault(stageNumber, new HashSet<>()) + .stream() + .map(StageInputSpec::new).collect(Collectors.toList()); + + if (inputSpecs.isEmpty()) { + inputSpecs.add(new ControllerTestInputSpec()); + } + + queryDefinitionBuilder.add( + StageDefinition.builder(stageNumber) + .inputs(inputSpecs) + .processorFactory(Mockito.mock(FrameProcessorFactory.class)) + .shuffleSpec(shuffleSpec) + .signature(RowSignature.builder().add(SHUFFLE_KEY_COLUMN, ColumnType.STRING).build()) + .maxWorkerCount(maxWorkers) + ); + + return this; + } + + public MockQueryDefinitionBuilder defineStage(int stageNumber, boolean shuffling) + { + return defineStage(stageNumber, shuffling, 1); + } + + public MockQueryDefinitionBuilder defineStage(int stageNumber) + { + return defineStage(stageNumber, false); + } + + public QueryDefinitionBuilder getQueryDefinitionBuilder() + { + if (!verifyIfAcyclic()) { + throw new ISE("The stages of the query form a cycle. Cannot create a query definition builder"); + } + for (int i = 0; i < numStages; ++i) { + if (!definedStages.contains(i)) { + defineStage(i); + } + } + return queryDefinitionBuilder; + } + + /** + * Perform a basic check that the query definition that the user is trying to build is acyclic indeed. This method + * is not required in the source code because the DAG there is created by query toolkits. + */ + private boolean verifyIfAcyclic() + { + Map visited = new HashMap<>(); + + for (int i = 0; i < numStages; i++) { + if (!checkAcyclic(i, visited)) { + return false; + } + } + return true; + } + + /** + * Checks for graph cycles using DFS + */ + private boolean checkAcyclic(int node, Map visited) + { + StageState state = visited.getOrDefault(node, StageState.NEW); + if (state == StageState.VISITED) { + return true; + } + if (state == StageState.VISITING) { + return false; + } else { + visited.put(node, StageState.VISITING); + for (int neighbour : adjacencyList.getOrDefault(node, Collections.emptySet())) { + if (!checkAcyclic(neighbour, visited)) { + return false; + } + } + visited.put(node, StageState.VISITED); + return true; + } + } + + private enum StageState + { + NEW, + VISITING, + VISITED + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/WorkerInputsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/WorkerInputsTest.java new file mode 100644 index 00000000000..5b06ce7f60e --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/WorkerInputsTest.java @@ -0,0 +1,32 @@ +/* + * 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.druid.msq.kernel.controller; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class WorkerInputsTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(WorkerInputs.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java new file mode 100644 index 00000000000..3246938b16a --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java @@ -0,0 +1,278 @@ +/* + * 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.druid.msq.querykit; + +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.ListenableFuture; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.testutil.FrameSequenceBuilder; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.server.QueryStackTests; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +public class BroadcastJoinHelperTest extends InitializedNullHandlingTest +{ + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private JoinableFactory joinableFactory; + private StorageAdapter adapter; + private File testDataFile1; + private File testDataFile2; + private FrameReader frameReader1; + private FrameReader frameReader2; + + @Before + public void setUp() throws IOException + { + final ArenaMemoryAllocator allocator = ArenaMemoryAllocator.createOnHeap(10_000); + + joinableFactory = QueryStackTests.makeJoinableFactoryFromDefault(null, null, null); + + adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + + // File 1: the entire test dataset. + testDataFile1 = FrameTestUtil.writeFrameFile( + FrameSequenceBuilder.fromAdapter(adapter) + .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types + .allocator(allocator) + .frames(), + temporaryFolder.newFile() + ); + + // File 2: just two rows. + testDataFile2 = FrameTestUtil.writeFrameFile( + FrameSequenceBuilder.fromAdapter(adapter) + .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types + .allocator(allocator) + .maxRowsPerFrame(1) + .frames() + .limit(2), + temporaryFolder.newFile() + ); + + frameReader1 = FrameReader.create(adapter.getRowSignature()); + frameReader2 = FrameReader.create(adapter.getRowSignature()); + } + + @Test + public void testBuildTableAndInlineData() throws IOException + { + final Int2IntMap sideStageChannelNumberMap = new Int2IntOpenHashMap(); + sideStageChannelNumberMap.put(3, 1); + sideStageChannelNumberMap.put(4, 2); + + final List channels = new ArrayList<>(); + channels.add(new ExplodingReadableFrameChannel()); + channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1))); + channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile2))); + + final List channelReaders = new ArrayList<>(); + channelReaders.add(null); + channelReaders.add(frameReader1); + channelReaders.add(frameReader2); + + final BroadcastJoinHelper broadcastJoinHelper = new BroadcastJoinHelper( + sideStageChannelNumberMap, + channels, + channelReaders, + new JoinableFactoryWrapper(joinableFactory), + 25_000_000L // High enough memory limit that we won't hit it + ); + + Assert.assertEquals(ImmutableSet.of(1, 2), broadcastJoinHelper.getSideChannelNumbers()); + + boolean doneReading = false; + while (!doneReading) { + final IntSet readableInputs = new IntOpenHashSet(); + for (int i = 1; i < channels.size(); i++) { + readableInputs.add(i); // Frame file channels are always ready, so this is OK. + } + doneReading = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); + } + + Assert.assertTrue(channels.get(1).isFinished()); + Assert.assertTrue(channels.get(2).isFinished()); + + Assert.assertEquals( + new InputNumberDataSource(0), + broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(0)) + ); + + Assert.assertEquals( + new InputNumberDataSource(1), + broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(1)) + ); + + Assert.assertEquals( + new InputNumberDataSource(2), + broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(2)) + ); + + final List rowsFromStage3 = + ((InlineDataSource) broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(3))).getRowsAsList(); + Assert.assertEquals(1209, rowsFromStage3.size()); + + FrameTestUtil.assertRowsEqual( + FrameTestUtil.readRowsFromAdapter(adapter, null, false), + Sequences.simple(rowsFromStage3.stream().map(Arrays::asList).collect(Collectors.toList())) + ); + + final List rowsFromStage4 = + ((InlineDataSource) broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(4))).getRowsAsList(); + Assert.assertEquals(2, rowsFromStage4.size()); + + FrameTestUtil.assertRowsEqual( + FrameTestUtil.readRowsFromAdapter(adapter, null, false).limit(2), + Sequences.simple(rowsFromStage4.stream().map(Arrays::asList).collect(Collectors.toList())) + ); + + final DataSource inlinedJoinDataSource = broadcastJoinHelper.inlineChannelData( + JoinDataSource.create( + new InputNumberDataSource(0), + new InputNumberDataSource(4), + "j.", + JoinConditionAnalysis.forExpression("x == \"j.x\"", "j.", ExprMacroTable.nil()), + JoinType.INNER, + null + ) + ); + + MatcherAssert.assertThat( + ((JoinDataSource) inlinedJoinDataSource).getRight(), + CoreMatchers.instanceOf(InlineDataSource.class) + ); + + Assert.assertEquals( + 2, + ((InlineDataSource) ((JoinDataSource) inlinedJoinDataSource).getRight()).getRowsAsList().size() + ); + } + + @Test + public void testBuildTableMemoryLimit() throws IOException + { + final Int2IntMap sideStageChannelNumberMap = new Int2IntOpenHashMap(); + sideStageChannelNumberMap.put(0, 0); + + final List channels = new ArrayList<>(); + channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1))); + + final List channelReaders = new ArrayList<>(); + channelReaders.add(frameReader1); + + final BroadcastJoinHelper broadcastJoinHelper = new BroadcastJoinHelper( + sideStageChannelNumberMap, + channels, + channelReaders, + new JoinableFactoryWrapper(joinableFactory), + 100_000 // Low memory limit; we will hit this + ); + + Assert.assertEquals(ImmutableSet.of(0), broadcastJoinHelper.getSideChannelNumbers()); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> { + boolean doneReading = false; + while (!doneReading) { + final IntSet readableInputs = new IntOpenHashSet(new int[]{0}); + doneReading = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); + } + } + ); + + Assert.assertEquals(new BroadcastTablesTooLargeFault(100_000), e.getFault()); + } + + /** + * Throws an error on every method call. Useful for ensuring that a channel is *not* read. + */ + private static class ExplodingReadableFrameChannel implements ReadableFrameChannel + { + @Override + public boolean isFinished() + { + throw new IllegalStateException(); + } + + @Override + public boolean canRead() + { + throw new IllegalStateException(); + } + + @Override + public Frame read() + { + throw new IllegalStateException(); + } + + @Override + public ListenableFuture readabilityFuture() + { + throw new IllegalStateException(); + } + + @Override + public void close() + { + throw new IllegalStateException(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQModeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQModeTest.java new file mode 100644 index 00000000000..ea9bb4ff5b3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQModeTest.java @@ -0,0 +1,57 @@ +/* + * 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.druid.msq.sql; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.indexing.error.MSQWarnings; +import org.apache.druid.query.QueryContext; +import org.junit.Assert; +import org.junit.Test; + +public class MSQModeTest +{ + + @Test + public void testPopulateQueryContextWhenNoSupercedingValuePresent() + { + QueryContext originalQueryContext = new QueryContext(); + MSQMode.populateDefaultQueryContext("strict", originalQueryContext); + Assert.assertEquals(ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0), originalQueryContext.getMergedParams()); + } + + @Test + public void testPopulateQueryContextWhenSupercedingValuePresent() + { + QueryContext originalQueryContext = new QueryContext(ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 10)); + MSQMode.populateDefaultQueryContext("strict", originalQueryContext); + Assert.assertEquals(ImmutableMap.of(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 10), originalQueryContext.getMergedParams()); + + } + + @Test + public void testPopulateQueryContextWhenInvalidMode() + { + QueryContext originalQueryContext = new QueryContext(); + Assert.assertThrows(ISE.class, () -> { + MSQMode.populateDefaultQueryContext("fake_mode", originalQueryContext); + }); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.java new file mode 100644 index 00000000000..4bd5f941408 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.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.druid.msq.sql; + +import com.google.common.collect.ImmutableList; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class MSQTaskQueryMakerTest +{ + @Test + public void testValidateSegmentSortOrder() + { + // These are all OK, so validateSegmentSortOrder does nothing. + MSQTaskQueryMaker.validateSegmentSortOrder(Collections.emptyList(), ImmutableList.of("__time", "a", "b")); + MSQTaskQueryMaker.validateSegmentSortOrder(ImmutableList.of("__time"), ImmutableList.of("__time", "a", "b")); + MSQTaskQueryMaker.validateSegmentSortOrder(ImmutableList.of("__time", "b"), ImmutableList.of("__time", "a", "b")); + MSQTaskQueryMaker.validateSegmentSortOrder(ImmutableList.of("b"), ImmutableList.of("a", "b")); + + // These are not OK. + Assert.assertThrows( + IllegalArgumentException.class, + () -> MSQTaskQueryMaker.validateSegmentSortOrder(ImmutableList.of("c"), ImmutableList.of("a", "b")) + ); + + Assert.assertThrows( + IllegalArgumentException.class, + () -> MSQTaskQueryMaker.validateSegmentSortOrder( + ImmutableList.of("b", "__time"), + ImmutableList.of("__time", "a", "b") + ) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java new file mode 100644 index 00000000000..03efc96e6de --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.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.druid.msq.sql; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.query.QueryException; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class SqlTaskStatusTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + + final SqlTaskStatus status = new SqlTaskStatus( + "taskid", + TaskState.FAILED, + new QueryException( + "error code", + "error message", + "error class", + "host" + ) + ); + + final SqlTaskStatus status2 = mapper.readValue(mapper.writeValueAsString(status), SqlTaskStatus.class); + + Assert.assertEquals(status.getTaskId(), status2.getTaskId()); + Assert.assertEquals(status.getState(), status2.getState()); + Assert.assertEquals(status.getError().getErrorCode(), status2.getError().getErrorCode()); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(SqlTaskStatus.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java new file mode 100644 index 00000000000..6976aa687f4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java @@ -0,0 +1,951 @@ +/* + * 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.druid.msq.statistics; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.math.LongMath; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.KeyTestUtils; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.key.RowKeyReader; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import java.math.RoundingMode; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Random; +import java.util.TreeMap; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlingTest +{ + private static final double PARTITION_SIZE_LEEWAY = 0.3; + + private static final RowSignature SIGNATURE = + RowSignature.builder().add("x", ColumnType.LONG).add("y", ColumnType.LONG).build(); + + private static final ClusterBy CLUSTER_BY_X = new ClusterBy( + ImmutableList.of(new SortColumn("x", false)), + 0 + ); + + private static final ClusterBy CLUSTER_BY_XY_BUCKET_BY_X = new ClusterBy( + ImmutableList.of(new SortColumn("x", false), new SortColumn("y", false)), + 1 + ); + + // These numbers are roughly 10x lower than authentic production numbers. (See StageDefinition.) + private static final int MAX_KEYS = 5000; + private static final int MAX_BUCKETS = 1000; + + @Test + public void test_clusterByX_unique() + { + final long numRows = 1_000_000; + final boolean aggregate = false; + final ClusterBy clusterBy = CLUSTER_BY_X; + final Iterable keys = () -> + LongStream.range(0, numRows) + .mapToObj(n -> createKey(clusterBy, n)) + .iterator(); + + final NavigableMap> sortedKeyWeights = + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + + doTest( + clusterBy, + aggregate, + keys, + (testName, collector) -> { + Assert.assertEquals(StringUtils.format("%s: tracked bucket count", testName), 1, trackedBuckets(collector)); + Assert.assertEquals(StringUtils.format("%s: tracked row count", testName), numRows, trackedRows(collector)); + + for (int targetPartitionWeight : new int[]{51111, 65432, (int) numRows + 10}) { + verifyPartitionsWithTargetWeight( + StringUtils.format("%s: generatePartitionsWithTargetWeight(%d)", testName, targetPartitionWeight), + collector, + targetPartitionWeight, + sortedKeyWeights, + aggregate + ); + } + + for (int maxPartitionCount : new int[]{1, 2, 10, 50}) { + verifyPartitionsWithMaxCount( + StringUtils.format("%s: generatePartitionsWithMaxCount(%d)", testName, maxPartitionCount), + collector, + maxPartitionCount, + sortedKeyWeights, + aggregate + ); + } + + verifySnapshotSerialization(testName, collector, aggregate); + } + ); + } + + @Test + public void test_clusterByX_everyKeyAppearsTwice() + { + final long numRows = 1_000_000; + final boolean aggregate = false; + final ClusterBy clusterBy = CLUSTER_BY_X; + final List keys = new ArrayList<>(); + + for (int i = 0; i < numRows / 2; i++) { + keys.add(createKey(clusterBy, (long) i)); + keys.add(createKey(clusterBy, (long) i)); + } + + final NavigableMap> sortedKeyWeights = + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + + doTest( + clusterBy, + aggregate, + keys, + (testName, collector) -> { + Assert.assertEquals(StringUtils.format("%s: tracked bucket count", testName), 1, trackedBuckets(collector)); + Assert.assertEquals(StringUtils.format("%s: tracked row count", testName), numRows, trackedRows(collector)); + + for (int targetPartitionWeight : new int[]{51111, 65432, (int) numRows + 10}) { + verifyPartitionsWithTargetWeight( + StringUtils.format("%s: generatePartitionsWithTargetWeight(%d)", testName, targetPartitionWeight), + collector, + targetPartitionWeight, + sortedKeyWeights, + aggregate + ); + } + + for (int maxPartitionCount : new int[]{1, 2, 10, 50}) { + verifyPartitionsWithMaxCount( + StringUtils.format("%s: generatePartitionsWithMaxCount(%d)", testName, maxPartitionCount), + collector, + maxPartitionCount, + sortedKeyWeights, + aggregate + ); + } + + verifySnapshotSerialization(testName, collector, aggregate); + } + ); + } + + @Test + public void test_clusterByX_everyKeyAppearsTwice_withAggregation() + { + final long numRows = 1_000_000; + final boolean aggregate = true; + final ClusterBy clusterBy = CLUSTER_BY_X; + final List keys = new ArrayList<>(); + final int duplicationFactor = 2; + + for (int i = 0; i < numRows / duplicationFactor; i++) { + for (int j = 0; j < duplicationFactor; j++) { + keys.add(createKey(clusterBy, (long) i)); + } + } + + final NavigableMap> sortedKeyWeights = + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + + doTest( + clusterBy, + aggregate, + keys, + (testName, collector) -> { + Assert.assertEquals(StringUtils.format("%s: tracked bucket count", testName), 1, trackedBuckets(collector)); + + final double expectedNumRows = (double) numRows / duplicationFactor; + Assert.assertEquals( + StringUtils.format("%s: tracked row count", testName), + expectedNumRows, + (double) trackedRows(collector), + expectedNumRows * .05 // Acceptable estimation error + ); + + for (int targetPartitionWeight : new int[]{51111, 65432, (int) numRows + 10}) { + verifyPartitionsWithTargetWeight( + StringUtils.format("%s: generatePartitionsWithTargetWeight(%d)", testName, targetPartitionWeight), + collector, + targetPartitionWeight, + sortedKeyWeights, + aggregate + ); + } + + for (int maxPartitionCount : new int[]{1, 2, 5, 25}) { + verifyPartitionsWithMaxCount( + StringUtils.format("%s: generatePartitionsWithMaxCount(%d)", testName, maxPartitionCount), + collector, + maxPartitionCount, + sortedKeyWeights, + aggregate + ); + } + + verifySnapshotSerialization(testName, collector, aggregate); + } + ); + } + + @Test + public void test_clusterByXYbucketByX_threeX_uniqueY() + { + final int numBuckets = 3; + final boolean aggregate = false; + final long numRows = 1_000_000; + final ClusterBy clusterBy = CLUSTER_BY_XY_BUCKET_BY_X; + final List keys = new ArrayList<>((int) numRows); + + for (int i = 0; i < numRows; i++) { + final Object[] key = new Object[2]; + key[0] = (long) (i % numBuckets); + key[1] = (long) i; + keys.add(createKey(clusterBy, key)); + } + + final NavigableMap> sortedKeyWeights = + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + + doTest( + clusterBy, + aggregate, + keys, + (testName, collector) -> { + Assert.assertEquals(StringUtils.format("%s: bucket count", testName), numBuckets, trackedBuckets(collector)); + Assert.assertEquals(StringUtils.format("%s: row count", testName), numRows, trackedRows(collector)); + + for (int targetPartitionWeight : new int[]{17001, 23007}) { + verifyPartitionsWithTargetWeight( + StringUtils.format("%s: generatePartitionsWithTargetWeight(%d)", testName, targetPartitionWeight), + collector, + targetPartitionWeight, + sortedKeyWeights, + aggregate + ); + } + + for (int maxPartitionCount : new int[]{1, 2, 3, 10, 50}) { + if (maxPartitionCount < numBuckets) { + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> collector.generatePartitionsWithMaxCount(maxPartitionCount) + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Unable to compute partition ranges")) + ); + } else { + verifyPartitionsWithMaxCount( + StringUtils.format("%s: generatePartitionsWithMaxCount(%d)", testName, maxPartitionCount), + collector, + maxPartitionCount, + sortedKeyWeights, + aggregate + ); + } + } + + verifySnapshotSerialization(testName, collector, aggregate); + } + ); + } + + @Test + public void test_clusterByXYbucketByX_maxX_uniqueY() + { + final int numBuckets = MAX_BUCKETS; + final boolean aggregate = false; + final long numRows = 1_000_000; + final ClusterBy clusterBy = CLUSTER_BY_XY_BUCKET_BY_X; + final List keys = new ArrayList<>((int) numRows); + + for (int i = 0; i < numRows; i++) { + final Object[] key = new Object[2]; + key[0] = (long) (i % numBuckets); + key[1] = (long) i; + keys.add(createKey(clusterBy, key)); + } + + final NavigableMap> sortedKeyWeights = + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + + doTest( + clusterBy, + aggregate, + keys, + (testName, collector) -> { + Assert.assertEquals(StringUtils.format("%s: bucket count", testName), numBuckets, trackedBuckets(collector)); + + for (int targetPartitionWeight : new int[]{17001, 23007}) { + verifyPartitionsWithTargetWeight( + StringUtils.format("%s: generatePartitionsWithTargetWeight(%d)", testName, targetPartitionWeight), + collector, + targetPartitionWeight, + sortedKeyWeights, + aggregate + ); + } + + for (int maxPartitionCount : new int[]{1, 10, numBuckets - 1, numBuckets}) { + if (maxPartitionCount < numBuckets) { + // Cannot compute partitions ranges when maxPartitionCount < numBuckets, because there must be at + // least one partition per bucket. + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> verifyPartitionsWithMaxCount( + StringUtils.format("%s: generatePartitionsWithMaxCount(%d)", testName, maxPartitionCount), + collector, + maxPartitionCount, + sortedKeyWeights, + aggregate + ) + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Unable to compute partition ranges")) + ); + } else { + verifyPartitionsWithMaxCount( + StringUtils.format("%s: generatePartitionsWithMaxCount(%d)", testName, maxPartitionCount), + collector, + maxPartitionCount, + sortedKeyWeights, + aggregate + ); + } + } + + verifySnapshotSerialization(testName, collector, aggregate); + } + ); + } + + @Test + public void test_clusterByXYbucketByX_maxX_lowCardinalityY_withAggregation() + { + final int numBuckets = MAX_BUCKETS; + final boolean aggregate = true; + final long numRows = 1_000_000; + final ClusterBy clusterBy = CLUSTER_BY_XY_BUCKET_BY_X; + final List keys = new ArrayList<>((int) numRows); + + for (int i = 0; i < numRows; i++) { + final Object[] key = new Object[2]; + key[0] = (long) (i % numBuckets); + key[1] = (long) (i % 5); // Only five different Ys + keys.add(createKey(clusterBy, key)); + } + + final NavigableMap> sortedKeyWeights = + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + + doTest( + clusterBy, + aggregate, + keys, + (testName, collector) -> { + Assert.assertEquals(StringUtils.format("%s: bucket count", testName), numBuckets, trackedBuckets(collector)); + + // trackedRows will equal numBuckets, because the collectors have been downsampled so much + Assert.assertEquals(StringUtils.format("%s: row count", testName), numBuckets, trackedRows(collector)); + + for (int targetPartitionWeight : new int[]{17001, 23007}) { + verifyPartitionsWithTargetWeight( + StringUtils.format("%s: generatePartitionsWithTargetWeight(%d)", testName, targetPartitionWeight), + collector, + targetPartitionWeight, + sortedKeyWeights, + aggregate + ); + } + + for (int maxPartitionCount : new int[]{1, 10, numBuckets, numBuckets + 1}) { + if (maxPartitionCount < numBuckets) { + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> collector.generatePartitionsWithMaxCount(maxPartitionCount) + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Unable to compute partition ranges")) + ); + } else { + verifyPartitionsWithMaxCount( + StringUtils.format("%s: generatePartitionsWithMaxCount(%d)", testName, maxPartitionCount), + collector, + maxPartitionCount, + sortedKeyWeights, + aggregate + ); + } + } + + verifySnapshotSerialization(testName, collector, aggregate); + } + ); + } + + @Test(expected = IllegalArgumentException.class) + public void testMoreBucketsThanKeysThrowsException() + { + ClusterByStatisticsCollectorImpl.create(ClusterBy.none(), + RowSignature.empty(), + 0, + 5, + false, + false); + } + + @Test(expected = TooManyBucketsException.class) + public void testTooManyBuckets() + { + ClusterByStatisticsCollector clusterByStatisticsCollector = ClusterByStatisticsCollectorImpl.create(ClusterBy.none(), + RowSignature.empty(), + 5, + 0, + false, + false); + clusterByStatisticsCollector.add(RowKey.empty(), 1); + } + + @Test + public void testGeneratePartitionWithoutAddCreatesUniversalPartition() + { + ClusterByStatisticsCollector clusterByStatisticsCollector = ClusterByStatisticsCollectorImpl.create(ClusterBy.none(), + RowSignature.empty(), + 5, + 0, + false, + false); + Assert.assertEquals(ClusterByPartitions.oneUniversalPartition(), clusterByStatisticsCollector.generatePartitionsWithTargetWeight(10)); + } + + @Test(expected = IllegalArgumentException.class) + public void testGeneratePartitionWithZeroTargetWeightThrowsException() + { + ClusterByStatisticsCollector clusterByStatisticsCollector = ClusterByStatisticsCollectorImpl.create(ClusterBy.none(), + RowSignature.empty(), + 5, + 0, + false, false); + clusterByStatisticsCollector.generatePartitionsWithTargetWeight(0); + } + + @Test(expected = IllegalArgumentException.class) + public void testGeneratePartitionWithZeroCountThrowsException() + { + ClusterByStatisticsCollector clusterByStatisticsCollector = ClusterByStatisticsCollectorImpl.create(ClusterBy.none(), + RowSignature.empty(), + 5, + 0, + false, + false); + clusterByStatisticsCollector.generatePartitionsWithMaxCount(0); + } + + @Test(expected = IllegalStateException.class) + public void testHasMultipleValuesFalseThrowsException() + { + ClusterByStatisticsCollector clusterByStatisticsCollector = ClusterByStatisticsCollectorImpl.create(ClusterBy.none(), + RowSignature.empty(), + 5, + 0, + false, + false); + clusterByStatisticsCollector.hasMultipleValues(0); + } + + @Test(expected = ISE.class) + public void testHasMultipleValuesInvalidKeyThrowsException() + { + ClusterByStatisticsCollector clusterByStatisticsCollector = ClusterByStatisticsCollectorImpl.create(ClusterBy.none(), + RowSignature.empty(), + 5, + 0, + false, + false); + clusterByStatisticsCollector.hasMultipleValues(-1); + } + + private void doTest( + final ClusterBy clusterBy, + final boolean aggregate, + final Iterable keys, + final BiConsumer testFn + ) + { + final Comparator comparator = clusterBy.keyComparator(); + + // Load into single collector, sorted order. + final ClusterByStatisticsCollectorImpl sortedCollector = makeCollector(clusterBy, aggregate); + final List sortedKeys = Lists.newArrayList(keys); + sortedKeys.sort(comparator); + sortedKeys.forEach(k -> sortedCollector.add(k, 1)); + testFn.accept("single collector, sorted order", sortedCollector); + + // Load into single collector, reverse sorted order. + final ClusterByStatisticsCollectorImpl reverseSortedCollector = makeCollector(clusterBy, aggregate); + final List reverseSortedKeys = Lists.newArrayList(keys); + reverseSortedKeys.sort(comparator.reversed()); + reverseSortedKeys.forEach(k -> reverseSortedCollector.add(k, 1)); + testFn.accept("single collector, reverse sorted order", reverseSortedCollector); + + // Randomized load into single collector. + final ClusterByStatisticsCollectorImpl randomizedCollector = makeCollector(clusterBy, aggregate); + final List randomizedKeys = Lists.newArrayList(keys); + Collections.shuffle(randomizedKeys, new Random(7 /* Consistent seed from run to run */)); + randomizedKeys.forEach(k -> randomizedCollector.add(k, 1)); + testFn.accept("single collector, random order", randomizedCollector); + + // Split randomized load into three collectors of the same size, followed by merge. + final List threeEqualSizedCollectors = new ArrayList<>(); + threeEqualSizedCollectors.add(makeCollector(clusterBy, aggregate)); + threeEqualSizedCollectors.add(makeCollector(clusterBy, aggregate)); + threeEqualSizedCollectors.add(makeCollector(clusterBy, aggregate)); + + final Iterator iterator1 = randomizedKeys.iterator(); + for (int i = 0; iterator1.hasNext(); i++) { + final RowKey key = iterator1.next(); + threeEqualSizedCollectors.get(i % threeEqualSizedCollectors.size()).add(key, 1); + } + + threeEqualSizedCollectors.get(0).addAll(threeEqualSizedCollectors.get(1)); // Regular add + threeEqualSizedCollectors.get(0).addAll(threeEqualSizedCollectors.get(2).snapshot()); // Snapshot add + + testFn.accept("three merged collectors, equal sizes", threeEqualSizedCollectors.get(0)); + + // Split randomized load into three collectors of different sizes, followed by merge. + final List threeDifferentlySizedCollectors = new ArrayList<>(); + threeDifferentlySizedCollectors.add(makeCollector(clusterBy, aggregate)); + threeDifferentlySizedCollectors.add(makeCollector(clusterBy, aggregate)); + threeDifferentlySizedCollectors.add(makeCollector(clusterBy, aggregate)); + + final Iterator iterator2 = randomizedKeys.iterator(); + for (int i = 0; iterator2.hasNext(); i++) { + final RowKey key = iterator2.next(); + + if (i % 100 < 2) { + // 2% of space + threeDifferentlySizedCollectors.get(0).add(key, 1); + } else if (i % 100 < 20) { + // 18% of space + threeDifferentlySizedCollectors.get(1).add(key, 1); + } else { + // 80% of space + threeDifferentlySizedCollectors.get(2).add(key, 1); + } + } + + threeDifferentlySizedCollectors.get(0).addAll(threeDifferentlySizedCollectors.get(1)); // Big into small + threeDifferentlySizedCollectors.get(2).addAll(threeDifferentlySizedCollectors.get(0)); // Small into big + + testFn.accept("three merged collectors, different sizes", threeDifferentlySizedCollectors.get(2)); + } + + private ClusterByStatisticsCollectorImpl makeCollector(final ClusterBy clusterBy, final boolean aggregate) + { + return (ClusterByStatisticsCollectorImpl) + ClusterByStatisticsCollectorImpl.create(clusterBy, SIGNATURE, MAX_KEYS, MAX_BUCKETS, aggregate, false); + } + + private static void verifyPartitions( + final String testName, + final ClusterBy clusterBy, + final ClusterByPartitions partitions, + final NavigableMap> sortedKeyWeights, + final boolean aggregate, + final long expectedPartitionSize + ) + { + final RowKeyReader keyReader = clusterBy.keyReader(SIGNATURE); + + final int expectedNumberOfBuckets = + sortedKeyWeights.keySet() + .stream() + .map(key -> keyReader.trim(key, clusterBy.getBucketByCount())) + .collect(Collectors.toSet()) + .size(); + + verifyNumberOfBuckets(testName, clusterBy, partitions, expectedNumberOfBuckets); + verifyPartitionsRespectBucketBoundaries(testName, clusterBy, partitions, sortedKeyWeights); + verifyPartitionsCoverKeySpace( + testName, + partitions, + sortedKeyWeights.firstKey(), + clusterBy.keyComparator() + ); + verifyPartitionWeights(testName, clusterBy, partitions, sortedKeyWeights, aggregate, expectedPartitionSize); + } + + private static RowKey createKey(final ClusterBy clusterBy, final Object... objects) + { + return KeyTestUtils.createKey( + KeyTestUtils.createKeySignature(clusterBy.getColumns(), SIGNATURE), + objects + ); + } + + private static void verifyPartitionsWithTargetWeight( + final String testName, + final ClusterByStatisticsCollector collector, + final int targetPartitionWeight, + final NavigableMap> sortedKeyWeights, + final boolean aggregate + ) + { + verifyPartitions( + testName, + collector.getClusterBy(), + collector.generatePartitionsWithTargetWeight(targetPartitionWeight), + sortedKeyWeights, + aggregate, + targetPartitionWeight + ); + } + + private static void verifyPartitionsWithMaxCount( + final String testName, + final ClusterByStatisticsCollector collector, + final int maxPartitionCount, + final NavigableMap> sortedKeyWeights, + final boolean aggregate + ) + { + final ClusterByPartitions partitions = collector.generatePartitionsWithMaxCount(maxPartitionCount); + + verifyPartitions( + testName, + collector.getClusterBy(), + partitions, + sortedKeyWeights, + aggregate, + LongMath.divide( + totalWeight(sortedKeyWeights, new ClusterByPartition(null, null), aggregate), + maxPartitionCount, + RoundingMode.CEILING + ) + ); + + MatcherAssert.assertThat( + StringUtils.format("%s: number of partitions ≤ max", testName), + partitions.size(), + Matchers.lessThanOrEqualTo(maxPartitionCount) + ); + } + + private static void verifyNumberOfBuckets( + final String testName, + final ClusterBy clusterBy, + final ClusterByPartitions partitions, + final int expectedNumberOfBuckets + ) + { + final RowKeyReader keyReader = clusterBy.keyReader(SIGNATURE); + + Assert.assertEquals( + StringUtils.format("%s: number of buckets", testName), + expectedNumberOfBuckets, + partitions.ranges() + .stream() + .map(partition -> keyReader.trim(partition.getStart(), clusterBy.getBucketByCount())) + .distinct() + .count() + ); + } + + /** + * Verify that: + * + * - Partitions are all abutting + * - The start of the first partition matches the minimum key (if there are keys) + * - The end of the last partition is null + * - Each partition's end is after its start + */ + static void verifyPartitionsCoverKeySpace( + final String testName, + final ClusterByPartitions partitions, + final RowKey expectedMinKey, + final Comparator comparator + ) + { + Assert.assertTrue(StringUtils.format("%s: partitions abutting", testName), partitions.allAbutting()); + + final List ranges = partitions.ranges(); + + for (int i = 0; i < ranges.size(); i++) { + final ClusterByPartition partition = ranges.get(i); + + // Check expected nullness of the start key. + if (i == 0) { + Assert.assertEquals( + StringUtils.format("%s: partition %d: start is min key", testName, i), + expectedMinKey, + partition.getStart() + ); + } else { + Assert.assertNotNull( + StringUtils.format("%s: partition %d: start is nonnull", testName, i), + partition.getStart() + ); + } + + // Check expected nullness of the end key. + if (i == ranges.size() - 1) { + Assert.assertNull( + StringUtils.format("%s: partition %d (final): end is null", testName, i), + partition.getEnd() + ); + } else { + Assert.assertNotNull( + StringUtils.format("%s: partition %d: end is nonnull", testName, i), + partition.getEnd() + ); + } + + // Check that the ends are all after the starts. + if (partition.getStart() != null && partition.getEnd() != null) { + MatcherAssert.assertThat( + StringUtils.format("%s: partition %d: start compareTo end", testName, i), + comparator.compare(partition.getStart(), partition.getEnd()), + Matchers.lessThan(0) + ); + } + } + } + + /** + * Verify that no partition spans more than one bucket. + */ + private static void verifyPartitionsRespectBucketBoundaries( + final String testName, + final ClusterBy clusterBy, + final ClusterByPartitions partitions, + final NavigableMap> sortedKeyWeights + ) + { + final RowKeyReader keyReader = clusterBy.keyReader(SIGNATURE); + final List ranges = partitions.ranges(); + + for (int i = 0; i < ranges.size(); i++) { + final ClusterByPartition partition = ranges.get(i); + final RowKey firstBucketKey = keyReader.trim(partition.getStart(), clusterBy.getBucketByCount()); + final RowKey lastBucketKey = keyReader.trim( + partition.getEnd() == null + ? sortedKeyWeights.lastKey() + : sortedKeyWeights.subMap(partition.getStart(), true, partition.getEnd(), false).lastKey(), + clusterBy.getBucketByCount() + ); + + Assert.assertEquals( + StringUtils.format("%s: partition %d: first, last bucket key are equal", testName, i), + firstBucketKey, + lastBucketKey + ); + } + } + + /** + * Verify that partitions have "reasonable" sizes. + */ + static void verifyPartitionWeights( + final String testName, + final ClusterBy clusterBy, + final ClusterByPartitions partitions, + final NavigableMap> sortedKeyWeights, + final boolean aggregate, + final long expectedPartitionSize + ) + { + final RowKeyReader keyReader = clusterBy.keyReader(SIGNATURE); + final List ranges = partitions.ranges(); + + // Compute actual number of rows per partition. + final Map rowsPerPartition = new HashMap<>(); + + for (final ClusterByPartition partition : partitions) { + rowsPerPartition.put( + partition.getStart(), + totalWeight(sortedKeyWeights, partition, aggregate) + ); + } + + // Compare actual size to desired size. + for (int i = 0; i < ranges.size(); i++) { + final ClusterByPartition partition = ranges.get(i); + final RowKey bucketKey = keyReader.trim(partition.getStart(), clusterBy.getBucketByCount()); + final long actualNumberOfRows = rowsPerPartition.get(partition.getStart()); + + // Reasonable maximum number of rows per partition. + MatcherAssert.assertThat( + StringUtils.format("%s: partition #%d: number of rows", testName, i), + actualNumberOfRows, + Matchers.lessThanOrEqualTo((long) ((1 + PARTITION_SIZE_LEEWAY) * expectedPartitionSize)) + ); + + // Reasonable minimum number of rows per partition, for all partitions except the last in a bucket. + // Our algorithm allows the last partition of each bucket to be extra-small. + final boolean isLastInBucket = + i == partitions.size() - 1 + || !keyReader.trim(partitions.get(i + 1).getStart(), clusterBy.getBucketByCount()).equals(bucketKey); + + if (!isLastInBucket) { + MatcherAssert.assertThat( + StringUtils.format("%s: partition #%d: number of rows", testName, i), + actualNumberOfRows, + Matchers.greaterThanOrEqualTo((long) ((1 - PARTITION_SIZE_LEEWAY) * expectedPartitionSize)) + ); + } + } + } + + static NavigableMap> computeSortedKeyWeightsFromWeightedKeys( + final Iterable> keys, + final Comparator comparator + ) + { + final NavigableMap> sortedKeyWeights = new TreeMap<>(comparator); + + for (final Pair key : keys) { + sortedKeyWeights.computeIfAbsent(key.lhs, k -> new ArrayList<>()).add(key.rhs); + } + + return sortedKeyWeights; + } + + static NavigableMap> computeSortedKeyWeightsFromUnweightedKeys( + final Iterable keys, + final Comparator comparator + ) + { + return computeSortedKeyWeightsFromWeightedKeys( + Iterables.transform(keys, key -> Pair.of(key, 1)), + comparator + ); + } + + static long totalWeight( + final NavigableMap> sortedKeyWeights, + final ClusterByPartition partition, + final boolean aggregate + ) + { + final NavigableMap> partitionWeights = + sortedKeyWeights.subMap( + partition.getStart() == null ? sortedKeyWeights.firstKey() : partition.getStart(), + true, + partition.getEnd() == null ? sortedKeyWeights.lastKey() : partition.getEnd(), + partition.getEnd() == null + ); + + long retVal = 0; + + for (final Collection weights : partitionWeights.values()) { + if (aggregate) { + retVal += Collections.max(weights); + } else { + for (int w : weights) { + retVal += w; + } + } + } + + return retVal; + } + + private static long trackedBuckets(final ClusterByStatisticsCollectorImpl collector) + { + return collector.getKeyCollectors().size(); + } + + private static long trackedRows(final ClusterByStatisticsCollectorImpl collector) + { + long count = 0; + for (final KeyCollector keyCollector : collector.getKeyCollectors()) { + count += keyCollector.estimatedTotalWeight(); + } + return count; + } + + private static void verifySnapshotSerialization( + final String testName, + final ClusterByStatisticsCollector collector, + final boolean aggregate + ) + { + try { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + jsonMapper.registerModule( + new KeyCollectorSnapshotDeserializerModule( + KeyCollectors.makeStandardFactory( + collector.getClusterBy(), + aggregate + ) + ) + ); + + final ClusterByStatisticsSnapshot snapshot = collector.snapshot(); + final ClusterByStatisticsSnapshot snapshot2 = jsonMapper.readValue( + jsonMapper.writeValueAsString(snapshot), + ClusterByStatisticsSnapshot.class + ); + + Assert.assertEquals(StringUtils.format("%s: snapshot is serializable", testName), snapshot, snapshot2); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java new file mode 100644 index 00000000000..f5a07de1af5 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.statistics; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.KeyTestUtils; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Comparator; +import java.util.NoSuchElementException; + +public class DelegateOrMinKeyCollectorTest +{ + private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new SortColumn("x", false)), 0); + private final RowSignature signature = RowSignature.builder().add("x", ColumnType.LONG).build(); + private final Comparator comparator = clusterBy.keyComparator(); + + @Test + public void testEmpty() + { + final DelegateOrMinKeyCollector collector = + new DelegateOrMinKeyCollectorFactory<>( + comparator, + QuantilesSketchKeyCollectorFactory.create(clusterBy) + ).newKeyCollector(); + + Assert.assertTrue(collector.getDelegate().isPresent()); + Assert.assertTrue(collector.isEmpty()); + Assert.assertThrows(NoSuchElementException.class, collector::minKey); + Assert.assertEquals(0, collector.estimatedRetainedKeys()); + Assert.assertEquals(0, collector.estimatedTotalWeight()); + MatcherAssert.assertThat(collector.getDelegate().get(), CoreMatchers.instanceOf(QuantilesSketchKeyCollector.class)); + } + + @Test(expected = ISE.class) + public void testDelegateAndMinKeyNotNullThrowsException() + { + ClusterBy clusterBy = ClusterBy.none(); + new DelegateOrMinKeyCollector<>( + clusterBy.keyComparator(), + QuantilesSketchKeyCollectorFactory.create(clusterBy).newKeyCollector(), + RowKey.empty() + ); + } + + @Test + public void testAdd() + { + final DelegateOrMinKeyCollector collector = + new DelegateOrMinKeyCollectorFactory<>( + comparator, + QuantilesSketchKeyCollectorFactory.create(clusterBy) + ).newKeyCollector(); + + collector.add(createKey(1L), 1); + + Assert.assertTrue(collector.getDelegate().isPresent()); + Assert.assertFalse(collector.isEmpty()); + Assert.assertEquals(createKey(1L), collector.minKey()); + Assert.assertEquals(1, collector.estimatedRetainedKeys()); + Assert.assertEquals(1, collector.estimatedTotalWeight()); + } + + @Test + public void testDownSampleSingleKey() + { + final DelegateOrMinKeyCollector collector = + new DelegateOrMinKeyCollectorFactory<>( + comparator, + QuantilesSketchKeyCollectorFactory.create(clusterBy) + ).newKeyCollector(); + + collector.add(createKey(1L), 1); + Assert.assertTrue(collector.downSample()); + + Assert.assertTrue(collector.getDelegate().isPresent()); + Assert.assertFalse(collector.isEmpty()); + Assert.assertEquals(createKey(1L), collector.minKey()); + Assert.assertEquals(1, collector.estimatedRetainedKeys()); + Assert.assertEquals(1, collector.estimatedTotalWeight()); + + // Should not have actually downsampled, because the quantiles-based collector does nothing when + // downsampling on a single key. + Assert.assertEquals( + QuantilesSketchKeyCollectorFactory.SKETCH_INITIAL_K, + collector.getDelegate().get().getSketch().getK() + ); + } + + @Test + public void testDownSampleTwoKeys() + { + final DelegateOrMinKeyCollector collector = + new DelegateOrMinKeyCollectorFactory<>( + comparator, + QuantilesSketchKeyCollectorFactory.create(clusterBy) + ).newKeyCollector(); + + collector.add(createKey(1L), 1); + collector.add(createKey(1L), 1); + + Assert.assertTrue(collector.getDelegate().isPresent()); + Assert.assertFalse(collector.isEmpty()); + Assert.assertEquals(createKey(1L), collector.minKey()); + Assert.assertEquals(2, collector.estimatedRetainedKeys()); + Assert.assertEquals(2, collector.estimatedTotalWeight()); + + while (collector.getDelegate().isPresent()) { + Assert.assertTrue(collector.downSample()); + } + + Assert.assertFalse(collector.getDelegate().isPresent()); + Assert.assertFalse(collector.isEmpty()); + Assert.assertEquals(createKey(1L), collector.minKey()); + Assert.assertEquals(1, collector.estimatedRetainedKeys()); + Assert.assertEquals(1, collector.estimatedTotalWeight()); + } + + private RowKey createKey(final Object... objects) + { + return KeyTestUtils.createKey( + KeyTestUtils.createKeySignature(clusterBy.getColumns(), signature), + objects + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java new file mode 100644 index 00000000000..d853dc994ff --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java @@ -0,0 +1,243 @@ +/* + * 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.druid.msq.statistics; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.Pair; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.NavigableMap; +import java.util.NoSuchElementException; + +public class DistinctKeyCollectorTest +{ + private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new SortColumn("x", false)), 0); + private final Comparator comparator = clusterBy.keyComparator(); + private final int numKeys = 500_000; + + @Test + public void test_empty() + { + KeyCollectorTestUtils.doTest( + DistinctKeyCollectorFactory.create(clusterBy), + Collections.emptyList(), + comparator, + (testName, collector) -> { + Assert.assertTrue(collector.isEmpty()); + Assert.assertThrows(NoSuchElementException.class, collector::minKey); + Assert.assertEquals(testName, 0, collector.estimatedTotalWeight()); + Assert.assertEquals( + ClusterByPartitions.oneUniversalPartition(), + collector.generatePartitionsWithTargetWeight(1000) + ); + } + ); + } + + @Test + public void test_sequentialKeys_unweighted() + { + final List> keyWeights = KeyCollectorTestUtils.sequentialKeys(numKeys); + + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + DistinctKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals(numKeys, collector.estimatedTotalWeight(), numKeys * 0.05); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + @Test + public void test_uniformRandomKeys_unweighted() + { + final List> keyWeights = KeyCollectorTestUtils.uniformRandomKeys(numKeys); + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + DistinctKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals( + testName, + sortedKeyWeights.size(), + collector.estimatedTotalWeight(), + sortedKeyWeights.size() * 0.05 + ); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + @Test(expected = IllegalArgumentException.class) + public void test_generateWithNegativeTargetWeight_throwsException() + { + DistinctKeyCollector distinctKeyCollector = DistinctKeyCollectorFactory.create(clusterBy).newKeyCollector(); + distinctKeyCollector.generatePartitionsWithTargetWeight(-1); + } + + @Test + public void test_uniformRandomKeys_unweighted_downSampledToOneKey() + { + final List> keyWeights = KeyCollectorTestUtils.uniformRandomKeys(numKeys); + final RowKey finalMinKey = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator).firstKey(); + + KeyCollectorTestUtils.doTest( + DistinctKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + while (collector.downSample()) { + // Intentionally empty loop body. + } + + Assert.assertEquals(DistinctKeyCollector.SMALLEST_MAX_KEYS, collector.getMaxKeys()); + MatcherAssert.assertThat( + testName, + collector.estimatedRetainedKeys(), + Matchers.lessThanOrEqualTo(DistinctKeyCollector.SMALLEST_MAX_KEYS) + ); + + // Don't use verifyCollector, since this collector is downsampled so aggressively that it can't possibly + // hope to pass those tests. Grade on a curve. + final ClusterByPartitions partitions = collector.generatePartitionsWithTargetWeight(10_000); + ClusterByStatisticsCollectorImplTest.verifyPartitionsCoverKeySpace( + testName, + partitions, + finalMinKey, + comparator + ); + } + ); + } + + @Test + public void test_nonUniformRandomKeys_unweighted() + { + final List> keyWeights = KeyCollectorTestUtils.nonUniformRandomKeys(numKeys); + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + DistinctKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals( + sortedKeyWeights.size(), + collector.estimatedTotalWeight(), + sortedKeyWeights.size() * 0.05 + ); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + @Test + public void test_uniformRandomKeys_barbellWeighted() + { + final List> keyWeights = + KeyCollectorTestUtils.uniformRandomBarbellWeightedKeys(numKeys); + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + DistinctKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals( + ClusterByStatisticsCollectorImplTest.totalWeight( + sortedKeyWeights, + new ClusterByPartition(null, null), + true + ), + collector.estimatedTotalWeight(), + sortedKeyWeights.size() * 0.05 + ); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + @Test + public void test_uniformRandomKeys_inverseBarbellWeighted() + { + final List> keyWeights = + KeyCollectorTestUtils.uniformRandomInverseBarbellWeightedKeys(numKeys); + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + DistinctKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals( + ClusterByStatisticsCollectorImplTest.totalWeight( + sortedKeyWeights, + new ClusterByPartition(null, null), + true + ), + collector.estimatedTotalWeight(), + sortedKeyWeights.size() * 0.05 + ); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + private static void verifyCollector( + final DistinctKeyCollector collector, + final ClusterBy clusterBy, + final Comparator comparator, + final NavigableMap> sortedKeyWeights + ) + { + Assert.assertEquals(collector.getRetainedKeys().size(), collector.estimatedRetainedKeys()); + MatcherAssert.assertThat(collector.getRetainedKeys().size(), Matchers.lessThan(collector.getMaxKeys())); + + KeyCollectorTestUtils.verifyCollector( + collector, + clusterBy, + comparator, + sortedKeyWeights + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/KeyCollectorTestUtils.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/KeyCollectorTestUtils.java new file mode 100644 index 00000000000..f62eaeb58b0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/KeyCollectorTestUtils.java @@ -0,0 +1,296 @@ +/* + * 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.druid.msq.statistics; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.KeyTestUtils; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NavigableMap; +import java.util.Random; +import java.util.function.BiConsumer; + +public class KeyCollectorTestUtils +{ + private KeyCollectorTestUtils() + { + // No instantiation. + } + + static , TSnapshot extends KeyCollectorSnapshot> void doTest( + final KeyCollectorFactory keyCollectorFactory, + final Iterable> keys, + final Comparator comparator, + final BiConsumer testFn + ) + { + // Load into single collector, sorted order. + final TCollector sortedCollector = keyCollectorFactory.newKeyCollector(); + final List> sortedKeys = Lists.newArrayList(keys); + sortedKeys.sort(Comparator.comparing(pair -> pair.lhs, comparator)); + sortedKeys.forEach(pair -> sortedCollector.add(pair.lhs, pair.rhs)); + testFn.accept("single collector, sorted order", sortedCollector); + + // Load into single collector, reverse sorted order. + final TCollector reverseSortedCollector = keyCollectorFactory.newKeyCollector(); + Lists.reverse(sortedKeys).forEach(key -> reverseSortedCollector.add(key.lhs, key.rhs)); + testFn.accept("single collector, reverse sorted order", reverseSortedCollector); + + // Randomized load into single collector. + final TCollector randomizedCollector = keyCollectorFactory.newKeyCollector(); + final List> randomizedKeys = Lists.newArrayList(keys); + Collections.shuffle(randomizedKeys, new Random(7 /* Consistent seed from run to run */)); + randomizedKeys.forEach(pair -> randomizedCollector.add(pair.lhs, pair.rhs)); + testFn.accept("single collector, random order", randomizedCollector); + + // Split randomized load into three collectors of the same size, followed by merge. + final List threeEqualSizedCollectors = new ArrayList<>(); + threeEqualSizedCollectors.add(keyCollectorFactory.newKeyCollector()); + threeEqualSizedCollectors.add(keyCollectorFactory.newKeyCollector()); + threeEqualSizedCollectors.add(keyCollectorFactory.newKeyCollector()); + + final Iterator> iterator1 = randomizedKeys.iterator(); + for (int i = 0; iterator1.hasNext(); i++) { + final Pair key = iterator1.next(); + threeEqualSizedCollectors.get(i % threeEqualSizedCollectors.size()).add(key.lhs, key.rhs); + } + + // Regular add + threeEqualSizedCollectors.get(0).addAll(threeEqualSizedCollectors.get(1)); + + // Snapshot add + threeEqualSizedCollectors.get(0).addAll( + keyCollectorFactory.fromSnapshot(keyCollectorFactory.toSnapshot(threeEqualSizedCollectors.get(2))) + ); + + testFn.accept("three merged collectors, equal sizes", threeEqualSizedCollectors.get(0)); + + // Split randomized load into three collectors of different sizes, followed by merge. + final List threeDifferentlySizedCollectors = new ArrayList<>(); + threeDifferentlySizedCollectors.add(keyCollectorFactory.newKeyCollector()); + threeDifferentlySizedCollectors.add(keyCollectorFactory.newKeyCollector()); + threeDifferentlySizedCollectors.add(keyCollectorFactory.newKeyCollector()); + + boolean didDownsampleLargeCollector = false; + final Iterator> iterator2 = randomizedKeys.iterator(); + for (int i = 0; iterator2.hasNext(); i++) { + final Pair key = iterator2.next(); + + if (i % 100 < 2) { + // 2% of space + threeDifferentlySizedCollectors.get(0).add(key.lhs, key.rhs); + } else if (i % 100 < 20) { + // 18% of space + threeDifferentlySizedCollectors.get(1).add(key.lhs, key.rhs); + } else { + // 80% of space + threeDifferentlySizedCollectors.get(2).add(key.lhs, key.rhs); + + // Downsample once during the add process. + if (!didDownsampleLargeCollector) { + threeDifferentlySizedCollectors.get(2).downSample(); + didDownsampleLargeCollector = true; + } + } + } + + // Downsample medium, large collectors (so: two total times for the large one). + threeDifferentlySizedCollectors.get(1).downSample(); + threeDifferentlySizedCollectors.get(2).downSample(); + + threeDifferentlySizedCollectors.get(0).addAll(threeDifferentlySizedCollectors.get(1)); // Big into small + threeDifferentlySizedCollectors.get(2).addAll(threeDifferentlySizedCollectors.get(0)); // Small into big + + testFn.accept( + "three merged collectors, different sizes, with downsampling", + threeDifferentlySizedCollectors.get(2) + ); + } + + static void verifyCollector( + final KeyCollector collector, + final ClusterBy clusterBy, + final Comparator comparator, + final NavigableMap> sortedKeyWeights + ) + { + Assert.assertEquals(sortedKeyWeights.isEmpty() ? null : sortedKeyWeights.firstKey(), collector.minKey()); + + for (int targetWeight : new int[]{10_000, 50_000, 300_000}) { + final ClusterByPartitions partitions = collector.generatePartitionsWithTargetWeight(targetWeight); + final String testName = StringUtils.format("target weight = %,d", targetWeight); + + ClusterByStatisticsCollectorImplTest.verifyPartitionsCoverKeySpace( + testName, + partitions, + sortedKeyWeights.firstKey(), + comparator + ); + + ClusterByStatisticsCollectorImplTest.verifyPartitionWeights( + testName, + clusterBy, + partitions, + sortedKeyWeights, + collector instanceof DistinctKeyCollector, + targetWeight + ); + } + } + + /** + * Generates sequential keys from the range {@code [0, numKeys)}. + */ + static List> sequentialKeys(final int numKeys) + { + final List> keyWeights = new ArrayList<>(); + + for (int i = 0; i < numKeys; i++) { + final RowKey key = createSingleLongKey((long) i); + keyWeights.add(Pair.of(key, 1)); + } + + return keyWeights; + } + + /** + * Generates a certain number of keys drawn from a uniform random distribution on {@code [0, numKeys)}. + */ + static List> uniformRandomKeys(final int numKeys) + { + final List> keyWeights = new ArrayList<>(); + + // Fixed seed; "random" but deterministic. + final Random random = new Random(0); + + for (int i = 0; i < numKeys; i++) { + final long keyNumber = random.nextInt(numKeys); + final RowKey key = createSingleLongKey(keyNumber); // Uniformly random + keyWeights.add(Pair.of(key, 1)); + } + + return keyWeights; + } + + /** + * Generates a certain number of keys drawn from a uniform random distribution on {@code [0, numKeys)}. A contiguous + * 10% of the keyspace on either end is "heavyweight" and the rest of the keyspace is "lightweight". + */ + static List> uniformRandomBarbellWeightedKeys(final int numKeys) + { + final int firstTenPercent = numKeys / 10; + final int lastTenPercent = numKeys * 9 / 10; + + final List> keyWeights = new ArrayList<>(); + + // Fixed seed; "random" but deterministic. + final Random random = new Random(0); + + for (int i = 0; i < numKeys; i++) { + final long keyNumber = random.nextInt(numKeys); + final RowKey key = createSingleLongKey(keyNumber); // Uniformly random + final int weight = keyNumber < firstTenPercent && keyNumber >= lastTenPercent ? 3 : 1; + keyWeights.add(Pair.of(key, weight)); + } + + return keyWeights; + } + + /** + * Generates a certain number of keys drawn from a uniform random distribution on {@code [0, numKeys)}. A contiguous + * 10% of the keyspace on either end is "lightweight" and the rest of the keyspace is "heavyweight". + */ + static List> uniformRandomInverseBarbellWeightedKeys(final int numKeys) + { + final int firstTenPercent = numKeys / 10; + final int lastTenPercent = numKeys * 9 / 10; + + final List> keyWeights = new ArrayList<>(); + + // Fixed seed; "random" but deterministic. + final Random random = new Random(0); + + for (int i = 0; i < numKeys; i++) { + final long keyNumber = random.nextInt(numKeys); + final RowKey key = createSingleLongKey(keyNumber); // Uniformly random + final int weight = keyNumber >= firstTenPercent && keyNumber < lastTenPercent ? 3 : 1; + keyWeights.add(Pair.of(key, weight)); + } + + return keyWeights; + } + + /** + * Generates a certain number of keys drawn from a nonuniform random distribution on + * {@code [0, numKeys) ∪ {100, 201, 302, 403}}. The keys 100, 201, 302, and 403 are much more likely to occur than + * any other keys. + */ + static List> nonUniformRandomKeys(final int numKeys) + { + final List> keyWeights = new ArrayList<>(); + + // Fixed seed; "random" but deterministic. + final Random random = new Random(0); + + for (int i = 0; i < numKeys; i++) { + final long randomNumber = random.nextInt(numKeys * 10); + final long keyNumber; + + if (randomNumber < numKeys) { + keyNumber = randomNumber; + } else if (randomNumber < numKeys * 5L) { + keyNumber = 100; + } else if (randomNumber < numKeys * 8L) { + keyNumber = 201; + } else if (randomNumber < numKeys * 9L) { + keyNumber = 302; + } else { + keyNumber = 403; + } + + final RowKey key = createSingleLongKey(keyNumber); + keyWeights.add(Pair.of(key, 1)); + } + + return keyWeights; + } + + private static RowKey createSingleLongKey(final long n) + { + final RowSignature signature = RowSignature.builder().add("x", ColumnType.LONG).build(); + final List sortColumns = ImmutableList.of(new SortColumn("x", false)); + final RowSignature keySignature = KeyTestUtils.createKeySignature(sortColumns, signature); + return KeyTestUtils.createKey(keySignature, n); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java new file mode 100644 index 00000000000..0555c7c5d55 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java @@ -0,0 +1,200 @@ +/* + * 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.druid.msq.statistics; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.Pair; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.NavigableMap; +import java.util.NoSuchElementException; + +public class QuantilesSketchKeyCollectorTest +{ + private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new SortColumn("x", false)), 0); + private final Comparator comparator = clusterBy.keyComparator(); + private final int numKeys = 500_000; + + @Test + public void test_empty() + { + KeyCollectorTestUtils.doTest( + QuantilesSketchKeyCollectorFactory.create(clusterBy), + Collections.emptyList(), + comparator, + (testName, collector) -> { + Assert.assertTrue(collector.isEmpty()); + Assert.assertThrows(NoSuchElementException.class, collector::minKey); + Assert.assertEquals(testName, 0, collector.estimatedTotalWeight()); + Assert.assertEquals( + ClusterByPartitions.oneUniversalPartition(), + collector.generatePartitionsWithTargetWeight(1000) + ); + } + ); + } + + @Test + public void test_sequentialKeys_unweighted() + { + final List> keyWeights = KeyCollectorTestUtils.sequentialKeys(numKeys); + + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + QuantilesSketchKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals(testName, numKeys, collector.estimatedTotalWeight()); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + @Test + public void test_uniformRandomKeys_unweighted() + { + final List> keyWeights = KeyCollectorTestUtils.uniformRandomKeys(numKeys); + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + QuantilesSketchKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals(testName, numKeys, collector.estimatedTotalWeight()); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + @Test + public void test_uniformRandomKeys_unweighted_downSampledToSmallestSize() + { + final List> keyWeights = KeyCollectorTestUtils.uniformRandomKeys(numKeys); + final RowKey finalMinKey = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator).firstKey(); + + KeyCollectorTestUtils.doTest( + QuantilesSketchKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + while (collector.downSample()) { + // Intentionally empty loop body. + } + + Assert.assertEquals(testName, 2, collector.getSketch().getK()); + Assert.assertEquals(testName, 22, collector.estimatedRetainedKeys()); + + // Don't use verifyCollector, since this collector is downsampled so aggressively that it can't possibly + // hope to pass those tests. Grade on a curve. + final ClusterByPartitions partitions = collector.generatePartitionsWithTargetWeight(10_000); + ClusterByStatisticsCollectorImplTest.verifyPartitionsCoverKeySpace( + testName, + partitions, + finalMinKey, + comparator + ); + } + ); + } + + @Test + public void test_uniformRandomKeys_barbellWeighted() + { + final List> keyWeights = + KeyCollectorTestUtils.uniformRandomBarbellWeightedKeys(numKeys); + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + QuantilesSketchKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals( + testName, + ClusterByStatisticsCollectorImplTest.totalWeight( + sortedKeyWeights, + new ClusterByPartition(null, null), + false + ), + collector.estimatedTotalWeight() + ); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + @Test + public void test_uniformRandomKeys_inverseBarbellWeighted() + { + final List> keyWeights = + KeyCollectorTestUtils.uniformRandomInverseBarbellWeightedKeys(numKeys); + final NavigableMap> sortedKeyWeights = + ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); + + KeyCollectorTestUtils.doTest( + QuantilesSketchKeyCollectorFactory.create(clusterBy), + keyWeights, + comparator, + (testName, collector) -> { + Assert.assertEquals( + testName, + ClusterByStatisticsCollectorImplTest.totalWeight( + sortedKeyWeights, + new ClusterByPartition(null, null), + false + ), + collector.estimatedTotalWeight() + ); + verifyCollector(collector, clusterBy, comparator, sortedKeyWeights); + } + ); + } + + private static void verifyCollector( + final QuantilesSketchKeyCollector collector, + final ClusterBy clusterBy, + final Comparator comparator, + final NavigableMap> sortedKeyWeights + ) + { + KeyCollectorTestUtils.verifyCollector( + collector, + clusterBy, + comparator, + sortedKeyWeights + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java new file mode 100644 index 00000000000..ae641fc7c79 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -0,0 +1,1068 @@ +/* + * 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.druid.msq.test; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.io.ByteStreams; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import com.google.inject.util.Providers; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.IndexingServiceTuningConfigModule; +import org.apache.druid.guice.JoinableFactoryModule; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.hll.HyperLogLogCollector; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.guice.MSQDurableStorageModule; +import org.apache.druid.msq.guice.MSQExternalDataSourceModule; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.guice.MSQSqlModule; +import org.apache.druid.msq.guice.MultiStageQuery; +import org.apache.druid.msq.indexing.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.error.InsertLockPreemptedFaultTest; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.report.MSQResultsReport; +import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.msq.querykit.LazyResourceHolder; +import org.apache.druid.msq.sql.MSQTaskQueryMaker; +import org.apache.druid.msq.sql.MSQTaskSqlEngine; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.ForwardingQueryProcessingPool; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.TestGroupByBuffers; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPuller; +import org.apache.druid.segment.loading.LocalDataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; +import org.apache.druid.segment.loading.LocalLoadSpec; +import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; +import org.apache.druid.server.security.AuthTestUtils; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.SqlQueryPlus; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.external.ExternalDataSource; +import org.apache.druid.sql.calcite.planner.CalciteRulesManager; +import org.apache.druid.sql.calcite.planner.PlannerConfig; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.planner.PlannerFactory; +import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; +import org.apache.druid.sql.calcite.schema.NoopDruidSchemaManager; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.apache.druid.sql.calcite.view.InProcessViewManager; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.StorageConnectorProvider; +import org.apache.druid.storage.local.LocalFileStorageConnectorProvider; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.PruneLoadSpec; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.easymock.EasyMock; +import org.hamcrest.Matcher; +import org.hamcrest.MatcherAssert; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE1; +import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE2; +import static org.apache.druid.sql.calcite.util.CalciteTests.ROWS1; +import static org.apache.druid.sql.calcite.util.CalciteTests.ROWS2; + +/** + * Base test runner for running MSQ unit tests. It sets up multi stage query execution environment + * and populates data for the datasources. The runner does not go via the HTTP layer for communication between the + * various MSQ processes. + * + * Controller -> Coordinator (Coordinator is mocked) + * + * In the Ut's we go from: + * {@link MSQTaskQueryMaker} -> {@link MSQTestOverlordServiceClient} -> {@link Controller} + * + * + * Controller -> Worker communication happens in {@link MSQTestControllerContext} + * + * Worker -> Controller communication happens in {@link MSQTestControllerClient} + * + * Controller -> Overlord communication happens in {@link MSQTestTaskActionClient} + */ +public class MSQTestBase extends BaseCalciteQueryTest +{ + public static final Map DEFAULT_MSQ_CONTEXT = + ImmutableMap.builder() + .put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, true) + .put(PlannerContext.CTX_SQL_QUERY_ID, "test-query") + .put(QueryContexts.FINALIZE_KEY, true) + .build(); + + public static final Map + ROLLUP_CONTEXT = ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false) + .put(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false) + .build(); + + public final boolean useDefault = NullHandling.replaceWithDefault(); + + protected File localFileStorageDir; + private static final Logger log = new Logger(MSQTestBase.class); + private ObjectMapper objectMapper; + private MSQTestOverlordServiceClient indexingServiceClient; + private SqlStatementFactory sqlStatementFactory; + private IndexIO indexIO; + + private MSQTestSegmentManager segmentManager; + private SegmentCacheManager segmentCacheManager; + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private TestGroupByBuffers groupByBuffers; + protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy(WorkerMemoryParameters.compute( + WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, + 2, + 10, + 2 + )); + + + @After + public void tearDown2() + { + groupByBuffers.close(); + } + + @Before + public void setUp2() + { + Injector secondInjector = GuiceInjectors.makeStartupInjector(); + + groupByBuffers = TestGroupByBuffers.createDefault(); + + ObjectMapper secondMapper = setupObjectMapper(secondInjector); + indexIO = new IndexIO(secondMapper, () -> 0); + + try { + segmentCacheManager = new SegmentCacheManagerFactory(secondMapper).manufacturate(temporaryFolder.newFolder("test")); + } + catch (IOException exception) { + throw new ISE(exception, "Unable to create segmentCacheManager"); + } + + MSQSqlModule sqlModule = new MSQSqlModule(); + + segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO); + + Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return "test"; + } + }; + + GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); + + binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig); + binder.bind(new TypeLiteral>() + { + }).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON)); + binder.bind(QueryProcessingPool.class) + .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); + binder.bind(DataSegmentProvider.class) + .toInstance((dataSegment, channelCounters) -> + new LazyResourceHolder<>(getSupplierForSegment(dataSegment))); + binder.bind(IndexIO.class).toInstance(indexIO); + binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(walker); + + binder.bind(GroupByStrategySelector.class) + .toInstance(GroupByQueryRunnerTest.makeQueryRunnerFactory(groupByQueryConfig, groupByBuffers) + .getStrategySelector()); + + LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig(); + try { + config.storageDirectory = tmpFolder.newFolder("localsegments"); + } + catch (IOException e) { + throw new ISE(e, "Unable to create folder"); + } + binder.bind(DataSegmentPusher.class).toInstance(new MSQTestDelegateDataSegmentPusher( + new LocalDataSegmentPusher(config), + segmentManager + )); + binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); + binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(false); + // Client is not used in tests + binder.bind(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)) + .toProvider(Providers.of(null)); + // fault tolerance module + try { + JsonConfigProvider.bind( + binder, + MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_PREFIX, + StorageConnectorProvider.class, + MultiStageQuery.class + ); + localFileStorageDir = tmpFolder.newFolder("fault"); + binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class)) + .toProvider(new LocalFileStorageConnectorProvider(localFileStorageDir)); + } + catch (IOException e) { + throw new ISE(e, "Unable to create setup storage connector"); + } + } + }, + binder -> { + // Requirements of JoinableFactoryModule + binder.bind(SegmentManager.class).toInstance(EasyMock.createMock(SegmentManager.class)); + binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance( + LookupEnabledTestExprMacroTable.createTestLookupProvider(Collections.emptyMap()) + ); + }, + new JoinableFactoryModule(), + new IndexingServiceTuningConfigModule(), + new MSQIndexingModule(), + new MSQSqlModule(), + new MSQExternalDataSourceModule() + )); + + objectMapper = setupObjectMapper(injector); + objectMapper.registerModules(sqlModule.getJacksonModules()); + + indexingServiceClient = new MSQTestOverlordServiceClient( + objectMapper, + injector, + new MSQTestTaskActionClient(objectMapper), + workerMemoryParameters + ); + final InProcessViewManager viewManager = new InProcessViewManager(CalciteTests.DRUID_VIEW_MACRO_FACTORY); + DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema( + conglomerate, + walker, + new PlannerConfig(), + viewManager, + new NoopDruidSchemaManager(), + CalciteTests.TEST_AUTHORIZER_MAPPER + ); + + final SqlEngine engine = new MSQTaskSqlEngine( + indexingServiceClient, + queryJsonMapper.copy().registerModules(new MSQSqlModule().getJacksonModules()) + ); + + PlannerFactory plannerFactory = new PlannerFactory( + rootSchema, + CalciteTests.createOperatorTable(), + CalciteTests.createExprMacroTable(), + PLANNER_CONFIG_DEFAULT, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + objectMapper, + CalciteTests.DRUID_SCHEMA_NAME, + new CalciteRulesManager(ImmutableSet.of()) + ); + + sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory); + } + + /** + * Returns query context expected for a scan query. Same as {@link #DEFAULT_MSQ_CONTEXT}, but + * includes {@link DruidQuery#CTX_SCAN_SIGNATURE}. + */ + protected Map defaultScanQueryContext(final RowSignature signature) + { + try { + return ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put(DruidQuery.CTX_SCAN_SIGNATURE, queryJsonMapper.writeValueAsString(signature)) + .build(); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + /** + * Helper method that copies a resource to a temporary file, then returns it. + */ + protected File getResourceAsTemporaryFile(final String resource) throws IOException + { + final File file = temporaryFolder.newFile(); + final InputStream stream = getClass().getResourceAsStream(resource); + + if (stream == null) { + throw new IOE("No such resource [%s]", resource); + } + + ByteStreams.copy(stream, Files.newOutputStream(file.toPath())); + return file; + } + + @Nonnull + private Supplier> getSupplierForSegment(SegmentId segmentId) + { + if (segmentManager.getSegment(segmentId) == null) { + final QueryableIndex index; + TemporaryFolder temporaryFolder = new TemporaryFolder(); + try { + temporaryFolder.create(); + } + catch (IOException e) { + throw new ISE(e, "Unable to create temporary folder for tests"); + } + try { + switch (segmentId.getDataSource()) { + case DATASOURCE1: + IncrementalIndexSchema foo1Schema = new IncrementalIndexSchema.Builder() + .withMetrics( + new CountAggregatorFactory("cnt"), + new FloatSumAggregatorFactory("m1", "m1"), + new DoubleSumAggregatorFactory("m2", "m2"), + new HyperUniquesAggregatorFactory("unique_dim1", "dim1") + ) + .withRollup(false) + .build(); + index = IndexBuilder + .create() + .tmpDir(new File(temporaryFolder.newFolder(), "1")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(foo1Schema) + .rows(ROWS1) + .buildMMappedIndex(); + break; + case DATASOURCE2: + final IncrementalIndexSchema indexSchemaDifferentDim3M1Types = new IncrementalIndexSchema.Builder() + .withDimensionsSpec( + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dim3") + ) + ) + ) + .withMetrics( + new CountAggregatorFactory("cnt"), + new LongSumAggregatorFactory("m1", "m1"), + new DoubleSumAggregatorFactory("m2", "m2"), + new HyperUniquesAggregatorFactory("unique_dim1", "dim1") + ) + .withRollup(false) + .build(); + index = IndexBuilder + .create() + .tmpDir(new File(temporaryFolder.newFolder(), "1")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(indexSchemaDifferentDim3M1Types) + .rows(ROWS2) + .buildMMappedIndex(); + break; + default: + throw new ISE("Cannot query segment %s in test runner", segmentId); + + } + } + catch (IOException e) { + throw new ISE(e, "Unable to load index for segment %s", segmentId); + } + Segment segment = new Segment() + { + @Override + public SegmentId getId() + { + return segmentId; + } + + @Override + public Interval getDataInterval() + { + return segmentId.getInterval(); + } + + @Nullable + @Override + public QueryableIndex asQueryableIndex() + { + return index; + } + + @Override + public StorageAdapter asStorageAdapter() + { + return new QueryableIndexStorageAdapter(index); + } + + @Override + public void close() + { + } + }; + segmentManager.addSegment(segment); + } + return new Supplier>() + { + @Override + public Pair get() + { + return new Pair<>(segmentManager.getSegment(segmentId), Closer.create()); + } + }; + } + + public SelectTester testSelectQuery() + { + return new SelectTester(); + } + + public IngestTester testIngestQuery() + { + return new IngestTester(); + } + + private ObjectMapper setupObjectMapper(Injector injector) + { + ObjectMapper mapper = injector.getInstance(ObjectMapper.class) + .registerModules(new SimpleModule(IndexingServiceTuningConfigModule.class.getSimpleName()) + .registerSubtypes( + new NamedType(IndexTask.IndexTuningConfig.class, "index"), + new NamedType( + ParallelIndexTuningConfig.class, + "index_parallel" + ), + new NamedType( + CompactionTask.CompactionTuningConfig.class, + "compaction" + ) + ).registerSubtypes(ExternalDataSource.class)); + mapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ObjectMapper.class, mapper) + .addValue(Injector.class, injector) + .addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT) + .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) + .addValue(ExprMacroTable.class, CalciteTests.createExprMacroTable()) + ); + + mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); + + // This should be reusing guice instead of using static classes + InsertLockPreemptedFaultTest.LockPreemptedHelper.preempt(false); + + return mapper; + } + + private String runMultiStageQuery(String query, Map context) + { + final DirectStatement stmt = sqlStatementFactory.directStatement( + new SqlQueryPlus( + query, + new QueryContext(context), + Collections.emptyList(), + CalciteTests.REGULAR_USER_AUTH_RESULT + ) + ); + + final List sequence = stmt.execute().toList(); + return (String) Iterables.getOnlyElement(sequence)[0]; + } + + private MSQTaskReportPayload getPayloadOrThrow(String controllerTaskId) + { + MSQTaskReportPayload payload = + (MSQTaskReportPayload) indexingServiceClient.getReportForTask(controllerTaskId) + .get(MSQTaskReport.REPORT_KEY) + .getPayload(); + if (payload.getStatus().getStatus().isFailure()) { + throw new ISE( + "Query task [%s] failed due to %s", + controllerTaskId, + payload.getStatus().getErrorReport().toString() + ); + } + + if (!payload.getStatus().getStatus().isComplete()) { + throw new ISE("Query task [%s] should have finished", controllerTaskId); + } + + return payload; + } + + private MSQErrorReport getErrorReportOrThrow(String controllerTaskId) + { + MSQTaskReportPayload payload = + (MSQTaskReportPayload) indexingServiceClient.getReportForTask(controllerTaskId) + .get(MSQTaskReport.REPORT_KEY) + .getPayload(); + if (!payload.getStatus().getStatus().isFailure()) { + throw new ISE( + "Query task [%s] was supposed to fail", + controllerTaskId + ); + } + + if (!payload.getStatus().getStatus().isComplete()) { + throw new ISE("Query task [%s] should have finished", controllerTaskId); + } + + return payload.getStatus().getErrorReport(); + + } + + private void assertMSQSpec(MSQSpec expectedMSQSpec, MSQSpec querySpecForTask) + { + Assert.assertEquals(expectedMSQSpec.getQuery(), querySpecForTask.getQuery()); + Assert.assertEquals(expectedMSQSpec.getAssignmentStrategy(), querySpecForTask.getAssignmentStrategy()); + Assert.assertEquals(expectedMSQSpec.getColumnMappings(), querySpecForTask.getColumnMappings()); + Assert.assertEquals(expectedMSQSpec.getDestination(), querySpecForTask.getDestination()); + } + + private void assertTuningConfig( + MSQTuningConfig expectedTuningConfig, + MSQTuningConfig tuningConfig + ) + { + Assert.assertEquals( + expectedTuningConfig.getMaxNumWorkers(), + tuningConfig.getMaxRowsInMemory() + ); + Assert.assertEquals( + expectedTuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxRowsInMemory() + ); + Assert.assertEquals( + expectedTuningConfig.getRowsPerSegment(), + tuningConfig.getRowsPerSegment() + ); + } + + private Optional>> getSignatureWithRows(MSQResultsReport resultsReport) + { + if (resultsReport == null) { + return Optional.empty(); + } else { + RowSignature rowSignature = resultsReport.getSignature(); + Yielder yielder = resultsReport.getResultYielder(); + List rows = new ArrayList<>(); + while (!yielder.isDone()) { + rows.add(yielder.get()); + yielder = yielder.next(null); + } + try { + yielder.close(); + } + catch (IOException e) { + throw new ISE("Unable to get results from the report"); + } + + return Optional.of(new Pair(rowSignature, rows)); + } + } + + + public abstract class MSQTester + { + protected String sql = null; + protected Map queryContext = DEFAULT_MSQ_CONTEXT; + protected RowSignature expectedRowSignature = null; + protected MSQSpec expectedMSQSpec = null; + protected MSQTuningConfig expectedTuningConfig = null; + protected Set expectedSegments = null; + protected List expectedResultRows = null; + protected Matcher expectedValidationErrorMatcher = null; + protected Matcher expectedExecutionErrorMatcher = null; + protected MSQFault expectedMSQFault = null; + + private boolean hasRun = false; + + public Builder setSql(String sql) + { + this.sql = sql; + return (Builder) this; + } + + public Builder setQueryContext(Map queryContext) + { + this.queryContext = queryContext; + return (Builder) this; + } + + public Builder setExpectedRowSignature(RowSignature expectedRowSignature) + { + Preconditions.checkArgument(!expectedRowSignature.equals(RowSignature.empty()), "Row signature cannot be empty"); + this.expectedRowSignature = expectedRowSignature; + return (Builder) this; + } + + public Builder setExpectedSegment(Set expectedSegments) + { + Preconditions.checkArgument(!expectedSegments.isEmpty(), "Segments cannot be empty"); + this.expectedSegments = expectedSegments; + return (Builder) this; + } + + public Builder setExpectedResultRows(List expectedResultRows) + { + Preconditions.checkArgument(expectedResultRows.size() > 0, "Results rows cannot be empty"); + this.expectedResultRows = expectedResultRows; + return (Builder) this; + } + + + public Builder setExpectedMSQSpec(MSQSpec expectedMSQSpec) + { + this.expectedMSQSpec = expectedMSQSpec; + return (Builder) this; + } + + + public Builder setExpectedValidationErrorMatcher(Matcher expectedValidationErrorMatcher) + { + this.expectedValidationErrorMatcher = expectedValidationErrorMatcher; + return (Builder) this; + } + + public Builder setExpectedExecutionErrorMatcher(Matcher expectedExecutionErrorMatcher) + { + this.expectedExecutionErrorMatcher = expectedExecutionErrorMatcher; + return (Builder) this; + } + + public Builder setExpectedMSQFault(MSQFault MSQFault) + { + this.expectedMSQFault = MSQFault; + return (Builder) this; + } + + public void verifyPlanningErrors() + { + Preconditions.checkArgument(expectedValidationErrorMatcher != null, "Validation error matcher cannot be null"); + Preconditions.checkArgument(sql != null, "Sql cannot be null"); + readyToRun(); + + final Throwable e = Assert.assertThrows( + Throwable.class, + () -> runMultiStageQuery(sql, queryContext) + ); + + MatcherAssert.assertThat(e, expectedValidationErrorMatcher); + } + + protected void readyToRun() + { + if (!hasRun) { + hasRun = true; + } else { + throw new ISE("Use one @Test method per tester"); + } + } + } + + public class IngestTester extends MSQTester + { + private String expectedDataSource; + + private Class expectedShardSpec = NumberedShardSpec.class; + + private boolean expectedRollUp = false; + + private Granularity expectedQueryGranularity = Granularities.NONE; + + private List expectedAggregatorFactories = new ArrayList<>(); + + private List expectedDestinationIntervals = null; + + private IngestTester() + { + // nothing to do + } + + + public IngestTester setExpectedDataSource(String expectedDataSource) + { + this.expectedDataSource = expectedDataSource; + return this; + } + + public IngestTester setExpectedShardSpec(Class expectedShardSpec) + { + this.expectedShardSpec = expectedShardSpec; + return this; + } + + public IngestTester setExpectedDestinationIntervals(List expectedDestinationIntervals) + { + this.expectedDestinationIntervals = expectedDestinationIntervals; + return this; + } + + public IngestTester setExpectedRollUp(boolean expectedRollUp) + { + this.expectedRollUp = expectedRollUp; + return this; + } + + public IngestTester setExpectedQueryGranularity(Granularity expectedQueryGranularity) + { + this.expectedQueryGranularity = expectedQueryGranularity; + return this; + } + + public IngestTester addExpectedAggregatorFactory(AggregatorFactory aggregatorFactory) + { + expectedAggregatorFactories.add(aggregatorFactory); + return this; + } + + public void verifyResults() + { + Preconditions.checkArgument(sql != null, "sql cannot be null"); + Preconditions.checkArgument(queryContext != null, "queryContext cannot be null"); + Preconditions.checkArgument(expectedDataSource != null, "dataSource cannot be null"); + Preconditions.checkArgument(expectedRowSignature != null, "expectedRowSignature cannot be null"); + Preconditions.checkArgument( + expectedResultRows != null || expectedMSQFault != null, + "atleast one of expectedResultRows or expectedMSQFault should be set to non null" + ); + Preconditions.checkArgument(expectedShardSpec != null, "shardSpecClass cannot be null"); + readyToRun(); + try { + String controllerId = runMultiStageQuery(sql, queryContext); + if (expectedMSQFault != null) { + MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); + Assert.assertEquals( + expectedMSQFault.getCodeWithMessage(), + msqErrorReport.getFault().getCodeWithMessage() + ); + return; + } + getPayloadOrThrow(controllerId); + MSQSpec foundSpec = indexingServiceClient.getQuerySpecForTask(controllerId); + log.info( + "found generated segments: %s", + segmentManager.getAllDataSegments().stream().map(s -> s.toString()).collect( + Collectors.joining("\n")) + ); + //check if segments are created + Assert.assertNotEquals(0, segmentManager.getAllDataSegments().size()); + + + String foundDataSource = null; + SortedMap>> segmentIdVsOutputRowsMap = new TreeMap<>(); + for (DataSegment dataSegment : segmentManager.getAllDataSegments()) { + + //Assert shard spec class + Assert.assertEquals(expectedShardSpec, dataSegment.getShardSpec().getClass()); + if (foundDataSource == null) { + foundDataSource = dataSegment.getDataSource(); + + } else if (!foundDataSource.equals(dataSegment.getDataSource())) { + throw new ISE( + "Expected only one datasource in the list of generated segments found [%s,%s]", + foundDataSource, + dataSegment.getDataSource() + ); + } + final QueryableIndex queryableIndex = indexIO.loadIndex(segmentCacheManager.getSegmentFiles( + dataSegment)); + final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex); + + // assert rowSignature + Assert.assertEquals(expectedRowSignature, storageAdapter.getRowSignature()); + + // assert rollup + Assert.assertEquals(expectedRollUp, queryableIndex.getMetadata().isRollup()); + + // asset query granularity + Assert.assertEquals(expectedQueryGranularity, queryableIndex.getMetadata().getQueryGranularity()); + + // assert aggregator factories + Assert.assertArrayEquals( + expectedAggregatorFactories.toArray(new AggregatorFactory[0]), + queryableIndex.getMetadata().getAggregators() + ); + + for (List row : FrameTestUtil.readRowsFromAdapter(storageAdapter, null, false).toList()) { + // transforming rows for sketch assertions + List transformedRow = row.stream() + .map(r -> { + if (r instanceof HyperLogLogCollector) { + return ((HyperLogLogCollector) r).estimateCardinalityRound(); + } else { + return r; + } + }) + .collect(Collectors.toList()); + segmentIdVsOutputRowsMap.computeIfAbsent(dataSegment.getId(), r -> new ArrayList<>()).add(transformedRow); + } + } + + log.info("Found spec: %s", objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(foundSpec)); + List transformedOutputRows = segmentIdVsOutputRowsMap.values() + .stream() + .flatMap(Collection::stream) + .map(List::toArray) + .collect(Collectors.toList()); + + log.info( + "Found rows which are sorted forcefully %s", + transformedOutputRows.stream().map(a -> Arrays.toString(a)).collect(Collectors.joining("\n")) + ); + + + // assert data source name + Assert.assertEquals(expectedDataSource, foundDataSource); + // assert spec + if (expectedMSQSpec != null) { + assertMSQSpec(expectedMSQSpec, foundSpec); + } + if (expectedTuningConfig != null) { + assertTuningConfig(expectedTuningConfig, foundSpec.getTuningConfig()); + } + if (expectedDestinationIntervals != null) { + Assert.assertNotNull(foundSpec); + DataSourceMSQDestination destination = (DataSourceMSQDestination) foundSpec.getDestination(); + Assert.assertEquals(expectedDestinationIntervals, destination.getReplaceTimeChunks()); + } + if (expectedSegments != null) { + Assert.assertEquals(expectedSegments, segmentIdVsOutputRowsMap.keySet()); + for (Object[] row : transformedOutputRows) { + List diskSegmentList = segmentIdVsOutputRowsMap.keySet() + .stream() + .filter(segmentId -> segmentId.getInterval() + .contains((Long) row[0])) + .collect(Collectors.toList()); + if (diskSegmentList.size() != 1) { + throw new IllegalStateException("Single key in multiple partitions"); + } + SegmentId diskSegment = diskSegmentList.get(0); + // Checking if the row belongs to the correct segment interval + Assert.assertTrue(segmentIdVsOutputRowsMap.get(diskSegment).contains(Arrays.asList(row))); + } + } + // assert results + assertResultsEquals(sql, expectedResultRows, transformedOutputRows); + } + catch (Exception e) { + throw new ISE(e, "Query %s failed", sql); + } + } + + public void verifyExecutionError() + { + Preconditions.checkArgument(sql != null, "sql cannot be null"); + Preconditions.checkArgument(queryContext != null, "queryContext cannot be null"); + Preconditions.checkArgument(expectedExecutionErrorMatcher != null, "Execution error matcher cannot be null"); + readyToRun(); + try { + String controllerId = runMultiStageQuery(sql, queryContext); + getPayloadOrThrow(controllerId); + Assert.fail(StringUtils.format("Query did not throw an exception (sql = [%s])", sql)); + } + catch (Exception e) { + MatcherAssert.assertThat( + StringUtils.format("Query error did not match expectations (sql = [%s])", sql), + e, + expectedExecutionErrorMatcher + ); + } + } + } + + public class SelectTester extends MSQTester + { + private SelectTester() + { + // nothing to do + } + + // Made the visibility public to aid adding ut's easily with minimum parameters to set. + @Nullable + public Pair>> runQueryWithResult() + { + readyToRun(); + Preconditions.checkArgument(sql != null, "sql cannot be null"); + Preconditions.checkArgument(queryContext != null, "queryContext cannot be null"); + + try { + String controllerId = runMultiStageQuery(sql, queryContext); + + if (expectedMSQFault != null) { + MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); + Assert.assertEquals( + expectedMSQFault.getCodeWithMessage(), + msqErrorReport.getFault().getCodeWithMessage() + ); + return null; + } + + MSQTaskReportPayload payload = getPayloadOrThrow(controllerId); + + if (payload.getStatus().getErrorReport() != null) { + throw new ISE("Query %s failed due to %s", sql, payload.getStatus().getErrorReport().toString()); + } else { + Optional>> rowSignatureListPair = getSignatureWithRows(payload.getResults()); + if (!rowSignatureListPair.isPresent()) { + throw new ISE("Query successful but no results found"); + } + log.info("found row signature %s", rowSignatureListPair.get().lhs); + log.info(rowSignatureListPair.get().rhs.stream() + .map(row -> Arrays.toString(row)) + .collect(Collectors.joining("\n"))); + + MSQSpec spec = indexingServiceClient.getQuerySpecForTask(controllerId); + log.info("Found spec: %s", objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(spec)); + return new Pair<>(spec, rowSignatureListPair.get()); + } + } + catch (Exception e) { + if (expectedExecutionErrorMatcher == null) { + throw new ISE(e, "Query %s failed", sql); + } + MatcherAssert.assertThat(e, expectedExecutionErrorMatcher); + return null; + } + } + + public void verifyResults() + { + Preconditions.checkArgument(expectedResultRows != null, "Result rows cannot be null"); + Preconditions.checkArgument(expectedRowSignature != null, "Row signature cannot be null"); + Preconditions.checkArgument(expectedMSQSpec != null, "MultiStageQuery Query spec not "); + Pair>> specAndResults = runQueryWithResult(); + + if (specAndResults == null) { // A fault was expected and the assertion has been done in the runQueryWithResult + return; + } + + Assert.assertEquals(expectedRowSignature, specAndResults.rhs.lhs); + assertResultsEquals(sql, expectedResultRows, specAndResults.rhs.rhs); + assertMSQSpec(expectedMSQSpec, specAndResults.lhs); + } + + public void verifyExecutionError() + { + Preconditions.checkArgument(expectedExecutionErrorMatcher != null, "Execution error matcher cannot be null"); + if (runQueryWithResult() != null) { + throw new ISE("Query %s did not throw an exception", sql); + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java new file mode 100644 index 00000000000..0e00a8e3edf --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.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.druid.msq.test; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; +import java.util.List; + +public class MSQTestControllerClient implements ControllerClient +{ + private final Controller controller; + + public MSQTestControllerClient(Controller controller) + { + this.controller = controller; + } + + @Override + public void postKeyStatistics( + StageId stageId, + int workerNumber, + ClusterByStatisticsSnapshot keyStatistics + ) + { + try { + controller.updateStatus(stageId.getStageNumber(), workerNumber, keyStatistics); + } + catch (Exception e) { + throw new ISE(e, "unable to post key statistics"); + } + } + + @Override + public void postCounters(CounterSnapshotsTree snapshotsTree) + { + if (snapshotsTree != null) { + controller.updateCounters(snapshotsTree); + } + } + + @Override + public void postResultsComplete(StageId stageId, int workerNumber, @Nullable Object resultObject) + { + controller.resultsComplete(stageId.getQueryId(), stageId.getStageNumber(), workerNumber, resultObject); + } + + @Override + public void postWorkerError(String workerId, MSQErrorReport errorWrapper) + { + controller.workerError(errorWrapper); + } + + @Override + public void postWorkerWarning(String workerId, List MSQErrorReports) + { + controller.workerWarning(MSQErrorReports); + } + + @Override + public List getTaskList() + { + return controller.getTaskIds(); + } + + @Override + public void close() + { + // Nothing to do. + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java new file mode 100644 index 00000000000..ef17a8d3237 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -0,0 +1,261 @@ +/* + * 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.druid.msq.test; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Injector; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.exec.WorkerImpl; +import org.apache.druid.msq.exec.WorkerManagerClient; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.indexing.MSQWorkerTask; +import org.apache.druid.server.DruidNode; +import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; + +public class MSQTestControllerContext implements ControllerContext +{ + private static final Logger log = new Logger(MSQTestControllerContext.class); + private final TaskActionClient taskActionClient; + private final Map inMemoryWorkers = new HashMap<>(); + private final ConcurrentMap statusMap = new ConcurrentHashMap<>(); + private final ListeningExecutorService executor = MoreExecutors.listeningDecorator(Execs.singleThreaded( + "MultiStageQuery-test-controller-client")); + private final CoordinatorClient coordinatorClient; + private final DruidNode node = new DruidNode( + "controller", + "localhost", + true, + 8080, + 8081, + true, + false + ); + private final Injector injector; + private final ObjectMapper mapper; + + private Controller controller; + private Map report = null; + private final WorkerMemoryParameters workerMemoryParameters; + + public MSQTestControllerContext( + ObjectMapper mapper, + Injector injector, + TaskActionClient taskActionClient, + WorkerMemoryParameters workerMemoryParameters + ) + { + this.mapper = mapper; + this.injector = injector; + this.taskActionClient = taskActionClient; + coordinatorClient = Mockito.mock(CoordinatorClient.class); + Mockito.when(coordinatorClient.fetchUsedSegmentsInDataSourceForIntervals( + ArgumentMatchers.anyString(), + ArgumentMatchers.anyList() + ) + ).thenAnswer(invocation -> + (injector.getInstance(SpecificSegmentsQuerySegmentWalker.class) + .getSegments() + .stream() + .filter(dataSegment -> dataSegment.getDataSource().equals(invocation.getArguments()[0])) + .collect(Collectors.toList()) + ) + ); + this.workerMemoryParameters = workerMemoryParameters; + } + + WorkerManagerClient workerManagerClient = new WorkerManagerClient() + { + @Override + public String run(String controllerId, MSQWorkerTask task) + { + if (controller == null) { + throw new ISE("Controller needs to be set using the register method"); + } + Worker worker = new WorkerImpl( + task, + new MSQTestWorkerContext(inMemoryWorkers, controller, mapper, injector, workerMemoryParameters) + ); + inMemoryWorkers.put(task.getId(), worker); + statusMap.put(task.getId(), TaskStatus.running(task.getId())); + + ListenableFuture future = executor.submit(worker::run); + + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable TaskStatus result) + { + statusMap.put(task.getId(), result); + } + + @Override + public void onFailure(Throwable t) + { + log.error(t, "error running worker task %s", task.getId()); + statusMap.put(task.getId(), TaskStatus.failure(task.getId(), t.getMessage())); + } + }); + + return task.getId(); + } + + @Override + public Map statuses(Set taskIds) + { + Map result = new HashMap<>(); + for (String taskId : taskIds) { + TaskStatus taskStatus = statusMap.get(taskId); + if (taskStatus != null) { + + if (taskStatus.getStatusCode().equals(TaskState.RUNNING) && !inMemoryWorkers.containsKey(taskId)) { + result.put(taskId, new TaskStatus(taskId, TaskState.FAILED, 0, null, null)); + } else { + result.put( + taskId, + new TaskStatus( + taskStatus.getId(), + taskStatus.getStatusCode(), + taskStatus.getDuration(), + null, + null + ) + ); + } + } + } + return result; + } + + @Override + public TaskLocation location(String workerId) + { + final TaskStatus status = statusMap.get(workerId); + if (status != null && status.getStatusCode().equals(TaskState.RUNNING) && inMemoryWorkers.containsKey(workerId)) { + return TaskLocation.create("host-" + workerId, 1, -1); + } else { + return TaskLocation.unknown(); + } + } + + @Override + public void cancel(String workerId) + { + final Worker worker = inMemoryWorkers.remove(workerId); + if (worker != null) { + worker.stopGracefully(); + } + } + + @Override + public void close() + { + //do nothing + } + }; + + @Override + public ObjectMapper jsonMapper() + { + return mapper; + } + + @Override + public Injector injector() + { + return injector; + } + + @Override + public DruidNode selfNode() + { + return node; + } + + @Override + public CoordinatorClient coordinatorClient() + { + return coordinatorClient; + } + + @Override + public TaskActionClient taskActionClient() + { + return taskActionClient; + } + + @Override + public WorkerManagerClient workerManager() + { + return workerManagerClient; + } + + @Override + public void registerController(Controller controller, Closer closer) + { + this.controller = controller; + } + + @Override + public WorkerClient taskClientFor(Controller controller) + { + return new MSQTestWorkerClient(inMemoryWorkers); + } + + @Override + public void writeReports(String controllerTaskId, Map taskReport) + { + if (controller != null && controller.id().equals(controllerTaskId)) { + report = taskReport; + } + } + + public Map getAllReports() + { + return report; + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java new file mode 100644 index 00000000000..61a4389e93d --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java @@ -0,0 +1,72 @@ +/* + * 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.druid.msq.test; + +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.timeline.DataSegment; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.Map; + +/** + * Data Segment pusher which populates the {@link MSQTestSegmentManager} + */ +public class MSQTestDelegateDataSegmentPusher implements DataSegmentPusher +{ + private final DataSegmentPusher delegate; + private final MSQTestSegmentManager segmentManager; + + public MSQTestDelegateDataSegmentPusher( + DataSegmentPusher dataSegmentPusher, + MSQTestSegmentManager segmentManager + ) + { + delegate = dataSegmentPusher; + this.segmentManager = segmentManager; + } + + @Override + public String getPathForHadoop(String dataSource) + { + return delegate.getPathForHadoop(dataSource); + } + + @Override + public String getPathForHadoop() + { + return delegate.getPathForHadoop(); + } + + @Override + public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException + { + final DataSegment dataSegment = delegate.push(file, segment, useUniquePath); + segmentManager.addDataSegment(dataSegment); + return dataSegment; + } + + @Override + public Map makeLoadSpec(URI finalIndexZipFilePath) + { + return delegate.makeLoadSpec(finalIndexZipFilePath); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java new file mode 100644 index 00000000000..87a6f86ee85 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.test; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Injector; +import org.apache.druid.client.indexing.NoopOverlordClient; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerImpl; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSpec; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.Map; + +public class MSQTestOverlordServiceClient extends NoopOverlordClient +{ + private final Injector injector; + private final ObjectMapper objectMapper; + private final TaskActionClient taskActionClient; + private final WorkerMemoryParameters workerMemoryParameters; + private Map inMemoryControllers = new HashMap<>(); + private Map> reports = new HashMap<>(); + private Map msqSpec = new HashMap<>(); + + public MSQTestOverlordServiceClient( + ObjectMapper objectMapper, + Injector injector, + TaskActionClient taskActionClient, + WorkerMemoryParameters workerMemoryParameters + ) + { + this.objectMapper = objectMapper; + this.injector = injector; + this.taskActionClient = taskActionClient; + this.workerMemoryParameters = workerMemoryParameters; + } + + @Override + public ListenableFuture runTask(String taskId, Object taskObject) + { + ControllerImpl controller = null; + MSQTestControllerContext msqTestControllerContext = null; + try { + msqTestControllerContext = new MSQTestControllerContext( + objectMapper, + injector, + taskActionClient, + workerMemoryParameters + ); + + MSQControllerTask cTask = objectMapper.convertValue(taskObject, MSQControllerTask.class); + msqSpec.put(cTask.getId(), cTask.getQuerySpec()); + + controller = new ControllerImpl( + cTask, + msqTestControllerContext + ); + + inMemoryControllers.put(cTask.getId(), controller); + + controller.run(); + return Futures.immediateFuture(null); + } + catch (Exception e) { + throw new ISE(e, "Unable to run"); + } + finally { + if (controller != null && msqTestControllerContext != null) { + reports.put(controller.id(), msqTestControllerContext.getAllReports()); + } + } + } + + @Override + public ListenableFuture cancelTask(String taskId) + { + inMemoryControllers.get(taskId).stopGracefully(); + return Futures.immediateFuture(null); + } + + // hooks to pull stuff out for testing + @Nullable + Map getReportForTask(String id) + { + return reports.get(id); + } + + @Nullable + MSQSpec getQuerySpecForTask(String id) + { + return msqSpec.get(id); + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java new file mode 100644 index 00000000000..6151cb37cc2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java @@ -0,0 +1,84 @@ +/* + * 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.druid.msq.test; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * Segment manager for tests to retrieve the generated segments in case of an insert query + */ +public class MSQTestSegmentManager +{ + private final ConcurrentMap dataSegments = new ConcurrentHashMap<>(); + private final ConcurrentMap segments = new ConcurrentHashMap<>(); + private final SegmentCacheManager segmentCacheManager; + private final IndexIO indexIO; + + final Object lock = new Object(); + + + public MSQTestSegmentManager(SegmentCacheManager segmentCacheManager, IndexIO indexIO) + { + this.segmentCacheManager = segmentCacheManager; + this.indexIO = indexIO; + } + + public void addDataSegment(DataSegment dataSegment) + { + synchronized (lock) { + dataSegments.put(dataSegment.getId(), dataSegment); + + try { + segmentCacheManager.getSegmentFiles(dataSegment); + } + catch (SegmentLoadingException e) { + throw new ISE(e, "Unable to load segment [%s]", dataSegment.getId()); + } + } + } + + public Collection getAllDataSegments() + { + return dataSegments.values(); + } + + public void addSegment(Segment segment) + { + segments.put(segment.getId(), segment); + } + + @Nullable + public Segment getSegment(SegmentId segmentId) + { + return segments.get(segmentId); + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java new file mode 100644 index 00000000000..5022cfe9291 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestTaskActionClient.java @@ -0,0 +1,98 @@ +/* + * 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.druid.msq.test; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.TaskAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.msq.indexing.error.InsertLockPreemptedFaultTest; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.Interval; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +public class MSQTestTaskActionClient implements TaskActionClient +{ + + private static final String VERSION = "test"; + private final ObjectMapper mapper; + private final ConcurrentHashMap segmentIdPartitionIdMap = new ConcurrentHashMap<>(); + + public MSQTestTaskActionClient(ObjectMapper mapper) + { + this.mapper = mapper; + } + + @Override + public RetType submit(TaskAction taskAction) + { + if (taskAction instanceof SegmentAllocateAction) { + SegmentAllocateAction segmentAllocateAction = (SegmentAllocateAction) taskAction; + InsertLockPreemptedFaultTest.LockPreemptedHelper.throwIfPreempted(); + Granularity granularity = segmentAllocateAction.getPreferredSegmentGranularity(); + Interval interval; + + if (granularity instanceof PeriodGranularity) { + PeriodGranularity periodGranularity = (PeriodGranularity) granularity; + interval = new Interval( + segmentAllocateAction.getTimestamp().toInstant(), + periodGranularity.getPeriod() + ); + } else { + interval = Intervals.ETERNITY; + } + + SegmentId segmentId = SegmentId.of(segmentAllocateAction.getDataSource(), interval, VERSION, 0); + AtomicInteger newPartitionId = segmentIdPartitionIdMap.computeIfAbsent(segmentId, k -> new AtomicInteger(-1)); + + return (RetType) new SegmentIdWithShardSpec( + segmentAllocateAction.getDataSource(), + interval, + VERSION, + segmentAllocateAction.getPartialShardSpec().complete(mapper, newPartitionId.addAndGet(1), 100) + ); + } else if (taskAction instanceof LockListAction) { + return (RetType) ImmutableList.of(new TimeChunkLock( + TaskLockType.EXCLUSIVE, + "group", + "ds", + Intervals.ETERNITY, + VERSION, + 0 + )); + } else if (taskAction instanceof RetrieveUsedSegmentsAction) { + return (RetType) ImmutableSet.of(); + } else { + return null; + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java new file mode 100644 index 00000000000..094c1bd16cf --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.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.druid.msq.test; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; + +import java.io.InputStream; +import java.util.Arrays; +import java.util.Map; + +public class MSQTestWorkerClient implements WorkerClient +{ + private final Map inMemoryWorkers; + + public MSQTestWorkerClient(Map inMemoryWorkers) + { + this.inMemoryWorkers = inMemoryWorkers; + } + + @Override + public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workOrder) + { + inMemoryWorkers.get(workerTaskId).postWorkOrder(workOrder); + return Futures.immediateFuture(null); + } + + @Override + public ListenableFuture postResultPartitionBoundaries( + String workerTaskId, + StageId stageId, + ClusterByPartitions partitionBoundaries + ) + { + try { + inMemoryWorkers.get(workerTaskId).postResultPartitionBoundaries( + partitionBoundaries, + stageId.getQueryId(), + stageId.getStageNumber() + ); + return Futures.immediateFuture(null); + } + catch (Exception e) { + throw new ISE(e, "unable to post result partition boundaries to workers"); + } + } + + @Override + public ListenableFuture postCleanupStage(String workerTaskId, StageId stageId) + { + inMemoryWorkers.get(workerTaskId).postCleanupStage(stageId); + return Futures.immediateFuture(null); + } + + @Override + public ListenableFuture postFinish(String taskId) + { + inMemoryWorkers.get(taskId).postFinish(); + return Futures.immediateFuture(null); + } + + @Override + public ListenableFuture getCounters(String taskId) + { + return Futures.immediateFuture(inMemoryWorkers.get(taskId).getCounters()); + } + + @Override + public ListenableFuture fetchChannelData( + final String workerTaskId, + final StageId stageId, + final int partitionNumber, + final long offset, + final ReadableByteChunksFrameChannel channel + ) + { + try (InputStream inputStream = inMemoryWorkers.get(workerTaskId).readChannel( + stageId.getQueryId(), + stageId.getStageNumber(), + partitionNumber, + offset + )) { + byte[] buffer = new byte[8 * 1024]; + int bytesRead; + while ((bytesRead = inputStream.read(buffer)) != -1) { + channel.addChunk(Arrays.copyOf(buffer, bytesRead)); + } + inputStream.close(); + + return Futures.immediateFuture(true); + } + catch (Exception e) { + throw new ISE(e, "Error reading frame file channel"); + } + + } + + @Override + public void close() + { + inMemoryWorkers.forEach((k, v) -> v.stopGracefully()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java new file mode 100644 index 00000000000..655077008db --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -0,0 +1,183 @@ +/* + * 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.druid.msq.test; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Injector; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.exec.WorkerContext; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.indexing.IndexerFrameContext; +import org.apache.druid.msq.indexing.IndexerWorkerContext; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.incremental.NoopRowIngestionMeters; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.apache.druid.server.security.AuthTestUtils; + +import java.io.File; +import java.util.Map; + +public class MSQTestWorkerContext implements WorkerContext +{ + private final Controller controller; + private final ObjectMapper mapper; + private final Injector injector; + private final Map inMemoryWorkers; + private final File file = FileUtils.createTempDir(); + private final WorkerMemoryParameters workerMemoryParameters; + + public MSQTestWorkerContext( + Map inMemoryWorkers, + Controller controller, + ObjectMapper mapper, + Injector injector, + WorkerMemoryParameters workerMemoryParameters + ) + { + this.inMemoryWorkers = inMemoryWorkers; + this.controller = controller; + this.mapper = mapper; + this.injector = injector; + this.workerMemoryParameters = workerMemoryParameters; + } + + @Override + public ObjectMapper jsonMapper() + { + return mapper; + } + + @Override + public Injector injector() + { + return injector; + } + + @Override + public void registerWorker(Worker worker, Closer closer) + { + + } + + @Override + public ControllerClient makeControllerClient(String controllerId) + { + return new MSQTestControllerClient(controller); + } + + @Override + public WorkerClient makeWorkerClient() + { + return new MSQTestWorkerClient(inMemoryWorkers); + } + + @Override + public File tempDir() + { + return file; + } + + @Override + public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) + { + IndexIO indexIO = new IndexIO( + mapper, + () -> 0 + ); + IndexMergerV9 indexMerger = new IndexMergerV9( + mapper, + indexIO, + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); + final TaskReportFileWriter reportFileWriter = new TaskReportFileWriter() + { + @Override + public void write(String taskId, Map reports) + { + + } + + @Override + public void setObjectMapper(ObjectMapper objectMapper) + { + + } + }; + + return new IndexerFrameContext( + new IndexerWorkerContext( + new TaskToolbox.Builder() + .segmentPusher(injector.getInstance(DataSegmentPusher.class)) + .segmentAnnouncer(injector.getInstance(DataSegmentAnnouncer.class)) + .jsonMapper(mapper) + .taskWorkDir(tempDir()) + .indexIO(indexIO) + .indexMergerV9(indexMerger) + .taskReportFileWriter(reportFileWriter) + .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) + .chatHandlerProvider(new NoopChatHandlerProvider()) + .rowIngestionMetersFactory(NoopRowIngestionMeters::new) + .build(), + injector, + indexIO, + null, + null + ), + indexIO, + injector.getInstance(DataSegmentProvider.class), + workerMemoryParameters + ); + } + + @Override + public int threadCount() + { + return 1; + } + + @Override + public DruidNode selfNode() + { + return new DruidNode("test", "123", true, 8080, 8081, true, false); + } + + @Override + public Bouncer processorBouncer() + { + return injector.getInstance(Bouncer.class); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java new file mode 100644 index 00000000000..6dd47313e7b --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java @@ -0,0 +1,91 @@ +/* + * 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.druid.msq.util; + +import org.apache.druid.java.util.common.Intervals; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +public class IntervalUtilsTest +{ + @Test + public void test_difference() + { + Assert.assertEquals( + intervals(), + IntervalUtils.difference(intervals(), intervals("2000/P1D")) + ); + + Assert.assertEquals( + intervals("2000/P1D"), + IntervalUtils.difference(intervals("2000/P1D"), intervals()) + ); + + Assert.assertEquals( + intervals("2000/2001"), + IntervalUtils.difference(intervals("2000/2001"), intervals("2003/2004")) + ); + + Assert.assertEquals( + intervals("2000-01-02/2001"), + IntervalUtils.difference(intervals("2000/2001"), intervals("2000/P1D")) + ); + + Assert.assertEquals( + intervals("2000/2000-02-01", "2000-02-02/2001"), + IntervalUtils.difference(intervals("2000/2001"), intervals("2000-02-01/P1D")) + ); + + Assert.assertEquals( + intervals(), + IntervalUtils.difference(intervals("2000/2001"), intervals("1999/2001")) + ); + + Assert.assertEquals( + intervals("2000-01-14/2000-02-01", "2000-02-02/2001"), + IntervalUtils.difference(intervals("2000/P1D", "2000-01-14/2001"), intervals("2000/P1D", "2000-02-01/P1D")) + ); + + Assert.assertEquals( + intervals("2000-01-01/2000-07-01", "2000-07-02/2001-01-01", "2002-01-01/2002-07-01", "2002-07-02/2003-01-01"), + IntervalUtils.difference(intervals("2000/P1Y", "2002/P1Y"), intervals("2000-07-01/P1D", "2002-07-01/P1D")) + ); + + Assert.assertEquals( + intervals(), + IntervalUtils.difference(intervals("2000-01-12/2000-01-15"), intervals("2000-01-12/2000-01-13", "2000-01-13/2000-01-16")) + ); + + Assert.assertEquals( + intervals("2000-07-14/2000-07-15"), + IntervalUtils.difference(intervals("2000/2001"), intervals("2000-01-01/2000-07-14", "2000-07-15/2001")) + ); + } + + public static List intervals(final String... intervals) + { + return Arrays.stream(intervals).map(Intervals::of).collect(Collectors.toList()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java new file mode 100644 index 00000000000..23beeebd8f5 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -0,0 +1,191 @@ +/* + * 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.druid.msq.util; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.query.QueryContext; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_DESTINATION; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MAX_NUM_TASKS; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MSQ_MODE; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ROWS_IN_MEMORY; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ROWS_PER_SEGMENT; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_SORT_ORDER; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY; +import static org.apache.druid.msq.util.MultiStageQueryContext.DEFAULT_MAX_NUM_TASKS; + +public class MultiStageQueryContextTest +{ + @Test + public void isDurableStorageEnabled_noParameterSetReturnsDefaultValue() + { + Assert.assertFalse(MultiStageQueryContext.isDurableStorageEnabled(ImmutableMap.of())); + } + + @Test + public void isDurableStorageEnabled_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, "true"); + Assert.assertTrue(MultiStageQueryContext.isDurableStorageEnabled(propertyMap)); + } + + @Test + public void isFinalizeAggregations_noParameterSetReturnsDefaultValue() + { + Assert.assertTrue(MultiStageQueryContext.isFinalizeAggregations(new QueryContext())); + } + + @Test + public void isFinalizeAggregations_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_FINALIZE_AGGREGATIONS, "false"); + Assert.assertFalse(MultiStageQueryContext.isFinalizeAggregations(new QueryContext(propertyMap))); + } + + @Test + public void getAssignmentStrategy_noParameterSetReturnsDefaultValue() + { + Assert.assertEquals(WorkerAssignmentStrategy.MAX, MultiStageQueryContext.getAssignmentStrategy(new QueryContext())); + } + + @Test + public void getAssignmentStrategy_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_TASK_ASSIGNMENT_STRATEGY, "AUTO"); + Assert.assertEquals( + WorkerAssignmentStrategy.AUTO, + MultiStageQueryContext.getAssignmentStrategy(new QueryContext(propertyMap)) + ); + } + + @Test + public void getMaxNumTasks_noParameterSetReturnsDefaultValue() + { + Assert.assertEquals(DEFAULT_MAX_NUM_TASKS, MultiStageQueryContext.getMaxNumTasks(new QueryContext())); + } + + @Test + public void getMaxNumTasks_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_MAX_NUM_TASKS, 101); + Assert.assertEquals(101, MultiStageQueryContext.getMaxNumTasks(new QueryContext(propertyMap))); + } + + @Test + public void getMaxNumTasks_legacyParameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_MAX_NUM_TASKS, 101); + Assert.assertEquals(101, MultiStageQueryContext.getMaxNumTasks(new QueryContext(propertyMap))); + } + + @Test + public void getDestination_noParameterSetReturnsDefaultValue() + { + Assert.assertNull(MultiStageQueryContext.getDestination(new QueryContext())); + } + + @Test + public void getDestination_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_DESTINATION, "dataSource"); + Assert.assertEquals("dataSource", MultiStageQueryContext.getDestination(new QueryContext(propertyMap))); + } + + @Test + public void getRowsPerSegment_noParameterSetReturnsDefaultValue() + { + Assert.assertEquals(1000, MultiStageQueryContext.getRowsPerSegment(new QueryContext(), 1000)); + } + + @Test + public void getRowsPerSegment_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_ROWS_PER_SEGMENT, 10); + Assert.assertEquals(10, MultiStageQueryContext.getRowsPerSegment(new QueryContext(propertyMap), 1000)); + } + + @Test + public void getRowsInMemory_noParameterSetReturnsDefaultValue() + { + Assert.assertEquals(1000, MultiStageQueryContext.getRowsInMemory(new QueryContext(), 1000)); + } + + @Test + public void getRowsInMemory_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_ROWS_IN_MEMORY, 10); + Assert.assertEquals(10, MultiStageQueryContext.getRowsInMemory(new QueryContext(propertyMap), 1000)); + } + + @Test + public void testDecodeSortOrder() + { + Assert.assertEquals(ImmutableList.of("a", "b", "c,d"), decodeSortOrder("a, b,\"c,d\"")); + Assert.assertEquals(ImmutableList.of("a", "b", "c,d"), decodeSortOrder(" a, b,\"c,d\"")); + Assert.assertEquals(ImmutableList.of("a", "b", "c,d"), decodeSortOrder("[\"a\", \"b\", \"c,d\"]")); + Assert.assertEquals(ImmutableList.of("a", "b", "c,d"), decodeSortOrder(" [\"a\", \"b\", \"c,d\"] ")); + Assert.assertEquals(ImmutableList.of(), decodeSortOrder("[]")); + Assert.assertEquals(ImmutableList.of(), decodeSortOrder("")); + Assert.assertEquals(ImmutableList.of(), decodeSortOrder(null)); + + Assert.assertThrows(IllegalArgumentException.class, () -> decodeSortOrder("[[")); + } + + @Test + public void getSortOrderNoParameterSetReturnsDefaultValue() + { + Assert.assertNull(MultiStageQueryContext.getSortOrder(new QueryContext())); + } + + @Test + public void getSortOrderParameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_SORT_ORDER, "a, b,\"c,d\""); + Assert.assertEquals("a, b,\"c,d\"", MultiStageQueryContext.getSortOrder(new QueryContext(propertyMap))); + } + + @Test + public void getMSQModeNoParameterSetReturnsDefaultValue() + { + Assert.assertEquals("strict", MultiStageQueryContext.getMSQMode(new QueryContext())); + } + + @Test + public void getMSQModeParameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_MSQ_MODE, "nonStrict"); + Assert.assertEquals("nonStrict", MultiStageQueryContext.getMSQMode(new QueryContext(propertyMap))); + } + + private static List decodeSortOrder(@Nullable final String input) + { + return MultiStageQueryContext.decodeSortOrder(input); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/PassthroughAggregatorFactoryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/PassthroughAggregatorFactoryTest.java new file mode 100644 index 00000000000..288d14ac994 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/PassthroughAggregatorFactoryTest.java @@ -0,0 +1,77 @@ +/* + * 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.druid.msq.util; + +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; +import org.junit.Assert; +import org.junit.Test; + +public class PassthroughAggregatorFactoryTest +{ + @Test + public void testRequiredFields() + { + Assert.assertEquals( + ImmutableList.of("x"), + new PassthroughAggregatorFactory("x", "y").requiredFields() + ); + } + + @Test + public void testGetCombiningFactory() + { + Assert.assertEquals( + new PassthroughAggregatorFactory("x", "y"), + new PassthroughAggregatorFactory("x", "y").getCombiningFactory() + ); + } + + @Test + public void testGetMergingFactoryOk() throws AggregatorFactoryNotMergeableException + { + final AggregatorFactory mergingFactory = + new PassthroughAggregatorFactory("x", "y").getMergingFactory(new PassthroughAggregatorFactory("x", "y")); + + Assert.assertEquals(new PassthroughAggregatorFactory("x", "y"), mergingFactory); + } + + @Test + public void testGetMergingFactoryNotOk() + { + Assert.assertThrows( + AggregatorFactoryNotMergeableException.class, + () -> new PassthroughAggregatorFactory("x", "y").getMergingFactory(new PassthroughAggregatorFactory("x", "z")) + ); + + Assert.assertThrows( + AggregatorFactoryNotMergeableException.class, + () -> new PassthroughAggregatorFactory("x", "y").getMergingFactory(new PassthroughAggregatorFactory("z", "y")) + ); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(PassthroughAggregatorFactory.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/resources/unparseable.gz b/extensions-core/multi-stage-query/src/test/resources/unparseable.gz new file mode 100644 index 0000000000000000000000000000000000000000..4d1ba375815de1545aec329a625a5287f47d3545 GIT binary patch literal 538 zcmb2|=HQqo+Lpw?T$)#qSX7*vn3R*s@OF}a|1AT7qwjxfPB@?NXo7vq?P+0q*KC{m zA}(sx2^FO`H3 z_^8_z_SpHc>!-OsocS%^QjhM<=&^Nt8=g4#;-1+dZn9PSZ0(@#1jDPv8~4q1j# z+l^PM>LboRKWuOz{jl}BJs;Nw{P9Q>EM7V5XJp)k_;!KW*LR-_IiIQ?>+!F1_hn}b zzmxZFG~T`-?U#^hnaaAjaMqextensions-core/kafka-extraction-namespace extensions-core/kafka-indexing-service extensions-core/kinesis-indexing-service + extensions-core/multi-stage-query extensions-core/mysql-metadata-storage extensions-core/orc-extensions extensions-core/parquet-extensions