From 5b2f93d808abe85910e09d8bcac4831a70961056 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 5 Jul 2023 19:39:25 +0530 Subject: [PATCH 01/15] Adding Ability for MSQ to write select results to durable storage. --- .../msq/counters/CounterSnapshotsTree.java | 7 + .../SegmentGeneratorMetricsWrapper.java | 3 +- .../org/apache/druid/msq/exec/Controller.java | 3 +- .../apache/druid/msq/exec/ControllerImpl.java | 162 ++++--- .../org/apache/druid/msq/exec/MSQTasks.java | 7 +- .../org/apache/druid/msq/exec/WorkerImpl.java | 97 ++-- .../msq/guice/MSQDurableStorageModule.java | 8 +- .../druid/msq/guice/MSQIndexingModule.java | 11 +- .../indexing/IndexerControllerContext.java | 3 + .../msq/indexing/IndexerWorkerContext.java | 3 + .../druid/msq/indexing/MSQControllerTask.java | 7 + .../druid/msq/indexing/MSQDestination.java | 6 +- .../apache/druid/msq/indexing/MSQSpec.java | 1 + .../{ => cleaner}/DurableStorageCleaner.java | 2 +- .../DurableStorageCleanerConfig.java | 2 +- .../{ => client}/ControllerChatHandler.java | 4 +- .../{ => client}/IndexerControllerClient.java | 3 +- .../{ => client}/IndexerWorkerClient.java | 34 +- .../IndexerWorkerManagerClient.java | 3 +- .../{ => client}/WorkerChatHandler.java | 3 +- .../DataSourceMSQDestination.java | 5 +- .../DurableStorageDestination.java | 49 ++ .../MSQSelectDestination.java | 2 +- .../TaskReportMSQDestination.java | 6 +- .../KeyStatisticsCollectionProcessor.java | 2 +- .../SegmentGeneratorFrameProcessor.java | 2 +- ...SegmentGeneratorFrameProcessorFactory.java | 3 +- .../msq/indexing/report/MSQResultsReport.java | 2 +- .../druid/msq/kernel/StageDefinition.java | 2 +- .../QueryResultFrameProcessorFactory.java | 93 ++++ .../results/QueryResultsFrameProcessor.java | 85 ++++ .../DurableStorageInputChannelFactory.java | 42 +- ...torageQueryResultsInputChannelFactory.java | 60 +++ ...urableStorageStageInputChannelFactory.java | 61 +++ .../WorkerInputChannelFactory.java | 2 +- .../DurableStorageOutputChannelFactory.java | 155 +++++++ ...bleStorageResultsOutputChannelFactory.java | 102 ++++ ...eStorageTaskOutputChannelFactoryImpl.java} | 134 ++---- .../druid/msq/sql/MSQTaskQueryMaker.java | 13 +- .../msq/sql/entity/ResultSetInformation.java | 6 +- .../sql/resources/SqlStatementResource.java | 438 +++++++++++++----- .../msq/util/MultiStageQueryContext.java | 14 +- .../msq/util/SqlStatementResourceHelper.java | 163 ++++++- .../msq/exec/MSQParseExceptionsTest.java | 2 +- .../apache/druid/msq/exec/MSQSelectTest.java | 189 +++++--- .../DataSourceMSQDestinationTest.java | 1 + .../indexing/DurableStorageCleanerTest.java | 2 + .../msq/indexing/MSQControllerTaskTest.java | 1 + .../msq/indexing/WorkerChatHandlerTest.java | 1 + ...urableStorageOutputChannelFactoryTest.java | 3 +- .../sql/SqlMsqStatementResourcePostTest.java | 24 +- .../msq/sql/SqlStatementResourceTest.java | 95 ++-- .../sql/entity/ResultSetInformationTest.java | 29 +- .../sql/entity/SqlStatementResultTest.java | 4 +- .../apache/druid/msq/test/MSQTestBase.java | 46 +- .../msq/util/MultiStageQueryContextTest.java | 2 +- .../druid/frame/util/DurableStorageUtils.java | 73 ++- .../druid/storage/NilStorageConnector.java | 90 ++++ 58 files changed, 1843 insertions(+), 529 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => cleaner}/DurableStorageCleaner.java (99%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => cleaner}/DurableStorageCleanerConfig.java (97%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => client}/ControllerChatHandler.java (97%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => client}/IndexerControllerClient.java (98%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => client}/IndexerWorkerClient.java (94%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => client}/IndexerWorkerManagerClient.java (96%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => client}/WorkerChatHandler.java (99%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => destination}/DataSourceMSQDestination.java (97%) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageDestination.java rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => destination}/MSQSelectDestination.java (96%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => destination}/TaskReportMSQDestination.java (89%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => processor}/KeyStatisticsCollectionProcessor.java (99%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => processor}/SegmentGeneratorFrameProcessor.java (99%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => processor}/SegmentGeneratorFrameProcessorFactory.java (99%) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.java rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/{ => input}/DurableStorageInputChannelFactory.java (84%) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/{ => input}/WorkerInputChannelFactory.java (98%) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageResultsOutputChannelFactory.java rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/{DurableStorageOutputChannelFactory.java => output/DurableStorageTaskOutputChannelFactoryImpl.java} (60%) create mode 100644 processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java 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 index 953f8b718dee..8936e104bd69 100644 --- 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 @@ -91,6 +91,13 @@ public Map> copyMap() return retVal; } + public Map snapshotForStage(int stageNumber) + { + synchronized (snapshotsMap) { + return snapshotsMap.getOrDefault(stageNumber, null); + } + } + private void putAll(final Map> otherMap) { synchronized (snapshotsMap) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java index ec187b9827db..8f3e2d79aaea 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java @@ -19,12 +19,13 @@ package org.apache.druid.msq.counters; +import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessor; import org.apache.druid.segment.realtime.FireDepartmentMetrics; /** * Wrapper around {@link FireDepartmentMetrics} which updates the progress counters while updating its metrics. This * is necessary as the {@link org.apache.druid.segment.realtime.appenderator.BatchAppenderator} used by the - * {@link org.apache.druid.msq.indexing.SegmentGeneratorFrameProcessor} is not part of the MSQ extension, and hence, + * {@link SegmentGeneratorFrameProcessor} is not part of the MSQ extension, and hence, * cannot update the counters used in MSQ reports as it persists and pushes segments to deep storage. */ public class SegmentGeneratorMetricsWrapper extends FireDepartmentMetrics 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 index 3552459f2359..b7a93de5ed02 100644 --- 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 @@ -26,6 +26,7 @@ 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.client.ControllerChatHandler; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; @@ -83,7 +84,7 @@ public String getId() /** * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate partiton boundaries. - * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. + * This is intended to be called by the {@link ControllerChatHandler}. */ void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject); 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 index 1a8ef12cb2fd..427d1ab6b9c7 100644 --- 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 @@ -54,7 +54,6 @@ import org.apache.druid.frame.key.RowKey; import org.apache.druid.frame.key.RowKeyReader; import org.apache.druid.frame.processor.FrameProcessorExecutor; -import org.apache.druid.frame.processor.FrameProcessors; import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -86,17 +85,18 @@ 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.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.MSQSelectDestination; 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.WorkerCount; +import org.apache.druid.msq.indexing.client.ControllerChatHandler; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.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; @@ -114,6 +114,7 @@ import org.apache.druid.msq.indexing.error.TooManyWarningsFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; +import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessorFactory; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQStagesReport; import org.apache.druid.msq.indexing.report.MSQStatusReport; @@ -154,24 +155,23 @@ 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.results.QueryResultFrameProcessorFactory; import org.apache.druid.msq.querykit.scan.ScanQueryKit; -import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory; -import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; +import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; +import org.apache.druid.msq.shuffle.input.WorkerInputChannelFactory; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; 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.msq.util.SqlStatementResourceHelper; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; 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; @@ -186,7 +186,6 @@ import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; -import org.apache.druid.sql.calcite.run.SqlResults; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; @@ -206,7 +205,6 @@ import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -600,13 +598,24 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) ImmutableMap.Builder taskContextOverridesBuilder = ImmutableMap.builder(); taskContextOverridesBuilder - .put( - MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, - isDurableStorageEnabled - ).put( - MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, - maxParseExceptions - ); + .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, isDurableStorageEnabled) + .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, maxParseExceptions); + + if (!MSQControllerTask.isIngestion(task.getQuerySpec())) { + if (MSQControllerTask.writeResultsToDurableStorage(task.getQuerySpec())) { + taskContextOverridesBuilder.put( + MultiStageQueryContext.CTX_SELECT_DESTINATION, + MSQSelectDestination.DURABLE_STORAGE.name() + ); + } else { + // we need not pass the value 'TaskReport' to the worker since the worker impl does not do anything in such a case. + // but we are passing it anyway for completeness + taskContextOverridesBuilder.put( + MultiStageQueryContext.CTX_SELECT_DESTINATION, + MSQSelectDestination.TASK_REPORT.name() + ); + } + } this.workerTaskLauncher = new MSQWorkerTaskLauncher( id(), task.getDataSource(), @@ -683,7 +692,7 @@ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault /** * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key * statistics information has been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate - * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. + * partiton boundaries. This is intended to be called by the {@link ControllerChatHandler}. */ @Override public void updatePartialKeyStatisticsInformation( @@ -1391,11 +1400,13 @@ private Yielder getFinalResultsYielder( final InputChannelFactory inputChannelFactory; - if (isDurableStorageEnabled) { + if (isDurableStorageEnabled || MSQControllerTask.writeResultsToDurableStorage(task.getQuerySpec())) { inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation( id(), - MSQTasks.makeStorageConnector(context.injector()), - closer + MSQTasks.makeStorageConnector( + context.injector()), + closer, + MSQControllerTask.writeResultsToDurableStorage(task.getQuerySpec()) ); } else { inputChannelFactory = new WorkerInputChannelFactory(netClient, () -> taskIds); @@ -1431,57 +1442,13 @@ private Yielder getFinalResultsYielder( } ).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 sqlTypeNames = task.getSqlTypeNames(); - Iterable retVal = () -> new Iterator() - { - @Override - public boolean hasNext() - { - return !cursor.isDone(); - } - - @Override - public Object[] next() - { - final Object[] row = new Object[columnMappings.size()]; - for (int i = 0; i < row.length; i++) { - final Object value = selectors.get(i).getObject(); - if (sqlTypeNames == null || task.getSqlResultsContext() == null) { - // SQL type unknown, or no SQL results context: pass-through as is. - row[i] = value; - } else { - row[i] = SqlResults.coerce( - context.jsonMapper(), - task.getSqlResultsContext(), - value, - sqlTypeNames.get(i), - columnMappings.getOutputColumnName(i) - ); - } - } - cursor.advance(); - return row; - } - }; - return Sequences.simple(retVal); - } + frame -> + SqlStatementResourceHelper.getResultSequence( + task, + queryDef.getFinalStageDefinition(), + frame, + context.jsonMapper() + ) ) .withBaggage(resultReaderExec::shutdownNow) ); @@ -1571,6 +1538,9 @@ private static QueryDefinition makeQueryDefinition( } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); queryToPlan = querySpec.getQuery(); + } else if (querySpec.getDestination() instanceof DurableStorageDestination) { + shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); + queryToPlan = querySpec.getQuery(); } else { throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); } @@ -1645,6 +1615,37 @@ private static QueryDefinition makeQueryDefinition( return builder.build(); } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { return queryDef; + } else if (querySpec.getDestination() instanceof DurableStorageDestination) { + StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition(); + if (!finalShuffleStageDef.doesSortDuringShuffle()) { + return queryDef; + } + // attaching new query results stage + final QueryDefinitionBuilder builder = QueryDefinition.builder(); + for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { + builder.add(StageDefinition.builder(stageDef)); + } + + + builder.add( + StageDefinition.builder(queryDef.getNextStageNumber()) + .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) + .maxWorkerCount(tuningConfig.getMaxNumWorkers()) + .signature(finalShuffleStageDef.getSignature()) + .shuffleSpec(finalShuffleStageDef.getClusterBy().isEmpty() + ? null + : ShuffleSpecFactories.singlePartition() + .build(finalShuffleStageDef.getClusterBy(), false)) + .shuffleCheckHasMultipleValues(finalShuffleStageDef.getShuffleCheckHasMultipleValues()) + .processorFactory( + new QueryResultFrameProcessorFactory( + ) + ) + ); + + return builder.build(); + + } else { throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); } @@ -1754,7 +1755,8 @@ private static boolean isRollupQuery(Query query) private static boolean isInlineResults(final MSQSpec querySpec) { - return querySpec.getDestination() instanceof TaskReportMSQDestination; + return querySpec.getDestination() instanceof TaskReportMSQDestination + || querySpec.getDestination() instanceof DurableStorageDestination; } private static boolean isTimeBucketedIngestion(final MSQSpec querySpec) @@ -2051,7 +2053,12 @@ private static MSQResultsReport makeResultsTaskReport( ); } - return MSQResultsReport.createReportAndLimitRowsIfNeeded(mappedSignature.build(), sqlTypeNames, resultsYielder, selectDestination); + return MSQResultsReport.createReportAndLimitRowsIfNeeded( + mappedSignature.build(), + sqlTypeNames, + resultsYielder, + selectDestination + ); } private static MSQStatusReport makeStatusReport( @@ -2597,7 +2604,8 @@ private void cleanUpEffectivelyFinishedStages() queryKernel, (netClient, taskId, workerNumber) -> netClient.postCleanupStage(taskId, stageId), queryKernel.getWorkerInputsForStage(stageId).workers(), - (ignore1) -> {}, + (ignore1) -> { + }, false ); queryKernel.finishStage(stageId, true); 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 index 44fe0dbce8d1..24a3fad8dbf1 100644 --- 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 @@ -45,6 +45,7 @@ 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.NilStorageConnector; import org.apache.druid.storage.StorageConnector; import javax.annotation.Nullable; @@ -155,7 +156,11 @@ static String getHostFromSelfNode(@Nullable final DruidNode selfNode) static StorageConnector makeStorageConnector(final Injector injector) { try { - return injector.getInstance(Key.get(StorageConnector.class, MultiStageQuery.class)); + StorageConnector storageConnector = injector.getInstance(Key.get(StorageConnector.class, MultiStageQuery.class)); + if (storageConnector instanceof NilStorageConnector) { + throw new Exception("Storage connector not configured."); + } + return storageConnector; } catch (Exception e) { throw new MSQException(new DurableStorageConfigurationFault(e.toString())); 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 index cb5881b3ba40..ed48f3d3c6c4 100644 --- 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 @@ -80,8 +80,8 @@ 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.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; import org.apache.druid.msq.indexing.error.MSQErrorReport; @@ -91,6 +91,7 @@ import org.apache.druid.msq.indexing.error.MSQWarningReportPublisher; import org.apache.druid.msq.indexing.error.MSQWarningReportSimplePublisher; import org.apache.druid.msq.indexing.error.MSQWarnings; +import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.InputSlices; @@ -119,9 +120,9 @@ 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.shuffle.DurableStorageInputChannelFactory; -import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory; -import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; +import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; +import org.apache.druid.msq.shuffle.input.WorkerInputChannelFactory; +import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; @@ -181,6 +182,11 @@ public class WorkerImpl implements Worker private final ByteTracker intermediateSuperSorterLocalStorageTracker; private final boolean durableStageStorageEnabled; private final WorkerStorageParameters workerStorageParameters; + /** + * Only set for select jobs. + */ + @Nullable + private final MSQSelectDestination selectDestination; /** * Set once in {@link #runTask} and never reassigned. @@ -205,7 +211,8 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context) context, WorkerStorageParameters.createProductionInstance( context.injector(), - MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(task.getContext())) // If Durable Storage is enabled, then super sorter intermediate storage can be enabled. + MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(task.getContext())) + // If Durable Storage is enabled, then super sorter intermediate storage can be enabled. ) ); } @@ -217,12 +224,14 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context, WorkerStorageParame this.context = context; this.selfDruidNode = context.selfNode(); this.processorBouncer = context.processorBouncer(); - this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled( - QueryContext.of(task.getContext()) - ); + QueryContext queryContext = QueryContext.of(task.getContext()); + this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(queryContext); + this.selectDestination = MultiStageQueryContext.getSelectDestinationOrNull(queryContext); this.workerStorageParameters = workerStorageParameters; - long maxBytes = workerStorageParameters.isIntermediateStorageLimitConfigured() ? workerStorageParameters.getIntermediateSuperSorterStorageMaxLocalBytes() : Long.MAX_VALUE; + long maxBytes = workerStorageParameters.isIntermediateStorageLimitConfigured() + ? workerStorageParameters.getIntermediateSuperSorterStorageMaxLocalBytes() + : Long.MAX_VALUE; this.intermediateSuperSorterLocalStorageTracker = new ByteTracker(maxBytes); } @@ -704,20 +713,26 @@ private InputChannelFactory makeBaseInputChannelFactory(final Closer closer) return DurableStorageInputChannelFactory.createStandardImplementation( task.getControllerTaskId(), MSQTasks.makeStorageConnector(context.injector()), - closer + closer, + false ); } else { return new WorkerOrLocalInputChannelFactory(workerTaskList); } } - private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext frameContext, final int stageNumber) + private OutputChannelFactory makeStageOutputChannelFactory( + final FrameContext frameContext, + final int stageNumber, + boolean isFinalStage + ) { // 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) { + if (durableStageStorageEnabled || (isFinalStage + && MSQSelectDestination.DURABLE_STORAGE.equals(selectDestination))) { return DurableStorageOutputChannelFactory.createStandardImplementation( task.getControllerTaskId(), task().getWorkerNumber(), @@ -725,7 +740,8 @@ private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext fr task().getId(), frameSize, MSQTasks.makeStorageConnector(context.injector()), - context.tempDir() + context.tempDir(), + (isFinalStage && MSQSelectDestination.DURABLE_STORAGE.equals(selectDestination)) ); } else { final File fileChannelDirectory = @@ -758,7 +774,8 @@ private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( task().getId(), frameSize, MSQTasks.makeStorageConnector(context.injector()), - tmpDir + tmpDir, + false ) ), frameSize @@ -1014,9 +1031,13 @@ private void start() throws IOException final WorkOrder workOrder = kernel.getWorkOrder(); final StageDefinition stageDef = workOrder.getStageDefinition(); + final boolean isFinalStage = stageDef.getStageNumber() == workOrder.getQueryDefinition() + .getFinalStageDefinition() + .getStageNumber(); + makeInputSliceReader(); - makeWorkOutputChannelFactory(); - makeShuffleOutputChannelFactory(); + makeWorkOutputChannelFactory(isFinalStage); + makeShuffleOutputChannelFactory(isFinalStage); makeAndRunWorkProcessors(); if (stageDef.doesShuffle()) { @@ -1027,7 +1048,7 @@ private void start() throws IOException Futures.immediateFuture(workResultAndOutputChannels.getOutputChannels().readOnly()); } - setUpCompletionCallbacks(); + setUpCompletionCallbacks(isFinalStage); } /** @@ -1072,7 +1093,7 @@ private void makeInputSliceReader() ); } - private void makeWorkOutputChannelFactory() + private void makeWorkOutputChannelFactory(boolean isFinalStage) { if (workOutputChannelFactory != null) { throw new ISE("processorOutputChannelFactory already created"); @@ -1096,7 +1117,7 @@ private void makeWorkOutputChannelFactory() } else { // Writing stage output. baseOutputChannelFactory = - makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber()); + makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber(), isFinalStage); } workOutputChannelFactory = new CountingOutputChannelFactory( @@ -1105,11 +1126,11 @@ private void makeWorkOutputChannelFactory() ); } - private void makeShuffleOutputChannelFactory() + private void makeShuffleOutputChannelFactory(boolean isFinalStage) { shuffleOutputChannelFactory = new CountingOutputChannelFactory( - makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber()), + makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber(), isFinalStage), counterTracker.channel(CounterNames.shuffleChannel()) ); } @@ -1237,7 +1258,7 @@ private ListenableFuture makeGlobalSortPartitionBoundariesF } } - private void setUpCompletionCallbacks() + private void setUpCompletionCallbacks(boolean isFinalStage) { final StageDefinition stageDef = kernel.getStageDefinition(); @@ -1273,7 +1294,7 @@ public void onSuccess(final List workerResultAndOutputChannelsResolved) // Once the outputs channels have been resolved and are ready for reading, write success file, if // using durable storage. - writeDurableStorageSuccessFileIfNeeded(stageDef.getStageNumber()); + writeDurableStorageSuccessFileIfNeeded(stageDef.getStageNumber(), isFinalStage); kernelManipulationQueue.add(holder -> holder.getStageKernelMap() .get(stageDef.getId()) @@ -1295,22 +1316,24 @@ public void onFailure(final Throwable t) /** * Write {@link DurableStorageUtils#SUCCESS_MARKER_FILENAME} for a particular stage, if durable storage is enabled. */ - private void writeDurableStorageSuccessFileIfNeeded(final int stageNumber) + private void writeDurableStorageSuccessFileIfNeeded(final int stageNumber, boolean isFinalStage) { - if (!durableStageStorageEnabled) { + final DurableStorageOutputChannelFactory durableStorageOutputChannelFactory; + if (durableStageStorageEnabled || (isFinalStage + && MSQSelectDestination.DURABLE_STORAGE.equals(selectDestination))) { + durableStorageOutputChannelFactory = DurableStorageOutputChannelFactory.createStandardImplementation( + task.getControllerTaskId(), + task().getWorkerNumber(), + stageNumber, + task().getId(), + frameContext.memoryParameters().getStandardFrameSize(), + MSQTasks.makeStorageConnector(context.injector()), + context.tempDir(), + (isFinalStage && MSQSelectDestination.DURABLE_STORAGE.equals(selectDestination)) + ); + } else { return; } - - DurableStorageOutputChannelFactory durableStorageOutputChannelFactory = - DurableStorageOutputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - task().getWorkerNumber(), - stageNumber, - task().getId(), - frameContext.memoryParameters().getStandardFrameSize(), - MSQTasks.makeStorageConnector(context.injector()), - context.tempDir() - ); try { durableStorageOutputChannelFactory.createSuccessFile(task.getId()); } @@ -1319,7 +1342,7 @@ private void writeDurableStorageSuccessFileIfNeeded(final int stageNumber) e, "Unable to create the success file [%s] at the location [%s]", DurableStorageUtils.SUCCESS_MARKER_FILENAME, - DurableStorageUtils.getSuccessFilePath( + DurableStorageUtils.getWorkerOutputSuccessFilePath( task.getControllerTaskId(), stageNumber, task().getWorkerNumber() 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 index 81c6119b8fa0..6e0266fe892a 100644 --- 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 @@ -31,8 +31,9 @@ import org.apache.druid.guice.annotations.Self; import org.apache.druid.indexing.overlord.duty.OverlordDuty; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.indexing.DurableStorageCleaner; -import org.apache.druid.msq.indexing.DurableStorageCleanerConfig; +import org.apache.druid.msq.indexing.cleaner.DurableStorageCleaner; +import org.apache.druid.msq.indexing.cleaner.DurableStorageCleanerConfig; +import org.apache.druid.storage.NilStorageConnector; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorProvider; @@ -98,6 +99,9 @@ public void configure(Binder binder) .addBinding() .to(DurableStorageCleaner.class); } + } else { + // bind with nil implementation so that configs are not required during service startups. + binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class)).toInstance(NilStorageConnector.getInstance()); } } 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 index 59300316d587..4af832705c3f 100644 --- 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 @@ -36,7 +36,6 @@ 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; @@ -69,6 +68,7 @@ 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.processor.SegmentGeneratorFrameProcessorFactory; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.input.NilInputSlice; import org.apache.druid.msq.input.NilInputSource; @@ -88,6 +88,7 @@ import org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessorFactory; import org.apache.druid.msq.querykit.groupby.GroupByPostShuffleFrameProcessorFactory; import org.apache.druid.msq.querykit.groupby.GroupByPreShuffleFrameProcessorFactory; +import org.apache.druid.msq.querykit.results.QueryResultFrameProcessorFactory; import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessorFactory; import org.apache.druid.msq.util.PassthroughAggregatorFactory; import org.apache.druid.query.DruidProcessingConfig; @@ -155,14 +156,8 @@ public List getJacksonModules() GroupByPostShuffleFrameProcessorFactory.class, OffsetLimitFrameProcessorFactory.class, NilExtraInfoHolder.class, - - // FrameChannelWorkerFactory and FrameChannelWorkerFactoryExtraInfoHolder classes - ScanQueryFrameProcessorFactory.class, - GroupByPreShuffleFrameProcessorFactory.class, - GroupByPostShuffleFrameProcessorFactory.class, SortMergeJoinFrameProcessorFactory.class, - OffsetLimitFrameProcessorFactory.class, - NilExtraInfoHolder.class, + QueryResultFrameProcessorFactory.class, // DataSource classes (note: ExternalDataSource is in MSQSqlModule) InputNumberDataSource.class, 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 index cc767626ecb4..5d17b005b941 100644 --- 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 @@ -32,6 +32,9 @@ 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.msq.indexing.client.ControllerChatHandler; +import org.apache.druid.msq.indexing.client.IndexerWorkerClient; +import org.apache.druid.msq.indexing.client.IndexerWorkerManagerClient; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.realtime.firehose.ChatHandler; 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 index fe50bc19acee..10855094bde2 100644 --- 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 @@ -38,6 +38,9 @@ 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.client.IndexerControllerClient; +import org.apache.druid.msq.indexing.client.IndexerWorkerClient; +import org.apache.druid.msq.indexing.client.WorkerChatHandler; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.rpc.CoordinatorServiceClient; 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 index 30047f227ffc..0b6fdffcc37f 100644 --- 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 @@ -47,6 +47,8 @@ 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.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageDestination; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; @@ -263,4 +265,9 @@ public static boolean isIngestion(final MSQSpec querySpec) { return querySpec.getDestination() instanceof DataSourceMSQDestination; } + + public static boolean writeResultsToDurableStorage(final MSQSpec querySpec) + { + return querySpec.getDestination() instanceof DurableStorageDestination; + } } 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 index d84a1635edcc..ddf703e50de4 100644 --- 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 @@ -21,11 +21,15 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; @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) + @JsonSubTypes.Type(name = TaskReportMSQDestination.TYPE, value = TaskReportMSQDestination.class), + @JsonSubTypes.Type(name = DurableStorageDestination.TYPE, value = DurableStorageDestination.class) }) public interface MSQDestination { 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 index 22dba052fa15..caa182b8e72b 100644 --- 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 @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.query.Query; import org.apache.druid.sql.calcite.planner.ColumnMappings; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/cleaner/DurableStorageCleaner.java similarity index 99% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/cleaner/DurableStorageCleaner.java index 195b1e26f893..630499bdd876 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/cleaner/DurableStorageCleaner.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.cleaner; import com.fasterxml.jackson.annotation.JacksonInject; import com.google.common.base.Optional; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleanerConfig.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/cleaner/DurableStorageCleanerConfig.java similarity index 97% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleanerConfig.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/cleaner/DurableStorageCleanerConfig.java index d6637d0ffbd7..d45950f37ed3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleanerConfig.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/cleaner/DurableStorageCleanerConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.cleaner; import com.fasterxml.jackson.annotation.JsonProperty; 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/client/ControllerChatHandler.java similarity index 97% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java index 5b36117d3f40..22d3b31cf10f 100644 --- 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/client/ControllerChatHandler.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.client; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; @@ -25,6 +25,8 @@ 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.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQTaskList; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; 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/client/IndexerControllerClient.java similarity index 98% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java index 00302ebfc578..493cbeb62424 100644 --- 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/client/IndexerControllerClient.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.client; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.common.guava.FutureUtils; @@ -27,6 +27,7 @@ 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.MSQTaskList; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; 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/client/IndexerWorkerClient.java similarity index 94% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerWorkerClient.java index 980c7f97bee5..39a9c40e08d4 100644 --- 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/client/IndexerWorkerClient.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.client; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -111,16 +111,20 @@ public ListenableFuture fetchClusterByStatisticsSna int stageNumber ) { - String path = StringUtils.format("/keyStatistics/%s/%d", - StringUtils.urlEncode(queryId), - stageNumber); + String path = StringUtils.format( + "/keyStatistics/%s/%d", + StringUtils.urlEncode(queryId), + stageNumber + ); return FutureUtils.transform( getClient(workerTaskId).asyncRequest( new RequestBuilder(HttpMethod.POST, path), new BytesFullResponseHandler() ), - holder -> deserialize(holder, new TypeReference() {}) + holder -> deserialize(holder, new TypeReference() + { + }) ); } @@ -132,17 +136,21 @@ public ListenableFuture fetchClusterByStatisticsSna long timeChunk ) { - String path = StringUtils.format("/keyStatisticsForTimeChunk/%s/%d/%d", - StringUtils.urlEncode(queryId), - stageNumber, - timeChunk); + String path = StringUtils.format( + "/keyStatisticsForTimeChunk/%s/%d/%d", + StringUtils.urlEncode(queryId), + stageNumber, + timeChunk + ); return FutureUtils.transform( getClient(workerTaskId).asyncRequest( new RequestBuilder(HttpMethod.POST, path), new BytesFullResponseHandler() ), - holder -> deserialize(holder, new TypeReference() {}) + holder -> deserialize(holder, new TypeReference() + { + }) ); } @@ -204,7 +212,9 @@ public ListenableFuture getCounters(String workerTaskId) new RequestBuilder(HttpMethod.GET, "/counters"), new BytesFullResponseHandler() ), - holder -> deserialize(holder, new TypeReference() {}) + holder -> deserialize(holder, new TypeReference() + { + }) ); } @@ -299,7 +309,7 @@ private ServiceClient getClient(final String workerTaskId) /** * 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) 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/client/IndexerWorkerManagerClient.java similarity index 96% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerWorkerManagerClient.java index 707ade86a52b..2940fc4f9c33 100644 --- 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/client/IndexerWorkerManagerClient.java @@ -17,13 +17,14 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.client; 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.msq.indexing.MSQWorkerTask; import org.apache.druid.rpc.indexing.OverlordClient; import java.util.Map; 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/client/WorkerChatHandler.java similarity index 99% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java index 3eae3b05ccf4..0d26a21f3dbb 100644 --- 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/client/WorkerChatHandler.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.client; import com.google.common.collect.ImmutableMap; import it.unimi.dsi.fastutil.bytes.ByteArrays; @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; 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/destination/DataSourceMSQDestination.java similarity index 97% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java index 6b1a683ff0ec..04c203941f49 100644 --- 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/destination/DataSourceMSQDestination.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; @@ -26,6 +26,7 @@ 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.apache.druid.msq.indexing.MSQDestination; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -35,7 +36,7 @@ public class DataSourceMSQDestination implements MSQDestination { - static final String TYPE = "dataSource"; + public static final String TYPE = "dataSource"; private final String dataSource; private final Granularity segmentGranularity; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageDestination.java new file mode 100644 index 000000000000..4cf1bbe24997 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageDestination.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.destination; + +import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.msq.indexing.MSQDestination; + +public class DurableStorageDestination implements MSQDestination +{ + public static final String TYPE = "durableStorage"; + + public static final DurableStorageDestination INSTANCE = new DurableStorageDestination(); + + + private DurableStorageDestination() + { + // Singleton. + } + + @JsonCreator + public static DurableStorageDestination instance() + { + return INSTANCE; + } + + @Override + public String toString() + { + return "DurableStorageDestination{}"; + } + +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSelectDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQSelectDestination.java similarity index 96% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSelectDestination.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQSelectDestination.java index 9ef8fe2c1cfd..d41a85622693 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSelectDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQSelectDestination.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.destination; /** * Determines the destination for results of select queries. 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/destination/TaskReportMSQDestination.java similarity index 89% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/TaskReportMSQDestination.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java index 3038f5d65dd9..30355e4dde17 100644 --- 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/destination/TaskReportMSQDestination.java @@ -17,14 +17,15 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.msq.indexing.MSQDestination; public class TaskReportMSQDestination implements MSQDestination { public static final TaskReportMSQDestination INSTANCE = new TaskReportMSQDestination(); - static final String TYPE = "taskReport"; + public static final String TYPE = "taskReport"; private TaskReportMSQDestination() { @@ -42,4 +43,5 @@ public String toString() { return "TaskReportMSQDestination{}"; } + } 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/processor/KeyStatisticsCollectionProcessor.java similarity index 99% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/KeyStatisticsCollectionProcessor.java index f9832670801c..2539dafcc9fd 100644 --- 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/processor/KeyStatisticsCollectionProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.processor; import com.google.common.primitives.Ints; import it.unimi.dsi.fastutil.ints.IntSet; 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/processor/SegmentGeneratorFrameProcessor.java similarity index 99% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessor.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index 4952ee1ab2ad..78b3e16f6702 100644 --- 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/processor/SegmentGeneratorFrameProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.processor; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; 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/processor/SegmentGeneratorFrameProcessorFactory.java similarity index 99% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/SegmentGeneratorFrameProcessorFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java index 74b7a9aa11dc..998e04295579 100644 --- 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/processor/SegmentGeneratorFrameProcessorFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.processor; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -38,6 +38,7 @@ import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.counters.SegmentGeneratorMetricsWrapper; import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.ReadableInput; 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 index fd06d1d79f7a..c12c55405736 100644 --- 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 @@ -29,7 +29,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.msq.exec.Limits; -import org.apache.druid.msq.indexing.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; 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 index d333edb7d2d4..39843db2ba7f 100644 --- 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 @@ -277,7 +277,7 @@ public int getMaxWorkerCount() @JsonProperty("shuffleCheckHasMultipleValues") @JsonInclude(JsonInclude.Include.NON_DEFAULT) - boolean getShuffleCheckHasMultipleValues() + public boolean getShuffleCheckHasMultipleValues() { return shuffleCheckHasMultipleValues; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java new file mode 100644 index 000000000000..20e2a20fa683 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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.results; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +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.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 javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; +import java.util.function.Consumer; + +@JsonTypeName("selectResults") +public class QueryResultFrameProcessorFactory extends BaseFrameProcessorFactory +{ + + @JsonCreator + public QueryResultFrameProcessorFactory() + { + } + + @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 + { + // Expecting a single input slice from some prior stage. + final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); + + if (inputSliceReader.numReadableInputs(slice) == 0) { + return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); + } + + + final OutputChannel outputChannel = outputChannelFactory.openChannel(0); + + final Sequence readableInputs = + Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); + + final Sequence> processors = readableInputs.map( + readableInput -> new QueryResultsFrameProcessor(readableInput.getChannel(), + outputChannel.getWritableChannel())); + + return new ProcessorsAndChannels<>( + processors, + OutputChannels.wrapReadOnly(ImmutableList.of(outputChannel)) + ); + + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.java new file mode 100644 index 000000000000..c945fd33e7c6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.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.querykit.results; + +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.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.ReturnOrAwait; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +public class QueryResultsFrameProcessor implements FrameProcessor +{ + long numRows = 0L; + private final ReadableFrameChannel inChannel; + private final WritableFrameChannel outChannel; + + public QueryResultsFrameProcessor( + final ReadableFrameChannel inChannel, + final WritableFrameChannel outChannel + ) + { + this.inChannel = inChannel; + this.outChannel = outChannel; + } + + @Override + public List inputChannels() + { + return Collections.singletonList(inChannel); + } + + @Override + public List outputChannels() + { + return Collections.singletonList(outChannel); + } + + @Override + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + { + if (readableInputs.isEmpty()) { + return ReturnOrAwait.awaitAll(1); + } + if (inChannel.isFinished()) { + return ReturnOrAwait.returnObject(numRows); + } + writeFrame(inChannel.read()); + return ReturnOrAwait.awaitAll(1); + } + + @Override + public void cleanup() throws IOException + { + FrameProcessors.closeAll(inputChannels(), outputChannels()); + } + + private void writeFrame(final Frame frame) throws IOException + { + outChannel.write(frame); + numRows += frame.numRows(); + } +} 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/input/DurableStorageInputChannelFactory.java similarity index 84% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java index 8e8315d9024b..dda4ac246676 100644 --- 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/input/DurableStorageInputChannelFactory.java @@ -17,13 +17,12 @@ * under the License. */ -package org.apache.druid.msq.shuffle; +package org.apache.druid.msq.shuffle.input; import com.google.common.base.Preconditions; import org.apache.commons.io.IOUtils; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableInputStreamFrameChannel; -import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; @@ -31,6 +30,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; import org.apache.druid.storage.StorageConnector; import java.io.IOException; @@ -42,7 +42,7 @@ /** * Provides input channels connected to durable storage. */ -public class DurableStorageInputChannelFactory implements InputChannelFactory +public abstract class DurableStorageInputChannelFactory implements InputChannelFactory { private static final Logger LOG = new Logger(DurableStorageInputChannelFactory.class); @@ -69,15 +69,24 @@ public DurableStorageInputChannelFactory( public static DurableStorageInputChannelFactory createStandardImplementation( final String controllerTaskId, final StorageConnector storageConnector, - final Closer closer + final Closer closer, + final boolean isQueryResults ) { final ExecutorService remoteInputStreamPool = Executors.newCachedThreadPool(Execs.makeThreadFactory(controllerTaskId + "-remote-fetcher-%d")); closer.register(remoteInputStreamPool::shutdownNow); - return new DurableStorageInputChannelFactory(controllerTaskId, storageConnector, remoteInputStreamPool); + if (isQueryResults) { + return new DurableStorageQueryResultsInputChannelFactory( + controllerTaskId, + storageConnector, + remoteInputStreamPool + ); + } + return new DurableStorageStageInputChannelFactory(controllerTaskId, storageConnector, remoteInputStreamPool); } + @Override public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException { @@ -138,11 +147,7 @@ public String findSuccessfulPartitionOutput( final int partitionNumber ) throws IOException { - String successfulFilePath = DurableStorageUtils.getSuccessFilePath( - controllerTaskId, - stageNumber, - workerNo - ); + String successfulFilePath = getWorkerOutputSuccessFilePath(controllerTaskId, stageNumber, workerNo); if (!storageConnector.pathExists(successfulFilePath)) { throw new ISE( @@ -169,12 +174,23 @@ public String findSuccessfulPartitionOutput( successfulTaskId ); - return DurableStorageUtils.getPartitionOutputsFileNameForPartition( + return getPartitionOutputsFileNameForPartition( controllerTaskId, stageNumber, workerNo, - successfulTaskId, - partitionNumber + partitionNumber, + successfulTaskId ); } + + public abstract String getPartitionOutputsFileNameForPartition( + String controllerTaskId, + int stageNumber, + int workerNo, + int partitionNumber, + String successfulTaskId + ); + + public abstract String getWorkerOutputSuccessFilePath(String controllerTaskId, int stageNumber, int workerNumber); + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java new file mode 100644 index 000000000000..bd8f5da5bed8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.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.shuffle.input; + +import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.storage.StorageConnector; + +import java.util.concurrent.ExecutorService; + +public class DurableStorageQueryResultsInputChannelFactory extends DurableStorageInputChannelFactory +{ + + public DurableStorageQueryResultsInputChannelFactory( + String controllerTaskId, + StorageConnector storageConnector, + ExecutorService remoteInputStreamPool + ) + { + super(controllerTaskId, storageConnector, remoteInputStreamPool); + } + + @Override + public String getPartitionOutputsFileNameForPartition( + String controllerTaskId, + int stageNumber, + int workerNo, + int partitionNumber, + String successfulTaskId + ) + { + return DurableStorageUtils.getQueryResultsFileNameForPartition(controllerTaskId, + stageNumber, + workerNo, + successfulTaskId, + partitionNumber); + } + + @Override + public String getWorkerOutputSuccessFilePath(String controllerTaskId, int stageNumber, int workerNo) + { + return DurableStorageUtils.getQueryResultsSuccessFilePath(controllerTaskId, stageNumber, workerNo); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java new file mode 100644 index 000000000000..d6d142924e2f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.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.shuffle.input; + +import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.storage.StorageConnector; + +import java.util.concurrent.ExecutorService; + +public class DurableStorageStageInputChannelFactory extends DurableStorageInputChannelFactory +{ + public DurableStorageStageInputChannelFactory( + String controllerTaskId, + StorageConnector storageConnector, + ExecutorService remoteInputStreamPool + ) + { + super(controllerTaskId, storageConnector, remoteInputStreamPool); + } + + @Override + public String getPartitionOutputsFileNameForPartition( + String controllerTaskId, + int stageNumber, + int workerNo, + int partitionNumber, + String successfulTaskId + ) + { + return DurableStorageUtils.getPartitionOutputsFileNameForPartition( + controllerTaskId, + stageNumber, + workerNo, + successfulTaskId, + partitionNumber + ); + } + + @Override + public String getWorkerOutputSuccessFilePath(String controllerTaskId, int stageNumber, int workerNo) + { + return DurableStorageUtils.getWorkerOutputSuccessFilePath(controllerTaskId, stageNumber, workerNo); + } +} 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/input/WorkerInputChannelFactory.java similarity index 98% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerInputChannelFactory.java index cfb7c64a1a7e..f583596bb88f 100644 --- 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/input/WorkerInputChannelFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.shuffle; +package org.apache.druid.msq.shuffle.input; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.FutureCallback; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java new file mode 100644 index 000000000000..a02993c03acc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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.output; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.channel.ByteTracker; +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.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.storage.StorageConnector; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public abstract class DurableStorageOutputChannelFactory implements OutputChannelFactory +{ + private static final Logger LOG = new Logger(DurableStorageTaskOutputChannelFactoryImpl.class); + protected final String controllerTaskId; + protected final int workerNumber; + protected final int stageNumber; + protected final String taskId; + protected final int frameSize; + protected final StorageConnector storageConnector; + protected final File tmpDir; + protected final ExecutorService remoteInputStreamPool; + + public DurableStorageOutputChannelFactory( + final String controllerTaskId, + final int workerNumber, + final int stageNumber, + final String taskId, + final int frameSize, + final StorageConnector storageConnector, + final File tmpDir + ) + { + this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId"); + this.workerNumber = workerNumber; + this.stageNumber = stageNumber; + this.taskId = taskId; + this.frameSize = frameSize; + this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector"); + this.tmpDir = Preconditions.checkNotNull(tmpDir, "tmpDir is null"); + this.remoteInputStreamPool = + Executors.newCachedThreadPool(Execs.makeThreadFactory("-remote-fetcher-%d")); + } + + /** + * 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 int workerNumber, + final int stageNumber, + final String taskId, + final int frameSize, + final StorageConnector storageConnector, + final File tmpDir, + final boolean isQueryResults + ) + { + if (isQueryResults) { + return new DurableStorageResultsOutputChannelFactory( + controllerTaskId, + workerNumber, + stageNumber, + taskId, + frameSize, + storageConnector, + tmpDir + ); + } else { + return new DurableStorageTaskOutputChannelFactoryImpl( + controllerTaskId, + workerNumber, + stageNumber, + taskId, + frameSize, + storageConnector, + tmpDir + ); + } + } + + + /** + * Creates a file with name __success and adds the worker's id which has successfully written its outputs. While reading + * this file can be used to find out the worker which has written its outputs completely. + * Rename operation is not very quick in cloud storage like S3 due to which this alternative + * route has been taken. + * If the success file is already present in the location, then this method is a noop + */ + public void createSuccessFile(String taskId) throws IOException + { + String fileName = getSuccessFilePath(); + if (storageConnector.pathExists(fileName)) { + LOG.warn("Path [%s] already exists. Won't attempt to rewrite on top of it.", fileName); + return; + } + OutputStreamWriter os = new OutputStreamWriter(storageConnector.write(fileName), StandardCharsets.UTF_8); + os.write(taskId); // Add some dummy content in the file + os.close(); + } + + protected abstract String getSuccessFilePath(); + + @Override + public OutputChannel openNilChannel(int partitionNumber) + { + final String fileName = getFileNameForPartition(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, ByteTracker.unboundedTracker()) + .close(); + return OutputChannel.nil(partitionNumber); + } + catch (IOException e) { + throw new ISE( + e, + "Unable to create empty remote output of stage [%d], partition [%d] for worker [%d]", + stageNumber, + partitionNumber, + workerNumber + ); + } + } + + protected abstract String getFileNameForPartition(int partitionNumber); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageResultsOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageResultsOutputChannelFactory.java new file mode 100644 index 000000000000..b058d0f69840 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageResultsOutputChannelFactory.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.shuffle.output; + +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.ByteTracker; +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.PartitionedOutputChannel; +import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.storage.StorageConnector; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.Channels; + +public class DurableStorageResultsOutputChannelFactory extends DurableStorageOutputChannelFactory +{ + + + public DurableStorageResultsOutputChannelFactory( + String controllerTaskId, + int workerNumber, + int stageNumber, + String taskId, + int frameSize, + StorageConnector storageConnector, + File tmpDir + ) + { + super(controllerTaskId, workerNumber, stageNumber, taskId, frameSize, storageConnector, tmpDir); + } + + @Override + protected String getSuccessFilePath() + { + return DurableStorageUtils.getQueryResultsSuccessFilePath( + controllerTaskId, + stageNumber, + workerNumber + ); + } + + @Override + protected String getFileNameForPartition(int partitionNumber) + { + return DurableStorageUtils.getQueryResultsFileNameForPartition(controllerTaskId, + stageNumber, + workerNumber, + taskId, + partitionNumber); + } + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + final String fileName = getFileNameForPartition(partitionNumber); + + final WritableFrameFileChannel writableChannel = + new WritableFrameFileChannel( + FrameFileWriter.open( + Channels.newChannel(storageConnector.write(fileName)), + null, + ByteTracker.unboundedTracker() + ) + ); + + return OutputChannel.pair( + writableChannel, + ArenaMemoryAllocator.createOnHeap(frameSize), + () -> ReadableNilFrameChannel.INSTANCE, + partitionNumber + ); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) + { + throw new UOE("%s does not support this call", DurableStorageResultsOutputChannelFactory.class.getSimpleName()); + } + +} 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/output/DurableStorageTaskOutputChannelFactoryImpl.java similarity index 60% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactoryImpl.java index ca7041cfa00c..fbe3adf4eb48 100644 --- 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/output/DurableStorageTaskOutputChannelFactoryImpl.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.shuffle; +package org.apache.druid.msq.shuffle.output; import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; @@ -33,100 +33,63 @@ import org.apache.druid.frame.file.FrameFileFooter; 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.frame.processor.PartitionedOutputChannel; import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.MappedByteBufferHandler; 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.storage.StorageConnector; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStreamWriter; import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.Channels; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.function.Supplier; -public class DurableStorageOutputChannelFactory implements OutputChannelFactory +public class DurableStorageTaskOutputChannelFactoryImpl + extends DurableStorageOutputChannelFactory { - private static final Logger LOG = new Logger(DurableStorageOutputChannelFactory.class); - - private final String controllerTaskId; - private final int workerNumber; - private final int stageNumber; - private final String taskId; - private final int frameSize; - private final StorageConnector storageConnector; - private final File tmpDir; - private final ExecutorService remoteInputStreamPool; - - public DurableStorageOutputChannelFactory( - final String controllerTaskId, - final int workerNumber, - final int stageNumber, - final String taskId, - final int frameSize, - final StorageConnector storageConnector, - final File tmpDir + public DurableStorageTaskOutputChannelFactoryImpl( + String controllerTaskId, + int workerNumber, + int stageNumber, + String taskId, + int frameSize, + StorageConnector storageConnector, + File tmpDir ) { - this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId"); - this.workerNumber = workerNumber; - this.stageNumber = stageNumber; - this.taskId = taskId; - this.frameSize = frameSize; - this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector"); - this.tmpDir = Preconditions.checkNotNull(tmpDir, "tmpDir is null"); - this.remoteInputStreamPool = - Executors.newCachedThreadPool(Execs.makeThreadFactory("-remote-fetcher-%d")); + super(controllerTaskId, workerNumber, stageNumber, taskId, frameSize, storageConnector, tmpDir); } - /** - * 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 int workerNumber, - final int stageNumber, - final String taskId, - final int frameSize, - final StorageConnector storageConnector, - final File tmpDir - ) + @Override + protected String getSuccessFilePath() { - return new DurableStorageOutputChannelFactory( - controllerTaskId, - workerNumber, - stageNumber, - taskId, - frameSize, - storageConnector, - tmpDir - ); + return DurableStorageUtils.getWorkerOutputSuccessFilePath(controllerTaskId, stageNumber, workerNumber); } @Override - public OutputChannel openChannel(int partitionNumber) throws IOException + protected String getFileNameForPartition(int partitionNumber) { - final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition( + return DurableStorageUtils.getPartitionOutputsFileNameForPartition( controllerTaskId, stageNumber, workerNumber, taskId, partitionNumber ); + } + + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + final String fileName = getFileNameForPartition(partitionNumber); final WritableFrameFileChannel writableChannel = new WritableFrameFileChannel( FrameFileWriter.open( @@ -211,8 +174,8 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele FileUtils.mkdirp(footerFile.getParentFile()); Preconditions.checkState(footerFile.createNewFile(), "Unable to create local footer file"); try (FileOutputStream footerFileStream = new FileOutputStream(footerFile); - InputStream footerInputStream = - storageConnector.readRange(fileName, channelSize - footerLength, footerLength)) { + InputStream footerInputStream = + storageConnector.readRange(fileName, channelSize - footerLength, footerLength)) { IOUtils.copy(footerInputStream, footerFileStream); } MappedByteBufferHandler mapHandle = FileUtils.map(footerFile); @@ -239,49 +202,4 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele ); } - @Override - public OutputChannel openNilChannel(int partitionNumber) - { - final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition( - controllerTaskId, - stageNumber, - workerNumber, - taskId, - 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, ByteTracker.unboundedTracker()).close(); - return OutputChannel.nil(partitionNumber); - } - catch (IOException e) { - throw new ISE( - e, - "Unable to create empty remote output of stage [%d], partition [%d] for worker [%d]", - stageNumber, - partitionNumber, - workerNumber - ); - } - } - - /** - * Creates a file with name __success and adds the worker's id which has successfully written its outputs. While reading - * this file can be used to find out the worker which has written its outputs completely. - * Rename operation is not very quick in cloud storage like S3 due to which this alternative - * route has been taken. - * If the success file is already present in the location, then this method is a noop - */ - public void createSuccessFile(String taskId) throws IOException - { - String fileName = DurableStorageUtils.getSuccessFilePath(controllerTaskId, stageNumber, workerNumber); - if (storageConnector.pathExists(fileName)) { - LOG.warn("Path [%s] already exists. Won't attempt to rewrite on top of it.", fileName); - return; - } - OutputStreamWriter os = new OutputStreamWriter(storageConnector.write(fileName), StandardCharsets.UTF_8); - os.write(taskId); // Add some dummy content in the file - os.close(); - } } 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 index c0de08a809f8..8b1a8e55c7e1 100644 --- 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 @@ -34,12 +34,14 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.msq.exec.MSQTasks; -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.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.util.MSQTaskQueryMakerUtils; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.QueryContext; @@ -229,8 +231,11 @@ public QueryResponse runQuery(final DruidQuery druidQuery) if (ctxDestination != null && !DESTINATION_REPORT.equals(ctxDestination)) { throw new IAE("Cannot SELECT with destination [%s]", ctxDestination); } - - destination = TaskReportMSQDestination.instance(); + if (MultiStageQueryContext.getSelectDestination(sqlQueryContext).equals(MSQSelectDestination.TASK_REPORT)) { + destination = TaskReportMSQDestination.instance(); + } else { + destination = DurableStorageDestination.instance(); + } } final Map nativeQueryContextOverrides = new HashMap<>(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java index e131fa85c737..84292d705592 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java @@ -38,7 +38,7 @@ public class ResultSetInformation @Nullable private final ResultFormat resultFormat; @Nullable - private final List records; + private final List records; @Nullable private final String dataSource; @Nullable @@ -50,7 +50,7 @@ public ResultSetInformation( @JsonProperty("totalSizeInBytes") @Nullable Long totalSizeInBytes, @JsonProperty("resultFormat") @Nullable ResultFormat resultFormat, @JsonProperty("dataSource") @Nullable String dataSource, - @JsonProperty("sampleRecords") @Nullable List records, + @JsonProperty("sampleRecords") @Nullable List records, @JsonProperty("pages") @Nullable List pages ) { @@ -97,7 +97,7 @@ public String getDataSource() @JsonProperty("sampleRecords") @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) - public List getRecords() + public List getRecords() { return records; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index ce30284e5936..b8e6e8bcc603 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.io.CountingOutputStream; import com.google.common.util.concurrent.ListenableFuture; @@ -32,17 +31,28 @@ import org.apache.druid.error.DruidException; import org.apache.druid.error.ErrorResponse; import org.apache.druid.error.QueryExceptionCompat; +import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.guice.annotations.MSQ; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; 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.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.guice.MultiStageQuery; import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQDestination; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; import org.apache.druid.msq.sql.MSQTaskQueryMaker; import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.sql.SqlStatementState; @@ -50,8 +60,10 @@ import org.apache.druid.msq.sql.entity.PageInformation; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.msq.util.SqlStatementResourceHelper; import org.apache.druid.query.ExecutionMode; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryException; import org.apache.druid.rpc.indexing.OverlordClient; @@ -68,6 +80,8 @@ import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.sql.http.SqlQuery; import org.apache.druid.sql.http.SqlResource; +import org.apache.druid.storage.NilStorageConnector; +import org.apache.druid.storage.StorageConnector; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -83,10 +97,13 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; +import java.util.stream.Collectors; @Path("/druid/v2/sql/statements/") @@ -98,6 +115,7 @@ public class SqlStatementResource private final AuthorizerMapper authorizerMapper; private final ObjectMapper jsonMapper; private final OverlordClient overlordClient; + private final StorageConnector storageConnector; @Inject @@ -105,15 +123,20 @@ public SqlStatementResource( final @MSQ SqlStatementFactory msqSqlStatementFactory, final AuthorizerMapper authorizerMapper, final ObjectMapper jsonMapper, - final OverlordClient overlordClient + final OverlordClient overlordClient, + final @MultiStageQuery StorageConnector storageConnector ) { this.msqSqlStatementFactory = msqSqlStatementFactory; this.authorizerMapper = authorizerMapper; this.jsonMapper = jsonMapper; this.overlordClient = overlordClient; + this.storageConnector = storageConnector; } + /** + * API for clients like web-console to check if this resource is enabled. + */ @GET @Path("/enabled") @@ -142,26 +165,11 @@ public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletReques final HttpStatement stmt = msqSqlStatementFactory.httpStatement(sqlQuery, req); final String sqlQueryId = stmt.sqlQueryId(); final String currThreadName = Thread.currentThread().getName(); + boolean isDebug = false; try { - ExecutionMode executionMode = QueryContexts.getAsEnum( - QueryContexts.CTX_EXECUTION_MODE, - sqlQuery.getContext().get(QueryContexts.CTX_EXECUTION_MODE), - ExecutionMode.class - ); - if (ExecutionMode.ASYNC != executionMode) { - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - StringUtils.format( - "The statement sql api only supports sync mode[%s]. Please set context parameter [%s=%s] in the context payload", - ExecutionMode.ASYNC, - QueryContexts.CTX_EXECUTION_MODE, - ExecutionMode.ASYNC - ) - ) - ); - } + QueryContext queryContext = QueryContext.of(sqlQuery.getContext()); + isDebug = queryContext.isDebug(); + contextChecks(queryContext); Thread.currentThread().setName(StringUtils.format("statement_sql[%s]", sqlQueryId)); @@ -201,8 +209,11 @@ public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletReques // Calcite throws java.lang.AssertionError at various points in planning/validation. catch (AssertionError | Exception e) { stmt.reporter().failed(e); - log.noStackTrace().warn(e, "Failed to handle query: %s", sqlQueryId); - + if (isDebug) { + log.warn(e, "Failed to handle query: %s", sqlQueryId); + } else { + log.noStackTrace().warn(e, "Failed to handle query: %s", sqlQueryId); + } return buildNonOkResponse( DruidException.forPersona(DruidException.Persona.DEVELOPER) .ofCategory(DruidException.Category.UNCATEGORIZED) @@ -286,13 +297,11 @@ public Response doGetResults( final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); if (page != null && page < 0) { - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "Page cannot be negative. Please pass a positive number." - ) - ); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Page cannot be negative. Please pass a positive number." + ); } TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); @@ -306,76 +315,127 @@ public Response doGetResults( } MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); - SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); + throwIfQueryIsNotSuccessful(queryId, statusPlus); - if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "Query[%s] is currently in [%s] state. Please wait for it to complete.", - queryId, - sqlStatementState - ) - ); - } else if (sqlStatementState == SqlStatementState.FAILED) { - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "Query[%s] failed. Hit status api for more details.", - queryId - ) - ); - } else { - Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); - if (!signature.isPresent()) { - return Response.ok().build(); - } + Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); + if (!signature.isPresent() || MSQControllerTask.isIngestion(msqControllerTask.getQuerySpec())) { + // Since it's not a select query, nothing to return. + return Response.ok().build(); + } + // returning results + final Closer closer = Closer.create(); + + final Optional> results; + if (msqControllerTask.getQuerySpec().getDestination() instanceof TaskReportMSQDestination) { if (page != null && page > 0) { // Results from task report are only present as one page. - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build("Page number is out of range of the results.") - ); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Page number is out of range of the results."); } - Optional> results = SqlStatementResourceHelper.getResults( - SqlStatementResourceHelper.getPayload( - contactOverlord(overlordClient.taskReportAsMap(queryId)) - ) - ); - - return Response.ok((StreamingOutput) outputStream -> { - CountingOutputStream os = new CountingOutputStream(outputStream); - - try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { - List rowSignature = signature.get(); - writer.writeResponseStart(); - - for (long k = 0; k < results.get().size(); k++) { - writer.writeRowStart(); - for (int i = 0; i < rowSignature.size(); i++) { - writer.writeRowField( - rowSignature.get(i).getColName(), - ((List) results.get().get(Math.toIntExact(k))).get(i) - ); - } - writer.writeRowEnd(); - } + MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId)) + ), MSQTaskReportPayload.class); - writer.writeResponseEnd(); - } - catch (Exception e) { - log.error(e, "Unable to stream results back for query[%s]", queryId); - throw new ISE(e, "Unable to stream results back for query[%s]", queryId); - } - }).build(); + if (msqTaskReportPayload.getResults().getResultYielder() == null) { + results = Optional.empty(); + } else { + results = Optional.of(msqTaskReportPayload.getResults().getResultYielder()); + } + + } else if (msqControllerTask.getQuerySpec().getDestination() instanceof DurableStorageDestination) { + + MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId)) + ), MSQTaskReportPayload.class); + + + List pages = SqlStatementResourceHelper.populatePageList( + msqTaskReportPayload, + msqControllerTask.getQuerySpec() + .getDestination() + ) + .get(); + + + final StageDefinition finalStage = Objects.requireNonNull(SqlStatementResourceHelper.getFinalStage( + msqTaskReportPayload)).getStageDefinition(); + + if (page == null) { + final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( + msqControllerTask.getId(), + storageConnector, + closer, + true + ); + results = Optional.of(Yielders.each( + Sequences.concat(pages.stream().map(pageInformation -> { + try { + return new FrameChannelSequence( + standardImplementation.openChannel(finalStage.getId(), + (int) pageInformation.getId(), 0 + ) + ); + } + catch (Exception e) { + throw new RuntimeException( + e); + } + }).collect(Collectors.toList())).flatMap( + frame -> SqlStatementResourceHelper.getResultSequence( + msqControllerTask, + finalStage, + frame, + jsonMapper + ) + + ))); + + } else { + PageInformation pageInformation = getPageInformationForPageId(pages, page); + final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( + msqControllerTask.getId(), + storageConnector, + closer, + true + ); + results = Optional.of(Yielders.each(new FrameChannelSequence(standardImplementation.openChannel( + finalStage.getId(), + (int) pageInformation.getId(), + 0 + )).flatMap(frame -> SqlStatementResourceHelper.getResultSequence( + msqControllerTask, + finalStage, + frame, + jsonMapper + )))); + } + } else { + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build( + "MSQ select destination[%s] not supported. Please reach out to druid slack community for more help.", + msqControllerTask.getQuerySpec().getDestination().toString() + ); } + if (!results.isPresent()) { + // no results, return empty + return Response.ok().build(); + } + + return Response.ok((StreamingOutput) outputStream -> resultPusher( + queryId, + signature, + closer, + results, + new CountingOutputStream(outputStream) + )).build(); } + + catch (DruidException e) { return buildNonOkResponse(e); } @@ -395,6 +455,84 @@ public Response doGetResults( } } + private PageInformation getPageInformationForPageId(List pages, Long pageId) + { + for (PageInformation pageInfo : pages) { + if (pageInfo.getId() == pageId) { + return pageInfo; + } + } + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Invalid page id [%d] passed.", pageId); + } + + private void resultPusher( + String queryId, + Optional> signature, + Closer closer, + Optional> results, + CountingOutputStream os + ) throws IOException + { + try { + try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { + Yielder yielder = results.get(); + List rowSignature = signature.get(); + writer.writeResponseStart(); + + while (!yielder.isDone()) { + writer.writeRowStart(); + Object[] row = yielder.get(); + for (int i = 0; i < Math.min(rowSignature.size(), row.length); i++) { + writer.writeRowField( + rowSignature.get(i).getColName(), + row[i] + ); + } + writer.writeRowEnd(); + yielder = yielder.next(null); + } + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to stream results back for query[%s]", queryId); + throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + } + } + catch (Exception e) { + log.error(e, "Unable to stream results back for query[%s]", queryId); + throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + } + finally { + closer.close(); + } + } + + private static void throwIfQueryIsNotSuccessful(String queryId, TaskStatusPlus statusPlus) + { + SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); + if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + queryId, + sqlStatementState + ); + } else if (sqlStatementState == SqlStatementState.FAILED) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] failed. Hit status api for more details.", + queryId + ); + } else { + // do nothing + System.out.println("abc"); + } + } + /** * Queries can be canceled while in any {@link SqlStatementState}. Canceling a query that has already completed will be a no-op. * @@ -562,35 +700,72 @@ private Response buildNonOkResponse(DruidException exception) .build(); } + @SuppressWarnings("ReassignedVariable") private Optional getSampleResults( String asyncResultId, - boolean isSelectQuery, String dataSource, - SqlStatementState sqlStatementState + SqlStatementState sqlStatementState, + MSQDestination msqDestination ) { if (sqlStatementState == SqlStatementState.SUCCESS) { Map payload = SqlStatementResourceHelper.getPayload(contactOverlord(overlordClient.taskReportAsMap( asyncResultId))); - Optional> rowsAndSize = SqlStatementResourceHelper.getRowsAndSizeFromPayload( - payload, - isSelectQuery + MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(payload, MSQTaskReportPayload.class); + + + Optional> pageList = SqlStatementResourceHelper.populatePageList( + msqTaskReportPayload, + msqDestination ); - return Optional.of(new ResultSetInformation( - rowsAndSize.orElse(new Pair<>(null, null)).lhs, - rowsAndSize.orElse(new Pair<>(null, null)).rhs, - null, - dataSource, - // only populate sample results in case a select query is successful - isSelectQuery ? SqlStatementResourceHelper.getResults(payload).orElse(null) : null, - ImmutableList.of( - new PageInformation( - rowsAndSize.orElse(new Pair<>(null, null)).lhs, - rowsAndSize.orElse(new Pair<>(null, null)).rhs, - 0 - ) + Long rows = null; + Long size = null; + boolean isSelectQuery = false; + if (pageList.isPresent()) { + if (msqDestination instanceof TaskReportMSQDestination) { + isSelectQuery = true; + if (pageList.get().size() == 1) { + rows = pageList.get().get(0).getNumRows(); + size = pageList.get().get(0).getSizeInBytes(); + } + } else if (msqDestination instanceof DataSourceMSQDestination) { + isSelectQuery = false; + if (pageList.get().size() == 1) { + rows = pageList.get().get(0).getNumRows(); + size = pageList.get().get(0).getSizeInBytes(); + } + } else { + isSelectQuery = true; + rows = 0L; + size = 0L; + for (PageInformation pageInformation : pageList.get()) { + rows += pageInformation.getNumRows(); + size += pageInformation.getSizeInBytes(); + } + } + } + + List results = null; + if (isSelectQuery) { + results = new ArrayList<>(); + Yielder yielder = msqTaskReportPayload.getResults().getResultYielder(); + while (!yielder.isDone()) { + results.add(yielder.get()); + yielder = yielder.next(null); + } + } + + return Optional.of( + new ResultSetInformation( + // since the rows can be sampled, get the number of rows from counters + rows, + size, + null, + dataSource, + results, + isSelectQuery ? pageList.get() : null ) - )); + ); } else { return Optional.empty(); } @@ -633,9 +808,9 @@ private Optional getStatementStatus(String queryId, String c taskResponse.getStatus().getDuration(), withResults ? getSampleResults( queryId, - signature.isPresent(), msqControllerTask.getDataSource(), - sqlStatementState + sqlStatementState, + msqControllerTask.getQuerySpec().getDestination() ).orElse(null) : null, null )); @@ -675,4 +850,43 @@ private T contactOverlord(final ListenableFuture future) .build("Unable to contact overlord " + e.getMessage()); } } + + + private void contextChecks(QueryContext queryContext) + { + ExecutionMode executionMode = queryContext.getEnum( + QueryContexts.CTX_EXECUTION_MODE, + ExecutionMode.class, + null + ); + if (ExecutionMode.ASYNC != executionMode) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + StringUtils.format( + "The statement sql api only supports sync mode[%s]. Please set context parameter [%s=%s] in the context payload", + ExecutionMode.ASYNC, + QueryContexts.CTX_EXECUTION_MODE, + ExecutionMode.ASYNC + ) + ); + } + + + MSQSelectDestination selectDestination = MultiStageQueryContext.getSelectDestination(queryContext); + if (selectDestination == MSQSelectDestination.DURABLE_STORAGE + && storageConnector instanceof NilStorageConnector) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + StringUtils.format( + "The statement sql api only supports select destination [%s=%s]. " + + "Its recommended to configure durable storage as it allows the user to fetch big results. " + + "Please contact your cluster admin to configure durable storage.", + MultiStageQueryContext.CTX_SELECT_DESTINATION, + selectDestination.name() + ) + ); + } + } } 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 index 3c951c7d0cc3..dda2dfd7f11b 100644 --- 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 @@ -27,7 +27,7 @@ import com.opencsv.RFC4180ParserBuilder; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Limits; -import org.apache.druid.msq.indexing.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.sql.MSQMode; import org.apache.druid.query.QueryContext; @@ -217,10 +217,20 @@ public static MSQSelectDestination getSelectDestination(final QueryContext query queryContext.getString( CTX_SELECT_DESTINATION, DEFAULT_SELECT_DESTINATION - ) + ).toUpperCase() ); } + @Nullable + public static MSQSelectDestination getSelectDestinationOrNull(final QueryContext queryContext) + { + String selectDestination = queryContext.getString(CTX_SELECT_DESTINATION); + if (selectDestination == null) { + return null; + } + return MSQSelectDestination.valueOf(selectDestination.toUpperCase()); + } + public static int getRowsInMemory(final QueryContext queryContext) { return queryContext.getInt(CTX_ROWS_IN_MEMORY, DEFAULT_ROWS_IN_MEMORY); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java index 4c0474e1f93f..761f0e6907c8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -20,28 +20,55 @@ package org.apache.druid.msq.util; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.error.DruidException; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.processor.FrameProcessors; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; 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.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.counters.CounterSnapshots; +import org.apache.druid.msq.counters.QueryCounterSnapshot; +import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.MSQDestination; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.report.MSQStagesReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.sql.SqlStatementState; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.PageInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.apache.druid.sql.calcite.run.SqlResults; +import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; public class SqlStatementResourceHelper { @@ -50,7 +77,7 @@ public static Optional> getSignature( ) { // only populate signature for select q's - if (msqControllerTask.getQuerySpec().getDestination().getClass() == TaskReportMSQDestination.class) { + if (!MSQControllerTask.isIngestion(msqControllerTask.getQuerySpec())) { ColumnMappings columnMappings = msqControllerTask.getQuerySpec().getColumnMappings(); List sqlTypeNames = msqControllerTask.getSqlTypeNames(); if (sqlTypeNames == null || sqlTypeNames.size() != columnMappings.size()) { @@ -137,6 +164,66 @@ public static long getLastIndex(Long numberOfRows, long start) return last; } + public static Optional> populatePageList( + MSQTaskReportPayload msqTaskReportPayload, + MSQDestination msqDestination + ) + { + if (msqTaskReportPayload.getStages() == null || msqTaskReportPayload.getCounters() == null) { + return Optional.empty(); + } + int finalStage = msqTaskReportPayload.getStages().getStages().size() - 1; + + Map workerCounters = msqTaskReportPayload.getCounters().snapshotForStage(finalStage); + + if (workerCounters == null) { + return Optional.empty(); + } + + if (msqDestination instanceof DataSourceMSQDestination) { + long rows = 0L; + for (CounterSnapshots counterSnapshots : workerCounters.values()) { + QueryCounterSnapshot queryCounterSnapshot = counterSnapshots.getMap() + .getOrDefault("segmentGenerationProgress", null); + if (queryCounterSnapshot != null && queryCounterSnapshot instanceof SegmentGenerationProgressCounter.Snapshot) { + rows += ((SegmentGenerationProgressCounter.Snapshot) queryCounterSnapshot).getRowsPushed(); + } + } + if (rows != 0L) { + return Optional.of(ImmutableList.of(new PageInformation(rows, null, 0))); + } else { + return Optional.empty(); + } + } else if (msqDestination instanceof TaskReportMSQDestination) { + long rows = 0L; + long size = 0L; + for (CounterSnapshots counterSnapshots : workerCounters.values()) { + QueryCounterSnapshot queryCounterSnapshot = counterSnapshots.getMap().getOrDefault("output", null); + if (queryCounterSnapshot != null && queryCounterSnapshot instanceof ChannelCounters.Snapshot) { + rows += Arrays.stream(((ChannelCounters.Snapshot) queryCounterSnapshot).getRows()).sum(); + size += Arrays.stream(((ChannelCounters.Snapshot) queryCounterSnapshot).getBytes()).sum(); + } + } + return Optional.of(ImmutableList.of(new PageInformation(rows, size, 0))); + + } else if (msqDestination instanceof DurableStorageDestination) { + List pageList = new ArrayList<>(); + for (Map.Entry counterSnapshots : workerCounters.entrySet()) { + long rows = 0L; + long size = 0L; + QueryCounterSnapshot queryCounterSnapshot = counterSnapshots.getValue().getMap().getOrDefault("output", null); + if (queryCounterSnapshot != null && queryCounterSnapshot instanceof ChannelCounters.Snapshot) { + rows += Arrays.stream(((ChannelCounters.Snapshot) queryCounterSnapshot).getRows()).sum(); + size += Arrays.stream(((ChannelCounters.Snapshot) queryCounterSnapshot).getBytes()).sum(); + } + pageList.add(new PageInformation(rows, size, counterSnapshots.getKey())); + } + return Optional.of(pageList); + } else { + return Optional.empty(); + } + } + public static Optional> getRowsAndSizeFromPayload(Map payload, boolean isSelectQuery) { List stages = getList(payload, "stages"); @@ -234,6 +321,74 @@ protected DruidException makeException(DruidException.DruidExceptionBuilder bob) )); } + public static Sequence getResultSequence( + MSQControllerTask msqControllerTask, + StageDefinition finalStage, + Frame frame, + ObjectMapper jsonMapper + ) + { + final Cursor cursor = FrameProcessors.makeCursor(frame, finalStage.getFrameReader()); + + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final ColumnMappings columnMappings = msqControllerTask.getQuerySpec().getColumnMappings(); + @SuppressWarnings("rawtypes") + final List selectors = columnMappings.getMappings() + .stream() + .map(mapping -> columnSelectorFactory.makeColumnValueSelector( + mapping.getQueryColumn())) + .collect(Collectors.toList()); + + final List sqlTypeNames = msqControllerTask.getSqlTypeNames(); + Iterable retVal = () -> new Iterator() + { + @Override + public boolean hasNext() + { + return !cursor.isDone(); + } + + @Override + public Object[] next() + { + final Object[] row = new Object[columnMappings.size()]; + for (int i = 0; i < row.length; i++) { + final Object value = selectors.get(i).getObject(); + if (sqlTypeNames == null || msqControllerTask.getSqlResultsContext() == null) { + // SQL type unknown, or no SQL results context: pass-through as is. + row[i] = value; + } else { + row[i] = SqlResults.coerce( + jsonMapper, + msqControllerTask.getSqlResultsContext(), + value, + sqlTypeNames.get(i), + columnMappings.getOutputColumnName(i) + ); + } + } + cursor.advance(); + return row; + } + }; + return Sequences.simple(retVal); + } + + @Nullable + public static MSQStagesReport.Stage getFinalStage(MSQTaskReportPayload msqTaskReportPayload) + { + if (msqTaskReportPayload == null || msqTaskReportPayload.getStages().getStages() == null) { + return null; + } + int finalStageNumber = msqTaskReportPayload.getStages().getStages().size() - 1; + + for (MSQStagesReport.Stage stage : msqTaskReportPayload.getStages().getStages()) { + if (stage.getStageNumber() == finalStageNumber) { + return stage; + } + } + return null; + } public static Map getQueryExceptionDetails(Map payload) { return getMap(getMap(payload, "status"), "errorReport"); @@ -260,7 +415,7 @@ public static List getList(Map map, String key) * Get results from report */ @SuppressWarnings("unchecked") - public static Optional> getResults(Map payload) + public static Optional> getResults(Map payload) { Map resultsHolder = getMap(payload, "results"); @@ -273,7 +428,7 @@ public static Optional> getResults(Map payload) if (data != null) { rows.addAll(data); } - return Optional.of(rows); + return Optional.of((Yielders.each(Sequences.simple(rows)))); } public static Map getPayload(Map results) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java index 7b0a5e52d102..3f4c953bcb52 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java @@ -24,9 +24,9 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -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.destination.DataSourceMSQDestination; import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; import org.apache.druid.msq.indexing.error.InvalidNullByteFault; import org.apache.druid.msq.querykit.scan.ExternalColumnSelectorFactory; 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 index 0f4210e7f59a..2dbfbec697b4 100644 --- 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 @@ -35,9 +35,11 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.msq.indexing.MSQSelectDestination; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.test.CounterSnapshotMatcher; import org.apache.druid.msq.test.MSQTestBase; @@ -91,13 +93,36 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; @RunWith(Parameterized.class) public class MSQSelectTest extends MSQTestBase { + + public static final String QUERY_RESULTS_WITH_DURABLE_STORAGE = "query_results_with_durable_storage"; + + public static final String QUERY_RESULTS_WITH_DEFAULT = "query_results_with_default_storage"; + + public static final Map QUERY_RESULTS_WITH_DURABLE_STORAGE_CONTEXT = + ImmutableMap.builder() + .putAll(DURABLE_STORAGE_MSQ_CONTEXT) + .put( + MultiStageQueryContext.CTX_SELECT_DESTINATION, + MSQSelectDestination.DURABLE_STORAGE.name().toLowerCase() + ) + .build(); + + + public static final Map QUERY_RESULTS_WITH_DEFAULT_CONTEXT = + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put( + MultiStageQueryContext.CTX_SELECT_DESTINATION, + MSQSelectDestination.DURABLE_STORAGE.name().toLowerCase() + ) + .build(); + @Parameterized.Parameters(name = "{index}:with context {0}") public static Collection data() { @@ -105,7 +130,9 @@ public static Collection data() {DEFAULT, DEFAULT_MSQ_CONTEXT}, {DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT}, {FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT}, - {PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT} + {PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT}, + {QUERY_RESULTS_WITH_DURABLE_STORAGE, QUERY_RESULTS_WITH_DURABLE_STORAGE_CONTEXT}, + {QUERY_RESULTS_WITH_DEFAULT, QUERY_RESULTS_WITH_DEFAULT_CONTEXT} }; return Arrays.asList(data); } @@ -142,6 +169,9 @@ public void testCalculator() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -171,6 +201,9 @@ public void testSelectOnFoo() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -226,6 +259,9 @@ public void testSelectOnFoo2() .build()) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -291,6 +327,9 @@ public void testSelectOnFooDuplicateColumnNames() ) .columnMappings(expectedColumnMappings) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -350,6 +389,9 @@ public void testSelectOnFooWhereMatchesNoSegments() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -381,6 +423,9 @@ public void testSelectOnFooWhereMatchesNoData() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -413,6 +458,9 @@ public void testSelectAndOrderByOnFooWhereMatchesNoData() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -455,6 +503,9 @@ public void testGroupByOnFoo() ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) @@ -519,6 +570,9 @@ public void testGroupByOrderByDimension() ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setQueryContext(context) @@ -573,6 +627,9 @@ public void testSelectWithLimit() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -639,6 +696,9 @@ public void testSelectWithGroupByLimit() ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) @@ -666,6 +726,9 @@ public void testSelectLookup() .build()) .columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "EXPR$0")))) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{4L})) @@ -719,6 +782,9 @@ public void testJoinWithLookup() ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows( @@ -761,6 +827,9 @@ public void testSubquery() .query(query) .columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "cnt")))) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -929,6 +998,9 @@ private void testJoin(final JoinAlgorithm joinAlgorithm) ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -984,6 +1056,9 @@ public void testGroupByOrderByAggregation() ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(rowSignature) @@ -1060,6 +1135,9 @@ public void testGroupByOrderByAggregationWithLimit() ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(rowSignature) @@ -1134,6 +1212,9 @@ public void testGroupByOrderByAggregationWithLimitAndOffset() ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(rowSignature) @@ -1226,6 +1307,9 @@ public void testExternSelect1() throws IOException ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedCountersForStageWorkerChannel( @@ -1336,6 +1420,9 @@ public void testSelectOnUserDefinedSourceContainingWith() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -1401,6 +1488,9 @@ public void testScanWithMultiValueSelectQuery() ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(expectedResultSignature) .setQueryContext(context) @@ -1469,6 +1559,9 @@ public void testHavingOnApproximateCountDistinct() new ColumnMapping("a0", "col") ))) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setQueryContext(context) .setExpectedRowSignature(resultSignature) @@ -1520,6 +1613,9 @@ public void testGroupByWithMultiValue() ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(expectedMultiValueFooRowsGroup()) @@ -1590,6 +1686,9 @@ public void testGroupByWithMultiValueMvToArray() ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows( @@ -1657,6 +1756,9 @@ public void testGroupByArrayWithMultiValueMvToArray() ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(expected) @@ -1773,6 +1875,9 @@ public void testGroupByMultiValueMeasureQuery() ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows( @@ -1824,6 +1929,9 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) @@ -1831,7 +1939,7 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio if (DURABLE_STORAGE.equals(contextName) || FAULT_TOLERANCE.equals(contextName)) { new File( localFileStorageDir, - DurableStorageUtils.getSuccessFilePath("query-test-query", 0, 0) + DurableStorageUtils.getWorkerOutputSuccessFilePath("query-test-query", 0, 0) ); Mockito.verify(localFileStorageConnector, Mockito.times(2)) @@ -1840,70 +1948,7 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio } @Test - public void testSelectRowsGetTruncatedInReports() throws IOException - { - RowSignature dummyRowSignature = RowSignature.builder().add("timestamp", ColumnType.LONG).build(); - - final int numFiles = 200; - - final File toRead = MSQTestFileUtils.getResourceAsTemporaryFile(temporaryFolder, this, "/wikipedia-sampled.json"); - final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(toRead.getAbsolutePath()); - - String externalFiles = String.join(", ", Collections.nCopies(numFiles, toReadFileNameAsJson)); - - List result = new ArrayList<>(); - for (int i = 0; i < Limits.MAX_SELECT_RESULT_ROWS; ++i) { - result.add(new Object[]{1}); - } - - Map queryContext = new HashMap<>(context); - queryContext.put(MultiStageQueryContext.CTX_SELECT_DESTINATION, MSQSelectDestination.DURABLE_STORAGE.toString()); - - testSelectQuery() - .setSql(StringUtils.format( - " SELECT 1 as \"timestamp\"\n" - + "FROM TABLE(\n" - + " EXTERN(\n" - + " '{ \"files\": [%s],\"type\":\"local\"}',\n" - + " '{\"type\": \"csv\", \"hasHeaderRow\": true}',\n" - + " '[{\"name\": \"timestamp\", \"type\": \"string\"}]'\n" - + " )\n" - + ")", - externalFiles - )) - .setExpectedRowSignature(dummyRowSignature) - .setExpectedMSQSpec( - MSQSpec - .builder() - .query(newScanQueryBuilder() - .dataSource(new ExternalDataSource( - new LocalInputSource(null, null, Collections.nCopies(numFiles, toRead)), - new CsvInputFormat(null, null, null, true, 0), - RowSignature.builder().add("timestamp", ColumnType.STRING).build() - )) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("v0") - .virtualColumns(new ExpressionVirtualColumn("v0", ExprEval.of(1L).toExpr(), ColumnType.LONG)) - .context(defaultScanQueryContext( - queryContext, - RowSignature.builder().add("v0", ColumnType.LONG).build() - )) - .build() - ) - .columnMappings(new ColumnMappings( - ImmutableList.of( - new ColumnMapping("v0", "timestamp") - ) - )) - .tuningConfig(MSQTuningConfig.defaultConfig()) - .build()) - .setQueryContext(queryContext) - .setExpectedResultRows(result) - .verifyResults(); - } - - @Test - public void testSelectRowsGetUntruncatedInReportsByDefault() throws IOException + public void testSelectRowsGetUntruncatedByDefault() throws IOException { RowSignature dummyRowSignature = RowSignature.builder().add("timestamp", ColumnType.LONG).build(); @@ -1958,6 +2003,9 @@ public void testSelectRowsGetUntruncatedInReportsByDefault() throws IOException ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setQueryContext(context) .setExpectedResultRows(result) @@ -2008,4 +2056,9 @@ private static Map enableMultiValueUnnesting(Map .build(); return localContext; } + + public boolean isDurableStorageDestination() + { + return QUERY_RESULTS_WITH_DURABLE_STORAGE.equals(contextName) || QUERY_RESULTS_WITH_DEFAULT_CONTEXT.equals(context); + } } 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 index 5063c0547773..21e2d7ffaa3d 100644 --- 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 @@ -21,6 +21,7 @@ import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.junit.Test; public class DataSourceMSQDestinationTest diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java index 7f7bf4fbfd10..3da4adcbc3f5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java @@ -27,6 +27,8 @@ import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.duty.DutySchedule; +import org.apache.druid.msq.indexing.cleaner.DurableStorageCleaner; +import org.apache.druid.msq.indexing.cleaner.DurableStorageCleanerConfig; import org.apache.druid.storage.StorageConnector; import org.easymock.Capture; import org.easymock.EasyMock; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index 33e0d8c24869..6aaf21ee3bde 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.query.Druids; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java index b0dbacee2420..c88f22d1b74b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java @@ -29,6 +29,7 @@ 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.indexing.client.WorkerChatHandler; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java index c17c916a94f0..42ebf1dfc2aa 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.shuffle; import org.apache.druid.frame.processor.OutputChannelFactoryTest; +import org.apache.druid.msq.shuffle.output.DurableStorageTaskOutputChannelFactoryImpl; import org.apache.druid.storage.local.LocalFileStorageConnector; import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; @@ -35,7 +36,7 @@ public DurableStorageOutputChannelFactoryTest() throws IOException { super( - new DurableStorageOutputChannelFactory( + new DurableStorageTaskOutputChannelFactoryImpl( "0", 0, 0, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index 51e10a93b2d3..0b7732ceba54 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.sql; -import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.calcite.sql.type.SqlTypeName; @@ -41,6 +40,7 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlQuery; +import org.apache.druid.storage.local.LocalFileStorageConnector; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -55,21 +55,24 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase { private SqlStatementResource resource; + @Before - public void init() + public void init() throws IOException { resource = new SqlStatementResource( sqlStatementFactory, CalciteTests.TEST_AUTHORIZER_MAPPER, objectMapper, - indexingServiceClient + indexingServiceClient, + new LocalFileStorageConnector(tmpFolder.newFolder("results")) + ); } @Test public void testMSQSelectQueryTest() throws IOException { - List results = ImmutableList.of( + List results = ImmutableList.of( new Object[]{1L, ""}, new Object[]{ 1L, @@ -117,19 +120,16 @@ public void testMSQSelectQueryTest() throws IOException 316L, null, MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, - objectMapper.readValue( - objectMapper.writeValueAsString( - results), - new TypeReference>() - { - } - ), + results, ImmutableList.of(new PageInformation(6L, 316L, 0)) ), null ); - Assert.assertEquals(expected, response.getEntity()); + Assert.assertEquals( + objectMapper.writeValueAsString(expected), + objectMapper.writeValueAsString(response.getEntity()) + ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 6a19f3f792ef..1dba4f8bc73a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -40,13 +40,15 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.counters.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.guice.MSQIndexingModule; -import org.apache.druid.msq.indexing.DataSourceMSQDestination; 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.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQStagesReport; import org.apache.druid.msq.indexing.report.MSQStatusReport; @@ -74,6 +76,7 @@ import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlResourceTest; +import org.apache.druid.storage.local.LocalFileStorageConnector; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Before; @@ -86,16 +89,13 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; public class SqlStatementResourceTest extends MSQTestBase { - public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private static final String ACCEPTED_SELECT_MSQ_QUERY = "QUERY_ID_1"; @@ -239,10 +239,26 @@ public class SqlStatementResourceTest extends MSQTestBase MSQTaskReportTest.QUERY_DEFINITION, ImmutableMap.of(), ImmutableMap.of(), - ImmutableMap.of(), - ImmutableMap.of() + ImmutableMap.of(0, 1), + ImmutableMap.of(0, 1) + ), - new CounterSnapshotsTree(), + CounterSnapshotsTree.fromMap(ImmutableMap.of( + 0, + ImmutableMap.of( + 0, + new CounterSnapshots(ImmutableMap.of( + "output", + new ChannelCounters.Snapshot(new long[]{1L, 2L}, + new long[]{3L, 5L}, + new long[]{}, + new long[]{}, + new long[]{} + ) + ) + ) + ) + )), new MSQResultsReport( ImmutableList.of( new MSQResultsReport.ColumnAndType( @@ -627,8 +643,9 @@ public void init() throws Exception resource = new SqlStatementResource( sqlStatementFactory, CalciteTests.TEST_AUTHORIZER_MAPPER, - JSON_MAPPER, - overlordClient + objectMapper, + overlordClient, + new LocalFileStorageConnector(tmpFolder.newFolder("local")) ); } @@ -704,24 +721,22 @@ public void testFinishedSelectMSQQuery() throws Exception { Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(new SqlStatementResult( + Assert.assertEquals(objectMapper.writeValueAsString(new SqlStatementResult( FINISHED_SELECT_MSQ_QUERY, SqlStatementState.SUCCESS, CREATED_TIME, COL_NAME_AND_TYPES, 100L, new ResultSetInformation( - null, - null, + 3L, + 8L, null, MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, - RESULT_ROWS.stream() - .map(Arrays::asList) - .collect(Collectors.toList()), - ImmutableList.of(new PageInformation(null, null, 0L)) + RESULT_ROWS, + ImmutableList.of(new PageInformation(3L, 8L, 0L)) ), null - ), response.getEntity()); + )), objectMapper.writeValueAsString(response.getEntity())); Response resultsResponse = resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, 0L, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); @@ -793,12 +808,18 @@ public void testFinishedInsertMSQQuery() CREATED_TIME, null, 100L, - new ResultSetInformation(null, null, null, "test", null, ImmutableList.of(new PageInformation(null, null, 0))), + new ResultSetInformation(null, null, null, "test", null, null), null ), response.getEntity()); - Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, 0L, makeOkRequest()).getStatus()); - Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, makeOkRequest()).getStatus()); + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, 0L, makeOkRequest()).getStatus() + ); + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, makeOkRequest()).getStatus() + ); Assert.assertEquals( Response.Status.BAD_REQUEST.getStatusCode(), @@ -885,16 +906,28 @@ public void testMSQInsertRunningQuery() @Test public void forbiddenTests() { - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), - resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, - makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), - resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, - 1L, - makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), - resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, - makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); + Assert.assertEquals( + Response.Status.FORBIDDEN.getStatusCode(), + resource.doGetStatus( + RUNNING_SELECT_MSQ_QUERY, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT) + ).getStatus() + ); + Assert.assertEquals( + Response.Status.FORBIDDEN.getStatusCode(), + resource.doGetResults( + RUNNING_SELECT_MSQ_QUERY, + 1L, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT) + ).getStatus() + ); + Assert.assertEquals( + Response.Status.FORBIDDEN.getStatusCode(), + resource.deleteQuery( + RUNNING_SELECT_MSQ_QUERY, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT) + ).getStatus() + ); } @Test diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java index 8e40d8daf1a7..67b77fa51b33 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java @@ -32,18 +32,29 @@ public class ResultSetInformationTest public static final ObjectMapper MAPPER = new ObjectMapper(); public static final ResultSetInformation RESULTS = new ResultSetInformation( + 1L, + 1L, + ResultFormat.OBJECT, + "ds", + null, + ImmutableList.of(new PageInformation(1L, 1L, 0)) + ); + + + public static final ResultSetInformation RESULTS_1 = new ResultSetInformation( 1L, 1L, ResultFormat.OBJECT, "ds", ImmutableList.of( - ImmutableList.of("1"), - ImmutableList.of("2"), - ImmutableList.of("3") + new String[]{"1"}, + new String[]{"2"}, + new String[]{"3"} ), ImmutableList.of(new PageInformation(1L, 1L, 0)) ); - public static final String JSON_STRING = "{\"numTotalRows\":1,\"totalSizeInBytes\":1,\"resultFormat\":\"object\",\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]],\"pages\":[{\"numRows\":1,\"sizeInBytes\":1,\"id\":0}]}"; + public static final String JSON_STRING = "{\"numTotalRows\":1,\"totalSizeInBytes\":1,\"resultFormat\":\"object\",\"dataSource\":\"ds\",\"pages\":[{\"numRows\":1,\"sizeInBytes\":1,\"id\":0}]}"; + public static final String JSON_STRING_1 = "{\"numTotalRows\":1,\"totalSizeInBytes\":1,\"resultFormat\":\"object\",\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]],\"pages\":[{\"numRows\":1,\"sizeInBytes\":1,\"id\":0}]}"; @Test public void sanityTest() throws JsonProcessingException @@ -55,8 +66,16 @@ public void sanityTest() throws JsonProcessingException MAPPER.readValue(MAPPER.writeValueAsString(RESULTS), ResultSetInformation.class).hashCode() ); Assert.assertEquals( - "ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds', pages=[PageInformation{numRows=1, sizeInBytes=1, id=0}]}", + "ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=null, dataSource='ds', pages=[PageInformation{numRows=1, sizeInBytes=1, id=0}]}", RESULTS.toString() ); } + + @Test + public void resultsSanityTest() throws JsonProcessingException + { + // Since we have a List as a field, we cannot call equals method after deserialization. + Assert.assertEquals(JSON_STRING_1, MAPPER.writeValueAsString(RESULTS_1)); + } + } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java index 1409450c6972..88cff552a713 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java @@ -43,7 +43,7 @@ public class SqlStatementResultTest + "\"createdAt\":\"2023-05-31T12:00:00.000Z\"," + "\"schema\":[{\"name\":\"_time\",\"type\":\"TIMESTAMP\",\"nativeType\":\"LONG\"},{\"name\":\"alias\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"},{\"name\":\"market\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"}]," + "\"durationMs\":100," - + "\"result\":{\"numTotalRows\":1,\"totalSizeInBytes\":1,\"resultFormat\":\"object\",\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]],\"pages\":[{\"numRows\":1,\"sizeInBytes\":1,\"id\":0}]}," + + "\"result\":{\"numTotalRows\":1,\"totalSizeInBytes\":1,\"resultFormat\":\"object\",\"dataSource\":\"ds\",\"pages\":[{\"numRows\":1,\"sizeInBytes\":1,\"id\":0}]}," + "\"errorDetails\":{\"error\":\"druidException\",\"errorCode\":\"QueryNotSupported\",\"persona\":\"USER\",\"category\":\"UNCATEGORIZED\",\"errorMessage\":\"QueryNotSupported\",\"context\":{}}}"; public static final SqlStatementResult SQL_STATEMENT_RESULT = new SqlStatementResult( @@ -87,7 +87,7 @@ public void sanityTest() throws JsonProcessingException + " createdAt=2023-05-31T12:00:00.000Z," + " sqlRowSignature=[ColumnNameAndTypes{colName='_time', sqlTypeName='TIMESTAMP', nativeTypeName='LONG'}, ColumnNameAndTypes{colName='alias', sqlTypeName='VARCHAR', nativeTypeName='STRING'}, ColumnNameAndTypes{colName='market', sqlTypeName='VARCHAR', nativeTypeName='STRING'}]," + " durationInMs=100," - + " resultSetInformation=ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds', pages=[PageInformation{numRows=1, sizeInBytes=1, id=0}]}," + + " resultSetInformation=ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=null, dataSource='ds', pages=[PageInformation{numRows=1, sizeInBytes=1, id=0}]}," + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}}", SQL_STATEMENT_RESULT.toString() ); 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 index 736ec2f430da..74d0021de9dd 100644 --- 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 @@ -43,6 +43,7 @@ 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.channel.FrameChannelSequence; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.DruidSecondaryModule; @@ -70,6 +71,7 @@ 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.metadata.input.InputSourceModule; @@ -85,9 +87,12 @@ 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.InputChannelFactory; +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.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.error.InsertLockPreemptedFaultTest; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQFault; @@ -97,10 +102,13 @@ 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.kernel.StageDefinition; import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; 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.msq.util.SqlStatementResourceHelper; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryContexts; @@ -190,6 +198,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -1292,21 +1301,52 @@ public Pair, List>> verifyCounters(payload.getCounters()); verifyWorkerCount(payload.getCounters()); + if (payload.getStatus().getErrorReport() != null) { throw new ISE("Query %s failed due to %s", sql, payload.getStatus().getErrorReport().toString()); } else { - final List rows = getRows(payload.getResults()); + MSQControllerTask msqControllerTask = indexingServiceClient.getMSQControllerTask(controllerId); + + final MSQSpec spec = msqControllerTask.getQuerySpec(); + final List rows; + + if (spec.getDestination() instanceof TaskReportMSQDestination) { + rows = getRows(payload.getResults()); + } else { + StageDefinition finalStage = Objects.requireNonNull(SqlStatementResourceHelper.getFinalStage( + payload)).getStageDefinition(); + InputChannelFactory inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation( + controllerId, + localFileStorageConnector, + Closer.create(), + true + ); + rows = new FrameChannelSequence(inputChannelFactory.openChannel( + finalStage.getId(), + 0, + 0 + )).flatMap(frame -> SqlStatementResourceHelper.getResultSequence( + msqControllerTask, + finalStage, + frame, + objectMapper + )).toList(); + + } if (rows == null) { throw new ISE("Query successful but no results found"); } log.info("found row signature %s", payload.getResults().getSignature()); log.info(rows.stream().map(Arrays::toString).collect(Collectors.joining("\n"))); - final MSQSpec spec = indexingServiceClient.getMSQControllerTask(controllerId).getQuerySpec(); + log.info("Found spec: %s", objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(spec)); return new Pair<>(spec, Pair.of(payload.getResults().getSignature(), rows)); } } + catch (JsonProcessingException ex) { + throw new RuntimeException(ex); + } catch (Exception e) { if (expectedExecutionErrorMatcher == null) { throw new ISE(e, "Query %s failed", sql); 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 index 567e6d8a1f71..16aad3d7b93a 100644 --- 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 @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.msq.indexing.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.query.BadQueryContextException; import org.apache.druid.query.QueryContext; diff --git a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java index 1585b865157f..a20bef6691f7 100644 --- a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java @@ -43,7 +43,18 @@ public static String getControllerDirectory(final String controllerTaskId) return StringUtils.format("controller_%s", IdUtils.validateId("controller task ID", controllerTaskId)); } - public static String getSuccessFilePath( + private static String getQueryResultsControllerDirectory(final String controllerTaskId) + { + return StringUtils.format( + "%s/controller_%s", + QUERY_RESULTS_DIR, + IdUtils.validateId("controller task ID", controllerTaskId) + ); + } + + + + public static String getWorkerOutputSuccessFilePath( final String controllerTaskId, final int stageNumber, final int workerNumber @@ -57,10 +68,24 @@ public static String getSuccessFilePath( return StringUtils.format("%s/%s", folderName, SUCCESS_MARKER_FILENAME); } + public static String getQueryResultsSuccessFilePath( + final String controllerTaskId, + final int stageNumber, + final int workerNumber + ) + { + String folderName = getQueryResultsWorkerOutputFolderName( + controllerTaskId, + stageNumber, + workerNumber + ); + return StringUtils.format("%s/%s", folderName, SUCCESS_MARKER_FILENAME); + } + /** * Fetches the directory location where workers will store the partition files corresponding to the stage number */ - public static String getWorkerOutputFolderName( + private static String getWorkerOutputFolderName( final String controllerTaskId, final int stageNumber, final int workerNumber @@ -74,6 +99,21 @@ public static String getWorkerOutputFolderName( ); } + + private static String getQueryResultsWorkerOutputFolderName( + final String controllerTaskId, + final int stageNumber, + final int workerNumber + ) + { + return StringUtils.format( + "%s/stage_%d/worker_%d", + getQueryResultsControllerDirectory(controllerTaskId), + stageNumber, + workerNumber + ); + } + /** * Fetches the directory location where a particular worker will store the partition files corresponding to the * stage number, and it's task id @@ -92,6 +132,20 @@ public static String getTaskIdOutputsFolderName( ); } + + public static String getQueryResultsForTaskIdFolderName( + final String controllerTaskId, + final int stageNumber, + final int workerNumber, + final String taskId + ) + { + return StringUtils.format( + "%s/taskId_%s", + getQueryResultsWorkerOutputFolderName(controllerTaskId, stageNumber, workerNumber), + taskId + ); + } /** * Fetches the file location where a particular worker writes the data corresponding to a particular stage * and partition @@ -111,6 +165,21 @@ public static String getPartitionOutputsFileNameForPartition( ); } + public static String getQueryResultsFileNameForPartition( + final String controllerTaskId, + final int stageNumber, + final int workerNumber, + final String taskId, + final int partitionNumber + ) + { + return StringUtils.format( + "%s/part_%d", + getQueryResultsForTaskIdFolderName(controllerTaskId, stageNumber, workerNumber, taskId), + partitionNumber + ); + } + /** * Fetches the file location where a particular worker writes the data corresponding to a particular stage * and a custom path name diff --git a/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java new file mode 100644 index 000000000000..a2c93fd07227 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.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.storage; + +import org.apache.druid.java.util.common.UOE; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Iterator; + +public class NilStorageConnector implements StorageConnector +{ + private static final NilStorageConnector NIL_STORAGE_CONNECTOR = new NilStorageConnector(); + + private NilStorageConnector() + { + + } + + public static NilStorageConnector getInstance() + { + return NIL_STORAGE_CONNECTOR; + } + + @Override + public boolean pathExists(String path) throws IOException + { + throw new UOE("Please configure durable storage."); + } + + @Override + public InputStream read(String path) throws IOException + { + throw new UOE("Please configure durable storage."); + } + + @Override + public InputStream readRange(String path, long from, long size) + { + throw new UOE("Please configure durable storage."); + } + + @Override + public OutputStream write(String path) + { + throw new UOE("Please configure durable storage."); + } + + @Override + public void deleteFile(String path) + { + throw new UOE("Please configure durable storage."); + } + + @Override + public void deleteFiles(Iterable paths) + { + throw new UOE("Please configure durable storage."); + } + + @Override + public void deleteRecursively(String path) + { + throw new UOE("Please configure durable storage."); + } + + @Override + public Iterator listDir(String dirName) + { + throw new UOE("Please configure durable storage."); + } +} From 482cccd957e1fe9d2f641333b3423f4a849f9ef6 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 5 Jul 2023 20:32:58 +0530 Subject: [PATCH 02/15] Removing forbidden API's --- .../apache/druid/msq/sql/resources/SqlStatementResource.java | 1 - .../org/apache/druid/msq/util/MultiStageQueryContext.java | 5 +++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index b8e6e8bcc603..094465f88720 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -529,7 +529,6 @@ private static void throwIfQueryIsNotSuccessful(String queryId, TaskStatusPlus s ); } else { // do nothing - System.out.println("abc"); } } 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 index dda2dfd7f11b..ca364be4c732 100644 --- 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 @@ -39,6 +39,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -217,7 +218,7 @@ public static MSQSelectDestination getSelectDestination(final QueryContext query queryContext.getString( CTX_SELECT_DESTINATION, DEFAULT_SELECT_DESTINATION - ).toUpperCase() + ).toUpperCase(Locale.ENGLISH) ); } @@ -228,7 +229,7 @@ public static MSQSelectDestination getSelectDestinationOrNull(final QueryContext if (selectDestination == null) { return null; } - return MSQSelectDestination.valueOf(selectDestination.toUpperCase()); + return MSQSelectDestination.valueOf(selectDestination.toUpperCase(Locale.ENGLISH)); } public static int getRowsInMemory(final QueryContext queryContext) From 43e71bd5041562bbea498d631baaf287f6248589 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 5 Jul 2023 21:49:45 +0530 Subject: [PATCH 03/15] Attaching result stage always. --- .../apache/druid/msq/exec/ControllerImpl.java | 32 ++++++------------- .../QueryResultFrameProcessorFactory.java | 32 ++++++++++++++++--- 2 files changed, 38 insertions(+), 26 deletions(-) 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 index 427d1ab6b9c7..290e7b8de2ef 100644 --- 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 @@ -1539,7 +1539,8 @@ private static QueryDefinition makeQueryDefinition( shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); queryToPlan = querySpec.getQuery(); } else if (querySpec.getDestination() instanceof DurableStorageDestination) { - shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); + // we add a final stage which generates one partition per worker. + shuffleSpecFactory = ShuffleSpecFactories.globalSortWithMaxPartitionCount(tuningConfig.getMaxNumWorkers()); queryToPlan = querySpec.getQuery(); } else { throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); @@ -1616,36 +1617,23 @@ private static QueryDefinition makeQueryDefinition( } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { return queryDef; } else if (querySpec.getDestination() instanceof DurableStorageDestination) { + + // attaching new query results stage always. StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition(); - if (!finalShuffleStageDef.doesSortDuringShuffle()) { - return queryDef; - } - // attaching new query results stage final QueryDefinitionBuilder builder = QueryDefinition.builder(); for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { builder.add(StageDefinition.builder(stageDef)); } - - builder.add( - StageDefinition.builder(queryDef.getNextStageNumber()) - .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) - .maxWorkerCount(tuningConfig.getMaxNumWorkers()) - .signature(finalShuffleStageDef.getSignature()) - .shuffleSpec(finalShuffleStageDef.getClusterBy().isEmpty() - ? null - : ShuffleSpecFactories.singlePartition() - .build(finalShuffleStageDef.getClusterBy(), false)) - .shuffleCheckHasMultipleValues(finalShuffleStageDef.getShuffleCheckHasMultipleValues()) - .processorFactory( - new QueryResultFrameProcessorFactory( - ) - ) + builder.add(StageDefinition.builder(queryDef.getNextStageNumber()) + .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) + .maxWorkerCount(tuningConfig.getMaxNumWorkers()) + .signature(finalShuffleStageDef.getSignature()) + .shuffleSpec(null) + .processorFactory(new QueryResultFrameProcessorFactory()) ); return builder.build(); - - } else { throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java index 20e2a20fa683..33319eb2cb5d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; @@ -33,6 +35,7 @@ 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; @@ -74,19 +77,40 @@ public ProcessorsAndChannels, Long> makeProcessors( return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); } + final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); - final OutputChannel outputChannel = outputChannelFactory.openChannel(0); + 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 -> new QueryResultsFrameProcessor(readableInput.getChannel(), - outputChannel.getWritableChannel())); + readableInput -> { + final OutputChannel outputChannel = + outputChannels.get(readableInput.getStagePartition().getPartitionNumber()); + + return new QueryResultsFrameProcessor( + readableInput.getChannel(), + outputChannel.getWritableChannel() + ); + } + ); return new ProcessorsAndChannels<>( processors, - OutputChannels.wrapReadOnly(ImmutableList.of(outputChannel)) + OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values())) ); } From e547230036d865402ffaf9e8194aa7a6f5e4edaf Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 5 Jul 2023 22:20:26 +0530 Subject: [PATCH 04/15] CI things --- .../test/java/org/apache/druid/msq/exec/MSQSelectTest.java | 5 +++-- .../java/org/apache/druid/storage/NilStorageConnector.java | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) 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 index 2dbfbec697b4..db6bbd71db5c 100644 --- 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 @@ -94,6 +94,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.Map; @RunWith(Parameterized.class) @@ -109,7 +110,7 @@ public class MSQSelectTest extends MSQTestBase .putAll(DURABLE_STORAGE_MSQ_CONTEXT) .put( MultiStageQueryContext.CTX_SELECT_DESTINATION, - MSQSelectDestination.DURABLE_STORAGE.name().toLowerCase() + MSQSelectDestination.DURABLE_STORAGE.name().toLowerCase(Locale.ENGLISH) ) .build(); @@ -119,7 +120,7 @@ public class MSQSelectTest extends MSQTestBase .putAll(DEFAULT_MSQ_CONTEXT) .put( MultiStageQueryContext.CTX_SELECT_DESTINATION, - MSQSelectDestination.DURABLE_STORAGE.name().toLowerCase() + MSQSelectDestination.DURABLE_STORAGE.name().toLowerCase(Locale.ENGLISH) ) .build(); diff --git a/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java index a2c93fd07227..533f0ebac769 100644 --- a/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java @@ -41,13 +41,13 @@ public static NilStorageConnector getInstance() } @Override - public boolean pathExists(String path) throws IOException + public boolean pathExists(String path) { throw new UOE("Please configure durable storage."); } @Override - public InputStream read(String path) throws IOException + public InputStream read(String path) { throw new UOE("Please configure durable storage."); } From 855e77696e869834bfda4f5373c131ed62e80a30 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 5 Jul 2023 22:21:26 +0530 Subject: [PATCH 05/15] CI things --- .../main/java/org/apache/druid/storage/NilStorageConnector.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java index 533f0ebac769..6e2dbb3bf28c 100644 --- a/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java @@ -21,7 +21,6 @@ import org.apache.druid.java.util.common.UOE; -import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Iterator; From 4e80ab5b629b6ae138adfd10f32f42703f7a0aa1 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 6 Jul 2023 12:01:23 +0530 Subject: [PATCH 06/15] Adding coverage for processing module. --- .../druid/frame/util/DurableStorageUtils.java | 1 - .../frame/util/DurableStorageUtilsTest.java | 76 ++++++++++++++++++- .../storage/NilStorageConnectorTest.java | 47 ++++++++++++ 3 files changed, 121 insertions(+), 3 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java diff --git a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java index a20bef6691f7..738f3de0809e 100644 --- a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java @@ -53,7 +53,6 @@ private static String getQueryResultsControllerDirectory(final String controller } - public static String getWorkerOutputSuccessFilePath( final String controllerTaskId, final int stageNumber, diff --git a/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java b/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java index 71b16633e921..093eeca7c46b 100644 --- a/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java @@ -26,13 +26,25 @@ public class DurableStorageUtilsTest { + private static final String CONTROLLER_ID = "controller_id_1"; + private static final String TASK_ID = "task_id_1"; + + private static final int WORKER_NUMBER = 2; + + private static final int STAGE_NUMBER = 1; + + private static final int PARTITION_NUMBER = 3; + + @Test public void getNextDirNameWithPrefixFromPath() { Assert.assertEquals("", DurableStorageUtils.getNextDirNameWithPrefixFromPath("/123/123")); Assert.assertEquals("123", DurableStorageUtils.getNextDirNameWithPrefixFromPath("123")); - Assert.assertEquals("controller_query_123", - DurableStorageUtils.getNextDirNameWithPrefixFromPath("controller_query_123/123")); + Assert.assertEquals( + "controller_query_123", + DurableStorageUtils.getNextDirNameWithPrefixFromPath("controller_query_123/123") + ); Assert.assertEquals("", DurableStorageUtils.getNextDirNameWithPrefixFromPath("")); Assert.assertNull(DurableStorageUtils.getNextDirNameWithPrefixFromPath(null)); } @@ -62,4 +74,64 @@ public void isQueryResultFileActive() ImmutableSet.of("123") )); } + + @Test + public void sanityTest() + { + + String baseString = "controller_" + CONTROLLER_ID + "/stage_" + STAGE_NUMBER + "/worker_" + WORKER_NUMBER + "/"; + + Assert.assertEquals( + baseString + "__success", + DurableStorageUtils.getWorkerOutputSuccessFilePath(CONTROLLER_ID, STAGE_NUMBER, WORKER_NUMBER) + ); + Assert.assertEquals( + DurableStorageUtils.QUERY_RESULTS_DIR + "/" + baseString + "__success", + DurableStorageUtils.getQueryResultsSuccessFilePath(CONTROLLER_ID, STAGE_NUMBER, WORKER_NUMBER) + ); + + + Assert.assertEquals( + baseString + "taskId_" + TASK_ID, + DurableStorageUtils.getTaskIdOutputsFolderName( + CONTROLLER_ID, + STAGE_NUMBER, + WORKER_NUMBER, + TASK_ID + ) + ); + Assert.assertEquals( + DurableStorageUtils.QUERY_RESULTS_DIR + "/" + baseString + "taskId_" + TASK_ID, + DurableStorageUtils.getQueryResultsForTaskIdFolderName( + CONTROLLER_ID, + STAGE_NUMBER, + WORKER_NUMBER, + TASK_ID + ) + ); + + + Assert.assertEquals( + baseString + "taskId_" + TASK_ID + "/part_3", + DurableStorageUtils.getPartitionOutputsFileNameForPartition( + CONTROLLER_ID, + STAGE_NUMBER, + WORKER_NUMBER, + TASK_ID, + PARTITION_NUMBER + ) + ); + Assert.assertEquals( + DurableStorageUtils.QUERY_RESULTS_DIR + "/" + baseString + "taskId_" + TASK_ID + "/part_3", + DurableStorageUtils.getQueryResultsFileNameForPartition( + CONTROLLER_ID, + STAGE_NUMBER, + WORKER_NUMBER, + TASK_ID, + PARTITION_NUMBER + ) + ); + + } + } diff --git a/processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java b/processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java new file mode 100644 index 000000000000..894f26cf2208 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.storage; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.UOE; +import org.junit.Assert; +import org.junit.Test; + +public class NilStorageConnectorTest +{ + + private static String ERROR_MESSAGE = "Please configure durable storage."; + + + @Test + public void sanity() + { + NilStorageConnector nilStorageConnector = NilStorageConnector.getInstance(); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.pathExists("null")); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.read("null")); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.readRange("null", 0, 0)); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.deleteFile("null")); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.deleteFiles(ImmutableList.of())); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.deleteRecursively("null")); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.listDir("null")); + Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.pathExists("null")); + } + +} From 4a2dd36c4f6d47c19771d4db6b7baf04bc49abb3 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 6 Jul 2023 12:12:55 +0530 Subject: [PATCH 07/15] Intellij inspections --- .../msq/querykit/results/QueryResultFrameProcessorFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java index 33319eb2cb5d..30c20fe22dad 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java @@ -68,7 +68,7 @@ public ProcessorsAndChannels, Long> makeProcessors( int maxOutstandingProcessors, CounterTracker counters, Consumer warningPublisher - ) throws IOException + ) { // Expecting a single input slice from some prior stage. final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); From 7c8509bcaede4e765cf4c4065da8405b0167e60c Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 6 Jul 2023 21:29:36 +0530 Subject: [PATCH 08/15] 1. Cleaning up code. 2. Addressing review comments. --- .../apache/druid/msq/exec/ControllerImpl.java | 8 +- .../org/apache/druid/msq/exec/WorkerImpl.java | 6 +- .../msq/guice/MSQDurableStorageModule.java | 6 +- .../druid/msq/indexing/MSQControllerTask.java | 5 +- .../apache/druid/msq/indexing/MSQSpec.java | 1 + .../destination/DataSourceMSQDestination.java | 1 - ...java => DurableStorageMSQDestination.java} | 9 +- .../{ => destination}/MSQDestination.java | 7 +- .../destination/TaskReportMSQDestination.java | 1 - .../DurableStorageOutputChannelFactory.java | 8 +- ...rageQueryResultsOutputChannelFactory.java} | 8 +- ...rableStorageTaskOutputChannelFactory.java} | 6 +- .../druid/msq/sql/MSQTaskQueryMaker.java | 11 +- .../sql/resources/SqlStatementResource.java | 428 +++++++++--------- .../msq/util/SqlStatementResourceHelper.java | 106 +---- .../apache/druid/msq/exec/MSQSelectTest.java | 58 +-- .../QueryResultsFrameProcessorTest.java | 115 +++++ ...urableStorageOutputChannelFactoryTest.java | 4 +- .../apache/druid/msq/test/MSQTestBase.java | 5 +- .../druid/storage/NilStorageConnector.java | 27 +- .../storage/NilStorageConnectorTest.java | 18 +- 21 files changed, 451 insertions(+), 387 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/{DurableStorageDestination.java => DurableStorageMSQDestination.java} (79%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{ => destination}/MSQDestination.java (78%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/{DurableStorageResultsOutputChannelFactory.java => DurableStorageQueryResultsOutputChannelFactory.java} (92%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/{DurableStorageTaskOutputChannelFactoryImpl.java => DurableStorageTaskOutputChannelFactory.java} (98%) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java 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 index 290e7b8de2ef..f5f7b0cc1aa3 100644 --- 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 @@ -94,7 +94,7 @@ import org.apache.druid.msq.indexing.WorkerCount; import org.apache.druid.msq.indexing.client.ControllerChatHandler; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.error.CanceledFault; @@ -1538,7 +1538,7 @@ private static QueryDefinition makeQueryDefinition( } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); queryToPlan = querySpec.getQuery(); - } else if (querySpec.getDestination() instanceof DurableStorageDestination) { + } else if (querySpec.getDestination() instanceof DurableStorageMSQDestination) { // we add a final stage which generates one partition per worker. shuffleSpecFactory = ShuffleSpecFactories.globalSortWithMaxPartitionCount(tuningConfig.getMaxNumWorkers()); queryToPlan = querySpec.getQuery(); @@ -1616,7 +1616,7 @@ private static QueryDefinition makeQueryDefinition( return builder.build(); } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { return queryDef; - } else if (querySpec.getDestination() instanceof DurableStorageDestination) { + } else if (querySpec.getDestination() instanceof DurableStorageMSQDestination) { // attaching new query results stage always. StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition(); @@ -1744,7 +1744,7 @@ private static boolean isRollupQuery(Query query) private static boolean isInlineResults(final MSQSpec querySpec) { return querySpec.getDestination() instanceof TaskReportMSQDestination - || querySpec.getDestination() instanceof DurableStorageDestination; + || querySpec.getDestination() instanceof DurableStorageMSQDestination; } private static boolean isTimeBucketedIngestion(final MSQSpec querySpec) 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 index ed48f3d3c6c4..942d8b44b0c9 100644 --- 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 @@ -1342,11 +1342,7 @@ private void writeDurableStorageSuccessFileIfNeeded(final int stageNumber, boole e, "Unable to create the success file [%s] at the location [%s]", DurableStorageUtils.SUCCESS_MARKER_FILENAME, - DurableStorageUtils.getWorkerOutputSuccessFilePath( - task.getControllerTaskId(), - stageNumber, - task().getWorkerNumber() - ) + durableStorageOutputChannelFactory.getSuccessFilePath() ); } } 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 index 6e0266fe892a..7139377495a8 100644 --- 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 @@ -99,9 +99,11 @@ public void configure(Binder binder) .addBinding() .to(DurableStorageCleaner.class); } - } else { - // bind with nil implementation so that configs are not required during service startups. + } else if (nodeRoles.contains(NodeRole.BROKER)) { + // bind with nil implementation so that configs are not required during service startups of broker since SQLStatementResource uses it. binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class)).toInstance(NilStorageConnector.getInstance()); + } else { + // do nothing } } 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 index 0b6fdffcc37f..c39b7637dd66 100644 --- 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 @@ -48,7 +48,8 @@ import org.apache.druid.msq.exec.ControllerImpl; import org.apache.druid.msq.exec.MSQTasks; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; +import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; @@ -268,6 +269,6 @@ public static boolean isIngestion(final MSQSpec querySpec) public static boolean writeResultsToDurableStorage(final MSQSpec querySpec) { - return querySpec.getDestination() instanceof DurableStorageDestination; + return querySpec.getDestination() instanceof DurableStorageMSQDestination; } } 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 index caa182b8e72b..065471d32bae 100644 --- 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 @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.query.Query; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java index 04c203941f49..5bf033a1aa03 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java @@ -26,7 +26,6 @@ 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.apache.druid.msq.indexing.MSQDestination; import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java similarity index 79% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageDestination.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java index 4cf1bbe24997..c401d2eee6e4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java @@ -20,22 +20,21 @@ package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.druid.msq.indexing.MSQDestination; -public class DurableStorageDestination implements MSQDestination +public class DurableStorageMSQDestination implements MSQDestination { public static final String TYPE = "durableStorage"; - public static final DurableStorageDestination INSTANCE = new DurableStorageDestination(); + public static final DurableStorageMSQDestination INSTANCE = new DurableStorageMSQDestination(); - private DurableStorageDestination() + private DurableStorageMSQDestination() { // Singleton. } @JsonCreator - public static DurableStorageDestination instance() + public static DurableStorageMSQDestination instance() { return INSTANCE; } 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/destination/MSQDestination.java similarity index 78% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQDestination.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQDestination.java index ddf703e50de4..52489d15a343 100644 --- 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/destination/MSQDestination.java @@ -17,19 +17,16 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.DurableStorageDestination; -import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; @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), - @JsonSubTypes.Type(name = DurableStorageDestination.TYPE, value = DurableStorageDestination.class) + @JsonSubTypes.Type(name = DurableStorageMSQDestination.TYPE, value = DurableStorageMSQDestination.class) }) public interface MSQDestination { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java index 30355e4dde17..ac1254abfe33 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.druid.msq.indexing.MSQDestination; public class TaskReportMSQDestination implements MSQDestination { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java index a02993c03acc..abfde486789b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java @@ -39,7 +39,7 @@ public abstract class DurableStorageOutputChannelFactory implements OutputChannelFactory { - private static final Logger LOG = new Logger(DurableStorageTaskOutputChannelFactoryImpl.class); + private static final Logger LOG = new Logger(DurableStorageOutputChannelFactory.class); protected final String controllerTaskId; protected final int workerNumber; protected final int stageNumber; @@ -86,7 +86,7 @@ public static DurableStorageOutputChannelFactory createStandardImplementation( ) { if (isQueryResults) { - return new DurableStorageResultsOutputChannelFactory( + return new DurableStorageQueryResultsOutputChannelFactory( controllerTaskId, workerNumber, stageNumber, @@ -96,7 +96,7 @@ public static DurableStorageOutputChannelFactory createStandardImplementation( tmpDir ); } else { - return new DurableStorageTaskOutputChannelFactoryImpl( + return new DurableStorageTaskOutputChannelFactory( controllerTaskId, workerNumber, stageNumber, @@ -128,7 +128,7 @@ public void createSuccessFile(String taskId) throws IOException os.close(); } - protected abstract String getSuccessFilePath(); + public abstract String getSuccessFilePath(); @Override public OutputChannel openNilChannel(int partitionNumber) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageResultsOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java similarity index 92% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageResultsOutputChannelFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java index b058d0f69840..c1baa1d54c96 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageResultsOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java @@ -34,11 +34,11 @@ import java.io.IOException; import java.nio.channels.Channels; -public class DurableStorageResultsOutputChannelFactory extends DurableStorageOutputChannelFactory +public class DurableStorageQueryResultsOutputChannelFactory extends DurableStorageOutputChannelFactory { - public DurableStorageResultsOutputChannelFactory( + public DurableStorageQueryResultsOutputChannelFactory( String controllerTaskId, int workerNumber, int stageNumber, @@ -52,7 +52,7 @@ public DurableStorageResultsOutputChannelFactory( } @Override - protected String getSuccessFilePath() + public String getSuccessFilePath() { return DurableStorageUtils.getQueryResultsSuccessFilePath( controllerTaskId, @@ -96,7 +96,7 @@ public OutputChannel openChannel(int partitionNumber) throws IOException @Override public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) { - throw new UOE("%s does not support this call", DurableStorageResultsOutputChannelFactory.class.getSimpleName()); + throw new UOE("%s does not support this call", DurableStorageQueryResultsOutputChannelFactory.class.getSimpleName()); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactoryImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java similarity index 98% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactoryImpl.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java index fbe3adf4eb48..c5bc83ffc8ee 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactoryImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java @@ -51,10 +51,10 @@ import java.nio.channels.Channels; import java.util.function.Supplier; -public class DurableStorageTaskOutputChannelFactoryImpl +public class DurableStorageTaskOutputChannelFactory extends DurableStorageOutputChannelFactory { - public DurableStorageTaskOutputChannelFactoryImpl( + public DurableStorageTaskOutputChannelFactory( String controllerTaskId, int workerNumber, int stageNumber, @@ -68,7 +68,7 @@ public DurableStorageTaskOutputChannelFactoryImpl( } @Override - protected String getSuccessFilePath() + public String getSuccessFilePath() { return DurableStorageUtils.getWorkerOutputSuccessFilePath(controllerTaskId, stageNumber, workerNumber); } 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 index 8b1a8e55c7e1..a5dc579419a9 100644 --- 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 @@ -35,11 +35,11 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.msq.exec.MSQTasks; 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.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; +import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.util.MSQTaskQueryMakerUtils; @@ -231,10 +231,13 @@ public QueryResponse runQuery(final DruidQuery druidQuery) if (ctxDestination != null && !DESTINATION_REPORT.equals(ctxDestination)) { throw new IAE("Cannot SELECT with destination [%s]", ctxDestination); } - if (MultiStageQueryContext.getSelectDestination(sqlQueryContext).equals(MSQSelectDestination.TASK_REPORT)) { + final MSQSelectDestination msqSelectDestination = MultiStageQueryContext.getSelectDestination(sqlQueryContext); + if (msqSelectDestination.equals(MSQSelectDestination.TASK_REPORT)) { destination = TaskReportMSQDestination.instance(); + } else if (msqSelectDestination.equals(MSQSelectDestination.DURABLE_STORAGE)) { + destination = DurableStorageMSQDestination.instance(); } else { - destination = DurableStorageDestination.instance(); + throw new IAE("Cannot SELECT with destination [%s]", msqSelectDestination.name()); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 094465f88720..fa7bd38d81c0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -45,9 +45,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.guice.MultiStageQuery; import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.MSQDestination; -import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; +import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; @@ -325,102 +324,8 @@ public Response doGetResults( // returning results final Closer closer = Closer.create(); - final Optional> results; - if (msqControllerTask.getQuerySpec().getDestination() instanceof TaskReportMSQDestination) { - if (page != null && page > 0) { - // Results from task report are only present as one page. - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build("Page number is out of range of the results."); - } - - MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(SqlStatementResourceHelper.getPayload( - contactOverlord(overlordClient.taskReportAsMap(queryId)) - ), MSQTaskReportPayload.class); - - if (msqTaskReportPayload.getResults().getResultYielder() == null) { - results = Optional.empty(); - } else { - results = Optional.of(msqTaskReportPayload.getResults().getResultYielder()); - } - - } else if (msqControllerTask.getQuerySpec().getDestination() instanceof DurableStorageDestination) { - - MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(SqlStatementResourceHelper.getPayload( - contactOverlord(overlordClient.taskReportAsMap(queryId)) - ), MSQTaskReportPayload.class); - - - List pages = SqlStatementResourceHelper.populatePageList( - msqTaskReportPayload, - msqControllerTask.getQuerySpec() - .getDestination() - ) - .get(); - - - final StageDefinition finalStage = Objects.requireNonNull(SqlStatementResourceHelper.getFinalStage( - msqTaskReportPayload)).getStageDefinition(); - - if (page == null) { - final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( - msqControllerTask.getId(), - storageConnector, - closer, - true - ); - results = Optional.of(Yielders.each( - Sequences.concat(pages.stream().map(pageInformation -> { - try { - return new FrameChannelSequence( - standardImplementation.openChannel(finalStage.getId(), - (int) pageInformation.getId(), 0 - ) - ); - } - catch (Exception e) { - throw new RuntimeException( - e); - } - }).collect(Collectors.toList())).flatMap( - frame -> SqlStatementResourceHelper.getResultSequence( - msqControllerTask, - finalStage, - frame, - jsonMapper - ) - - ))); - - } else { - PageInformation pageInformation = getPageInformationForPageId(pages, page); - final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( - msqControllerTask.getId(), - storageConnector, - closer, - true - ); - results = Optional.of(Yielders.each(new FrameChannelSequence(standardImplementation.openChannel( - finalStage.getId(), - (int) pageInformation.getId(), - 0 - )).flatMap(frame -> SqlStatementResourceHelper.getResultSequence( - msqControllerTask, - finalStage, - frame, - jsonMapper - )))); - - } - } else { - throw DruidException.forPersona(DruidException.Persona.DEVELOPER) - .ofCategory(DruidException.Category.UNCATEGORIZED) - .build( - "MSQ select destination[%s] not supported. Please reach out to druid slack community for more help.", - msqControllerTask.getQuerySpec().getDestination().toString() - ); - } + results = getResultYielder(queryId, page, msqControllerTask, closer); if (!results.isPresent()) { // no results, return empty return Response.ok().build(); @@ -455,83 +360,6 @@ public Response doGetResults( } } - private PageInformation getPageInformationForPageId(List pages, Long pageId) - { - for (PageInformation pageInfo : pages) { - if (pageInfo.getId() == pageId) { - return pageInfo; - } - } - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build("Invalid page id [%d] passed.", pageId); - } - - private void resultPusher( - String queryId, - Optional> signature, - Closer closer, - Optional> results, - CountingOutputStream os - ) throws IOException - { - try { - try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { - Yielder yielder = results.get(); - List rowSignature = signature.get(); - writer.writeResponseStart(); - - while (!yielder.isDone()) { - writer.writeRowStart(); - Object[] row = yielder.get(); - for (int i = 0; i < Math.min(rowSignature.size(), row.length); i++) { - writer.writeRowField( - rowSignature.get(i).getColName(), - row[i] - ); - } - writer.writeRowEnd(); - yielder = yielder.next(null); - } - writer.writeResponseEnd(); - } - catch (Exception e) { - log.error(e, "Unable to stream results back for query[%s]", queryId); - throw new ISE(e, "Unable to stream results back for query[%s]", queryId); - } - } - catch (Exception e) { - log.error(e, "Unable to stream results back for query[%s]", queryId); - throw new ISE(e, "Unable to stream results back for query[%s]", queryId); - } - finally { - closer.close(); - } - } - - private static void throwIfQueryIsNotSuccessful(String queryId, TaskStatusPlus statusPlus) - { - SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); - if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "Query[%s] is currently in [%s] state. Please wait for it to complete.", - queryId, - sqlStatementState - ); - } else if (sqlStatementState == SqlStatementState.FAILED) { - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "Query[%s] failed. Hit status api for more details.", - queryId - ); - } else { - // do nothing - } - } - /** * Queries can be canceled while in any {@link SqlStatementState}. Canceling a query that has already completed will be a no-op. * @@ -701,7 +529,7 @@ private Response buildNonOkResponse(DruidException exception) @SuppressWarnings("ReassignedVariable") private Optional getSampleResults( - String asyncResultId, + String queryId, String dataSource, SqlStatementState sqlStatementState, MSQDestination msqDestination @@ -709,60 +537,60 @@ private Optional getSampleResults( { if (sqlStatementState == SqlStatementState.SUCCESS) { Map payload = SqlStatementResourceHelper.getPayload(contactOverlord(overlordClient.taskReportAsMap( - asyncResultId))); + queryId))); MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(payload, MSQTaskReportPayload.class); - - Optional> pageList = SqlStatementResourceHelper.populatePageList( msqTaskReportPayload, msqDestination ); - Long rows = null; - Long size = null; - boolean isSelectQuery = false; + + // getting the total number of rows, size from page information. + Long rows = 0L; + Long size = 0L; if (pageList.isPresent()) { - if (msqDestination instanceof TaskReportMSQDestination) { - isSelectQuery = true; - if (pageList.get().size() == 1) { - rows = pageList.get().get(0).getNumRows(); - size = pageList.get().get(0).getSizeInBytes(); - } - } else if (msqDestination instanceof DataSourceMSQDestination) { - isSelectQuery = false; - if (pageList.get().size() == 1) { - rows = pageList.get().get(0).getNumRows(); - size = pageList.get().get(0).getSizeInBytes(); - } - } else { - isSelectQuery = true; - rows = 0L; - size = 0L; - for (PageInformation pageInformation : pageList.get()) { - rows += pageInformation.getNumRows(); - size += pageInformation.getSizeInBytes(); - } + for (PageInformation pageInformation : pageList.get()) { + rows += pageInformation.getNumRows(); + size += pageInformation.getSizeInBytes(); } } + boolean isSelectQuery = msqDestination instanceof TaskReportMSQDestination + || msqDestination instanceof DurableStorageMSQDestination; + List results = null; if (isSelectQuery) { results = new ArrayList<>(); - Yielder yielder = msqTaskReportPayload.getResults().getResultYielder(); - while (!yielder.isDone()) { - results.add(yielder.get()); - yielder = yielder.next(null); + Yielder yielder = null; + if (msqTaskReportPayload.getResults() != null) { + yielder = msqTaskReportPayload.getResults().getResultYielder(); + } + try { + while (yielder != null && !yielder.isDone()) { + results.add(yielder.get()); + yielder = yielder.next(null); + } } + finally { + if (yielder != null) { + try { + yielder.close(); + } + catch (IOException e) { + log.warn(e, StringUtils.format("Unable to close yielder for query[%s]", queryId)); + } + } + } + } return Optional.of( new ResultSetInformation( - // since the rows can be sampled, get the number of rows from counters rows, size, null, dataSource, results, - isSelectQuery ? pageList.get() : null + isSelectQuery ? pageList.orElse(null) : null ) ); } else { @@ -838,18 +666,176 @@ private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String cur return msqControllerTask; } - private T contactOverlord(final ListenableFuture future) + private Optional> getResultYielder( + String queryId, + Long page, + MSQControllerTask msqControllerTask, + Closer closer + ) { - try { - return FutureUtils.getUnchecked(future, true); - } - catch (RuntimeException e) { + final Optional> results; + + if (msqControllerTask.getQuerySpec().getDestination() instanceof TaskReportMSQDestination) { + // Results from task report are only present as one page. + if (page != null && page > 0) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Page number is out of range of the results."); + } + + MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId))), MSQTaskReportPayload.class); + + if (msqTaskReportPayload.getResults().getResultYielder() == null) { + results = Optional.empty(); + } else { + results = Optional.of(msqTaskReportPayload.getResults().getResultYielder()); + } + + } else if (msqControllerTask.getQuerySpec().getDestination() instanceof DurableStorageMSQDestination) { + + MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId))), MSQTaskReportPayload.class); + + List pages = + SqlStatementResourceHelper.populatePageList( + msqTaskReportPayload, + msqControllerTask.getQuerySpec().getDestination() + ).orElse(null); + + if (pages == null || pages.isEmpty()) { + return Optional.empty(); + } + + final StageDefinition finalStage = Objects.requireNonNull(SqlStatementResourceHelper.getFinalStage( + msqTaskReportPayload)).getStageDefinition(); + + // get all results + final long selectedPage; + if (page != null) { + selectedPage = getPageInformationForPageId(pages, page).getId(); + } else { + selectedPage = Long.MIN_VALUE; + } + final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( + msqControllerTask.getId(), + storageConnector, + closer, + true + ); + results = Optional.of(Yielders.each( + Sequences.concat(pages.stream() + .filter(pageInformation -> pageInformation.getId() == selectedPage) + .map(pageInformation -> { + try { + return new FrameChannelSequence(standardImplementation.openChannel( + finalStage.getId(), + (int) pageInformation.getId(), + 0 // we would always have 0 partition on each worker + )); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList())) + .flatMap(frame -> SqlStatementResourceHelper.getResultSequence( + msqControllerTask, + finalStage, + frame, + jsonMapper + ) + ) + .withBaggage(closer))); + + } else { throw DruidException.forPersona(DruidException.Persona.DEVELOPER) .ofCategory(DruidException.Category.UNCATEGORIZED) - .build("Unable to contact overlord " + e.getMessage()); + .build( + "MSQ select destination[%s] not supported. Please reach out to druid slack community for more help.", + msqControllerTask.getQuerySpec().getDestination().toString() + ); + } + return results; + } + + private PageInformation getPageInformationForPageId(List pages, Long pageId) + { + for (PageInformation pageInfo : pages) { + if (pageInfo.getId() == pageId) { + return pageInfo; + } } + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Invalid page id [%d] passed.", pageId); } + private void resultPusher( + String queryId, + Optional> signature, + Closer closer, + Optional> results, + CountingOutputStream os + ) throws IOException + { + try { + try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { + Yielder yielder = results.get(); + List rowSignature = signature.get(); + writer.writeResponseStart(); + + while (!yielder.isDone()) { + writer.writeRowStart(); + Object[] row = yielder.get(); + for (int i = 0; i < Math.min(rowSignature.size(), row.length); i++) { + writer.writeRowField( + rowSignature.get(i).getColName(), + row[i] + ); + } + writer.writeRowEnd(); + yielder = yielder.next(null); + } + writer.writeResponseEnd(); + yielder.close(); + } + catch (Exception e) { + log.error(e, "Unable to stream results back for query[%s]", queryId); + throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + } + } + catch (Exception e) { + log.error(e, "Unable to stream results back for query[%s]", queryId); + throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + } + finally { + closer.close(); + } + } + + private static void throwIfQueryIsNotSuccessful(String queryId, TaskStatusPlus statusPlus) + { + SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); + if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + queryId, + sqlStatementState + ); + } else if (sqlStatementState == SqlStatementState.FAILED) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] failed. Hit status api for more details.", + queryId + ); + } else { + // do nothing + } + } private void contextChecks(QueryContext queryContext) { @@ -888,4 +874,18 @@ private void contextChecks(QueryContext queryContext) ); } } + + private T contactOverlord(final ListenableFuture future) + { + try { + return FutureUtils.getUnchecked(future, true); + } + catch (RuntimeException e) { + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build("Unable to contact overlord " + e.getMessage()); + } + } + + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java index 761f0e6907c8..81b9a4e7716b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -32,19 +32,17 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; 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.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterSnapshots; +import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.counters.QueryCounterSnapshot; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.MSQDestination; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; +import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.report.MSQStagesReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; @@ -164,6 +162,17 @@ public static long getLastIndex(Long numberOfRows, long start) return last; } + /** + * Populates pages list from the {@link CounterSnapshotsTree}. + *
+ * The number of pages changes with respect to the destination + *
    + *
  1. {@link DataSourceMSQDestination} a single page is returned which adds all the counters of {@link SegmentGenerationProgressCounter.Snapshot}
  2. + *
  3. {@link TaskReportMSQDestination} a single page is returned which adds all the counters of {@link ChannelCounters}
  4. + *
  5. {@link DurableStorageMSQDestination} a page is returned for each worker which has generated output rows. + * If the worker generated 0 rows, we do no populated a page for it. {@link PageInformation#id} is equal to the worker number
  6. + *
+ */ public static Optional> populatePageList( MSQTaskReportPayload msqTaskReportPayload, MSQDestination msqDestination @@ -173,9 +182,7 @@ public static Optional> populatePageList( return Optional.empty(); } int finalStage = msqTaskReportPayload.getStages().getStages().size() - 1; - Map workerCounters = msqTaskReportPayload.getCounters().snapshotForStage(finalStage); - if (workerCounters == null) { return Optional.empty(); } @@ -204,9 +211,13 @@ public static Optional> populatePageList( size += Arrays.stream(((ChannelCounters.Snapshot) queryCounterSnapshot).getBytes()).sum(); } } - return Optional.of(ImmutableList.of(new PageInformation(rows, size, 0))); + if (rows != 0L) { + return Optional.of(ImmutableList.of(new PageInformation(rows, size, 0))); + } else { + return Optional.empty(); + } - } else if (msqDestination instanceof DurableStorageDestination) { + } else if (msqDestination instanceof DurableStorageMSQDestination) { List pageList = new ArrayList<>(); for (Map.Entry counterSnapshots : workerCounters.entrySet()) { long rows = 0L; @@ -216,7 +227,10 @@ public static Optional> populatePageList( rows += Arrays.stream(((ChannelCounters.Snapshot) queryCounterSnapshot).getRows()).sum(); size += Arrays.stream(((ChannelCounters.Snapshot) queryCounterSnapshot).getBytes()).sum(); } - pageList.add(new PageInformation(rows, size, counterSnapshots.getKey())); + // do not populate a page if the worker generated 0 rows. + if (rows != 0L) { + pageList.add(new PageInformation(rows, size, counterSnapshots.getKey())); + } } return Optional.of(pageList); } else { @@ -224,49 +238,6 @@ public static Optional> populatePageList( } } - public static Optional> getRowsAndSizeFromPayload(Map payload, boolean isSelectQuery) - { - List stages = getList(payload, "stages"); - if (stages == null || stages.isEmpty()) { - return Optional.empty(); - } else { - int maxStage = stages.size() - 1; // Last stage output is the total number of rows returned to the end user. - Map counterMap = getMap(getMap(payload, "counters"), String.valueOf(maxStage)); - long rows = -1L; - long sizeInBytes = -1L; - if (counterMap == null) { - return Optional.empty(); - } - for (Map.Entry worker : counterMap.entrySet()) { - Object workerChannels = worker.getValue(); - if (workerChannels == null || !(workerChannels instanceof Map)) { - return Optional.empty(); - } - if (isSelectQuery) { - Object output = ((Map) workerChannels).get("output"); - if (output != null && output instanceof Map) { - List rowsPerChannel = (List) ((Map) output).get("rows"); - List bytesPerChannel = (List) ((Map) output).get("bytes"); - for (Integer row : rowsPerChannel) { - rows = rows + row; - } - for (Integer bytes : bytesPerChannel) { - sizeInBytes = sizeInBytes + bytes; - } - } - } else { - Object output = ((Map) workerChannels).get("segmentGenerationProgress"); - if (output != null && output instanceof Map) { - rows += (Integer) ((Map) output).get("rowsPushed"); - } - } - } - - return Optional.of(new Pair<>(rows == -1L ? null : rows + 1, sizeInBytes == -1L ? null : sizeInBytes + 1)); - } - } - - public static Optional getExceptionPayload( String queryId, TaskStatusResponse taskResponse, @@ -402,35 +373,6 @@ public static Map getMap(Map map, String key) return (Map) map.get(key); } - @SuppressWarnings("rawtypes") - public static List getList(Map map, String key) - { - if (map == null) { - return null; - } - return (List) map.get(key); - } - - /** - * Get results from report - */ - @SuppressWarnings("unchecked") - public static Optional> getResults(Map payload) - { - Map resultsHolder = getMap(payload, "results"); - - if (resultsHolder == null) { - return Optional.empty(); - } - - List data = (List) resultsHolder.get("results"); - List rows = new ArrayList<>(); - if (data != null) { - rows.addAll(data); - } - return Optional.of((Yielders.each(Sequences.simple(rows)))); - } - public static Map getPayload(Map results) { Map msqReport = getMap(results, "multiStageQuery"); 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 index db6bbd71db5c..d751946f24a5 100644 --- 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 @@ -37,7 +37,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; -import org.apache.druid.msq.indexing.destination.DurableStorageDestination; +import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.report.MSQResultsReport; @@ -171,7 +171,7 @@ public void testCalculator() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -203,7 +203,7 @@ public void testSelectOnFoo() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -261,7 +261,7 @@ public void testSelectOnFoo2() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -329,7 +329,7 @@ public void testSelectOnFooDuplicateColumnNames() .columnMappings(expectedColumnMappings) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -391,7 +391,7 @@ public void testSelectOnFooWhereMatchesNoSegments() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -425,7 +425,7 @@ public void testSelectOnFooWhereMatchesNoData() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -460,7 +460,7 @@ public void testSelectAndOrderByOnFooWhereMatchesNoData() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -505,7 +505,7 @@ public void testGroupByOnFoo() )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -572,7 +572,7 @@ public void testGroupByOrderByDimension() )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -629,7 +629,7 @@ public void testSelectWithLimit() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -698,7 +698,7 @@ public void testSelectWithGroupByLimit() )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -728,7 +728,7 @@ public void testSelectLookup() .columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "EXPR$0")))) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -784,7 +784,7 @@ public void testJoinWithLookup() ) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -829,7 +829,7 @@ public void testSubquery() .columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "cnt")))) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -1000,7 +1000,7 @@ private void testJoin(final JoinAlgorithm joinAlgorithm) ) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -1058,7 +1058,7 @@ public void testGroupByOrderByAggregation() ) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -1137,7 +1137,7 @@ public void testGroupByOrderByAggregationWithLimit() ) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -1214,7 +1214,7 @@ public void testGroupByOrderByAggregationWithLimitAndOffset() ) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -1309,7 +1309,7 @@ public void testExternSelect1() throws IOException )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -1422,7 +1422,7 @@ public void testSelectOnUserDefinedSourceContainingWith() .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build() ) @@ -1490,7 +1490,7 @@ public void testScanWithMultiValueSelectQuery() ) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(expectedResultSignature) @@ -1561,7 +1561,7 @@ public void testHavingOnApproximateCountDistinct() ))) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setQueryContext(context) @@ -1615,7 +1615,7 @@ public void testGroupByWithMultiValue() )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -1688,7 +1688,7 @@ public void testGroupByWithMultiValueMvToArray() )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -1758,7 +1758,7 @@ public void testGroupByArrayWithMultiValueMvToArray() ) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -1877,7 +1877,7 @@ public void testGroupByMultiValueMeasureQuery() )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -1931,7 +1931,7 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) @@ -2005,7 +2005,7 @@ public void testSelectRowsGetUntruncatedByDefault() throws IOException )) .tuningConfig(MSQTuningConfig.defaultConfig()) .destination(isDurableStorageDestination() - ? DurableStorageDestination.INSTANCE + ? DurableStorageMSQDestination.INSTANCE : TaskReportMSQDestination.INSTANCE) .build()) .setQueryContext(context) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java new file mode 100644 index 000000000000..70700c026d63 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.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.querykit.results; + +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; +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.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessorExecutor; +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.concurrent.Execs; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +public class QueryResultsFrameProcessorTest extends InitializedNullHandlingTest +{ + private FrameProcessorExecutor exec; + + @Before + public void setUp() + { + exec = new FrameProcessorExecutor(MoreExecutors.listeningDecorator(Execs.singleThreaded("test-exec"))); + } + + @After + public void tearDown() throws Exception + { + exec.getExecutorService().shutdownNow(); + exec.getExecutorService().awaitTermination(10, TimeUnit.MINUTES); + } + + + @Test + public void sanityTest() throws ExecutionException, InterruptedException, IOException + { + + final IncrementalIndexStorageAdapter adapter = + new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + + final FrameSequenceBuilder frameSequenceBuilder = + FrameSequenceBuilder.fromAdapter(adapter) + .maxRowsPerFrame(5) + .frameType(FrameType.ROW_BASED) + .allocator(ArenaMemoryAllocator.createOnHeap(100_000)); + + final RowSignature signature = frameSequenceBuilder.signature(); + final List frames = frameSequenceBuilder.frames().toList(); + final BlockingQueueFrameChannel inputChannel = new BlockingQueueFrameChannel(frames.size()); + final BlockingQueueFrameChannel outputChannel = BlockingQueueFrameChannel.minimal(); + + try (final WritableFrameChannel writableInputChannel = inputChannel.writable()) { + for (final Frame frame : frames) { + writableInputChannel.write(frame); + } + } + + final StagePartition stagePartition = new StagePartition(new StageId("query", 0), 0); + + final QueryResultsFrameProcessor processor = + new QueryResultsFrameProcessor(ReadableInput.channel( + inputChannel.readable(), + FrameReader.create(signature), + stagePartition + ).getChannel(), outputChannel.writable()); + + ListenableFuture retVal = exec.runFully(processor, null); + final Sequence> rowsFromProcessor = FrameTestUtil.readRowsFromFrameChannel( + outputChannel.readable(), + FrameReader.create(signature) + ); + FrameTestUtil.assertRowsEqual( + FrameTestUtil.readRowsFromAdapter(adapter, signature, false), + rowsFromProcessor + ); + Assert.assertEquals(adapter.getNumRows(), (long) retVal.get()); + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java index 42ebf1dfc2aa..38d99dfe03d0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactoryTest.java @@ -20,7 +20,7 @@ package org.apache.druid.msq.shuffle; import org.apache.druid.frame.processor.OutputChannelFactoryTest; -import org.apache.druid.msq.shuffle.output.DurableStorageTaskOutputChannelFactoryImpl; +import org.apache.druid.msq.shuffle.output.DurableStorageTaskOutputChannelFactory; import org.apache.druid.storage.local.LocalFileStorageConnector; import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; @@ -36,7 +36,7 @@ public DurableStorageOutputChannelFactoryTest() throws IOException { super( - new DurableStorageTaskOutputChannelFactoryImpl( + new DurableStorageTaskOutputChannelFactory( "0", 0, 0, 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 index 74d0021de9dd..68965f40bfed 100644 --- 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 @@ -1315,10 +1315,11 @@ public Pair, List>> } else { StageDefinition finalStage = Objects.requireNonNull(SqlStatementResourceHelper.getFinalStage( payload)).getStageDefinition(); + Closer closer = Closer.create(); InputChannelFactory inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation( controllerId, localFileStorageConnector, - Closer.create(), + closer, true ); rows = new FrameChannelSequence(inputChannelFactory.openChannel( @@ -1330,7 +1331,7 @@ public Pair, List>> finalStage, frame, objectMapper - )).toList(); + )).withBaggage(closer).toList(); } if (rows == null) { diff --git a/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java index 6e2dbb3bf28c..e63d360c2ddf 100644 --- a/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java @@ -19,7 +19,7 @@ package org.apache.druid.storage; -import org.apache.druid.java.util.common.UOE; +import org.apache.druid.error.DruidException; import java.io.InputStream; import java.io.OutputStream; @@ -42,48 +42,57 @@ public static NilStorageConnector getInstance() @Override public boolean pathExists(String path) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); } @Override public InputStream read(String path) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); + } @Override public InputStream readRange(String path, long from, long size) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); + } @Override public OutputStream write(String path) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); } @Override public void deleteFile(String path) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); } @Override public void deleteFiles(Iterable paths) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); } @Override public void deleteRecursively(String path) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); } @Override public Iterator listDir(String dirName) { - throw new UOE("Please configure durable storage."); + throw notConfiguredException(); + } + + private DruidException notConfiguredException() + { + return DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.DEFENSIVE) + .build("Please configure durable storage."); } } diff --git a/processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java b/processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java index 894f26cf2208..8dfcb1b78f13 100644 --- a/processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java +++ b/processing/src/test/java/org/apache/druid/storage/NilStorageConnectorTest.java @@ -20,7 +20,7 @@ package org.apache.druid.storage; import com.google.common.collect.ImmutableList; -import org.apache.druid.java.util.common.UOE; +import org.apache.druid.error.DruidException; import org.junit.Assert; import org.junit.Test; @@ -34,14 +34,14 @@ public class NilStorageConnectorTest public void sanity() { NilStorageConnector nilStorageConnector = NilStorageConnector.getInstance(); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.pathExists("null")); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.read("null")); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.readRange("null", 0, 0)); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.deleteFile("null")); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.deleteFiles(ImmutableList.of())); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.deleteRecursively("null")); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.listDir("null")); - Assert.assertThrows(ERROR_MESSAGE, UOE.class, () -> nilStorageConnector.pathExists("null")); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.pathExists("null")); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.read("null")); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.readRange("null", 0, 0)); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.deleteFile("null")); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.deleteFiles(ImmutableList.of())); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.deleteRecursively("null")); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.listDir("null")); + Assert.assertThrows(ERROR_MESSAGE, DruidException.class, () -> nilStorageConnector.pathExists("null")); } } From c5f4205e20cc2b324892278b28a1af57351f42f1 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 7 Jul 2023 09:44:17 +0530 Subject: [PATCH 09/15] Test case fix. --- .../druid/msq/sql/resources/SqlStatementResource.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index fa7bd38d81c0..6942fe1d30d4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -545,9 +545,11 @@ private Optional getSampleResults( ); // getting the total number of rows, size from page information. - Long rows = 0L; - Long size = 0L; + Long rows = null; + Long size = null; if (pageList.isPresent()) { + rows = 0L; + size = 0L; for (PageInformation pageInformation : pageList.get()) { rows += pageInformation.getNumRows(); size += pageInformation.getSizeInBytes(); From 47fc66c3127c683de394433f456597a14a58276c Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 7 Jul 2023 10:33:28 +0530 Subject: [PATCH 10/15] Adding java docs. --- .../DurableStorageInputChannelFactory.java | 11 +++++++++-- ...StorageQueryResultsInputChannelFactory.java | 15 +++++++++------ ...DurableStorageStageInputChannelFactory.java | 8 ++++++-- .../DurableStorageOutputChannelFactory.java | 10 ++++++++-- ...torageQueryResultsOutputChannelFactory.java | 18 ++++++++++-------- ...DurableStorageTaskOutputChannelFactory.java | 16 +++++++++++----- .../druid/frame/util/DurableStorageUtils.java | 4 ++-- .../frame/util/DurableStorageUtilsTest.java | 4 ++-- 8 files changed, 57 insertions(+), 29 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java index dda4ac246676..c0e892b99bf0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java @@ -174,7 +174,7 @@ public String findSuccessfulPartitionOutput( successfulTaskId ); - return getPartitionOutputsFileNameForPartition( + return getPartitionOutputsFileNameWithPathForPartition( controllerTaskId, stageNumber, workerNo, @@ -183,7 +183,10 @@ public String findSuccessfulPartitionOutput( ); } - public abstract String getPartitionOutputsFileNameForPartition( + /** + * Get the filePath with filename for the partitioned output of the controller, stage, worker + */ + public abstract String getPartitionOutputsFileNameWithPathForPartition( String controllerTaskId, int stageNumber, int workerNo, @@ -191,6 +194,10 @@ public abstract String getPartitionOutputsFileNameForPartition( String successfulTaskId ); + /** + * Get the filepath for the success file . + */ + public abstract String getWorkerOutputSuccessFilePath(String controllerTaskId, int stageNumber, int workerNumber); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java index bd8f5da5bed8..b29220e790d7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java @@ -24,6 +24,9 @@ import java.util.concurrent.ExecutorService; +/** + * Used for reading results when select destination is {@link org.apache.druid.msq.indexing.destination.MSQSelectDestination#DURABLE_STORAGE} + */ public class DurableStorageQueryResultsInputChannelFactory extends DurableStorageInputChannelFactory { @@ -37,7 +40,7 @@ public DurableStorageQueryResultsInputChannelFactory( } @Override - public String getPartitionOutputsFileNameForPartition( + public String getPartitionOutputsFileNameWithPathForPartition( String controllerTaskId, int stageNumber, int workerNo, @@ -45,11 +48,11 @@ public String getPartitionOutputsFileNameForPartition( String successfulTaskId ) { - return DurableStorageUtils.getQueryResultsFileNameForPartition(controllerTaskId, - stageNumber, - workerNo, - successfulTaskId, - partitionNumber); + return DurableStorageUtils.getQueryResultsFileNameWithPathForPartition(controllerTaskId, + stageNumber, + workerNo, + successfulTaskId, + partitionNumber); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java index d6d142924e2f..1034b402679c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java @@ -24,6 +24,10 @@ import java.util.concurrent.ExecutorService; +/** + * Used for reading stage results when the output of each stage is written out to durable storage. + * If the user want's to read the output of a select query, please use {@link DurableStorageQueryResultsInputChannelFactory} + */ public class DurableStorageStageInputChannelFactory extends DurableStorageInputChannelFactory { public DurableStorageStageInputChannelFactory( @@ -36,7 +40,7 @@ public DurableStorageStageInputChannelFactory( } @Override - public String getPartitionOutputsFileNameForPartition( + public String getPartitionOutputsFileNameWithPathForPartition( String controllerTaskId, int stageNumber, int workerNo, @@ -44,7 +48,7 @@ public String getPartitionOutputsFileNameForPartition( String successfulTaskId ) { - return DurableStorageUtils.getPartitionOutputsFileNameForPartition( + return DurableStorageUtils.getPartitionOutputsFileNameWithPathForPartition( controllerTaskId, stageNumber, workerNo, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java index abfde486789b..009bfd973a41 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java @@ -128,12 +128,15 @@ public void createSuccessFile(String taskId) throws IOException os.close(); } + /** + * Get filepath to write success file in. + */ public abstract String getSuccessFilePath(); @Override public OutputChannel openNilChannel(int partitionNumber) { - final String fileName = getFileNameForPartition(partitionNumber); + final String fileName = getFileNameWithPathForPartition(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, ByteTracker.unboundedTracker()) @@ -151,5 +154,8 @@ public OutputChannel openNilChannel(int partitionNumber) } } - protected abstract String getFileNameForPartition(int partitionNumber); + /** + * Get fileName with path for partition + */ + protected abstract String getFileNameWithPathForPartition(int partitionNumber); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java index c1baa1d54c96..271129aa4350 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java @@ -34,10 +34,12 @@ import java.io.IOException; import java.nio.channels.Channels; +/** + * Used to write out select query results to durable storage. + */ public class DurableStorageQueryResultsOutputChannelFactory extends DurableStorageOutputChannelFactory { - public DurableStorageQueryResultsOutputChannelFactory( String controllerTaskId, int workerNumber, @@ -62,19 +64,19 @@ public String getSuccessFilePath() } @Override - protected String getFileNameForPartition(int partitionNumber) + protected String getFileNameWithPathForPartition(int partitionNumber) { - return DurableStorageUtils.getQueryResultsFileNameForPartition(controllerTaskId, - stageNumber, - workerNumber, - taskId, - partitionNumber); + return DurableStorageUtils.getQueryResultsFileNameWithPathForPartition(controllerTaskId, + stageNumber, + workerNumber, + taskId, + partitionNumber); } @Override public OutputChannel openChannel(int partitionNumber) throws IOException { - final String fileName = getFileNameForPartition(partitionNumber); + final String fileName = getFileNameWithPathForPartition(partitionNumber); final WritableFrameFileChannel writableChannel = new WritableFrameFileChannel( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java index c5bc83ffc8ee..96d4860d82c7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java @@ -51,6 +51,10 @@ import java.nio.channels.Channels; import java.util.function.Supplier; +/** + * Used to write out intermediate task output files to durable storage. To write final stage output files for select queries use + * {@link DurableStorageQueryResultsOutputChannelFactory} + */ public class DurableStorageTaskOutputChannelFactory extends DurableStorageOutputChannelFactory { @@ -74,9 +78,9 @@ public String getSuccessFilePath() } @Override - protected String getFileNameForPartition(int partitionNumber) + protected String getFileNameWithPathForPartition(int partitionNumber) { - return DurableStorageUtils.getPartitionOutputsFileNameForPartition( + return DurableStorageUtils.getPartitionOutputsFileNameWithPathForPartition( controllerTaskId, stageNumber, workerNumber, @@ -89,7 +93,7 @@ protected String getFileNameForPartition(int partitionNumber) @Override public OutputChannel openChannel(int partitionNumber) throws IOException { - final String fileName = getFileNameForPartition(partitionNumber); + final String fileName = getFileNameWithPathForPartition(partitionNumber); final WritableFrameFileChannel writableChannel = new WritableFrameFileChannel( FrameFileWriter.open( @@ -178,8 +182,10 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele storageConnector.readRange(fileName, channelSize - footerLength, footerLength)) { IOUtils.copy(footerInputStream, footerFileStream); } - MappedByteBufferHandler mapHandle = FileUtils.map(footerFile); - Memory footerMemory = Memory.wrap(mapHandle.get(), ByteOrder.LITTLE_ENDIAN); + Memory footerMemory; + try (MappedByteBufferHandler mapHandle = FileUtils.map(footerFile)) { + footerMemory = Memory.wrap(mapHandle.get(), ByteOrder.LITTLE_ENDIAN); + } // create a frame file footer from the mapper memory return new FrameFileFooter(footerMemory, channelSize); diff --git a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java index 738f3de0809e..3e5f2fe00a1f 100644 --- a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java @@ -149,7 +149,7 @@ public static String getQueryResultsForTaskIdFolderName( * Fetches the file location where a particular worker writes the data corresponding to a particular stage * and partition */ - public static String getPartitionOutputsFileNameForPartition( + public static String getPartitionOutputsFileNameWithPathForPartition( final String controllerTaskId, final int stageNumber, final int workerNumber, @@ -164,7 +164,7 @@ public static String getPartitionOutputsFileNameForPartition( ); } - public static String getQueryResultsFileNameForPartition( + public static String getQueryResultsFileNameWithPathForPartition( final String controllerTaskId, final int stageNumber, final int workerNumber, diff --git a/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java b/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java index 093eeca7c46b..2f01a402db43 100644 --- a/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java @@ -113,7 +113,7 @@ public void sanityTest() Assert.assertEquals( baseString + "taskId_" + TASK_ID + "/part_3", - DurableStorageUtils.getPartitionOutputsFileNameForPartition( + DurableStorageUtils.getPartitionOutputsFileNameWithPathForPartition( CONTROLLER_ID, STAGE_NUMBER, WORKER_NUMBER, @@ -123,7 +123,7 @@ public void sanityTest() ); Assert.assertEquals( DurableStorageUtils.QUERY_RESULTS_DIR + "/" + baseString + "taskId_" + TASK_ID + "/part_3", - DurableStorageUtils.getQueryResultsFileNameForPartition( + DurableStorageUtils.getQueryResultsFileNameWithPathForPartition( CONTROLLER_ID, STAGE_NUMBER, WORKER_NUMBER, From 73ed03d2407102279bcaefc2206b40ee7b8b12e7 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 7 Jul 2023 11:42:48 +0530 Subject: [PATCH 11/15] Reverting autocloseable change. --- .../output/DurableStorageTaskOutputChannelFactory.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java index 96d4860d82c7..7ece0dd31556 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageTaskOutputChannelFactory.java @@ -182,10 +182,8 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele storageConnector.readRange(fileName, channelSize - footerLength, footerLength)) { IOUtils.copy(footerInputStream, footerFileStream); } - Memory footerMemory; - try (MappedByteBufferHandler mapHandle = FileUtils.map(footerFile)) { - footerMemory = Memory.wrap(mapHandle.get(), ByteOrder.LITTLE_ENDIAN); - } + MappedByteBufferHandler mapHandle = FileUtils.map(footerFile); + Memory footerMemory = Memory.wrap(mapHandle.get(), ByteOrder.LITTLE_ENDIAN); // create a frame file footer from the mapper memory return new FrameFileFooter(footerMemory, channelSize); From c985be5c945e5162e4d9fde82b51ba6438157ecd Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 7 Jul 2023 13:31:04 +0530 Subject: [PATCH 12/15] Fixing pageID bug. --- .../QueryResultFrameProcessorFactory.java | 1 + .../druid/msq/sql/entity/PageInformation.java | 15 ++++++++++ .../sql/resources/SqlStatementResource.java | 30 +++++++++++-------- .../msq/util/SqlStatementResourceHelper.java | 2 ++ 4 files changed, 36 insertions(+), 12 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java index 30c20fe22dad..95a4c7df7d91 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java @@ -80,6 +80,7 @@ public ProcessorsAndChannels, Long> makeProcessors( final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); for (final ReadablePartition partition : slice.getPartitions()) { + System.out.println("partitionId " + partition.getPartitionNumber()); outputChannels.computeIfAbsent( partition.getPartitionNumber(), i -> { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java index 2754c52f1fe1..1c212f275ee1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.Objects; /** @@ -100,4 +101,18 @@ public String toString() ", id=" + id + '}'; } + + public static Comparator getIDComparator() + { + return new PageComparator(); + } + + public static class PageComparator implements Comparator + { + @Override + public int compare(PageInformation s1, PageInformation s2) + { + return Long.compare(s1.getId(), s2.getId()); + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 6942fe1d30d4..ab83b71853fb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -268,7 +268,7 @@ public Response doGetStatus( ); } catch (Exception e) { - log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + log.warn(e, "Failed to handle query: %s", queryId); return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) .ofCategory(DruidException.Category.UNCATEGORIZED) .build(e, "Failed to handle query: [%s]", queryId)); @@ -353,7 +353,7 @@ public Response doGetResults( ); } catch (Exception e) { - log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + log.warn(e, "Failed to handle query: %s", queryId); return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) .ofCategory(DruidException.Category.UNCATEGORIZED) .build(e, "Failed to handle query: [%s]", queryId)); @@ -420,7 +420,7 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina ); } catch (Exception e) { - log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + log.warn(e, "Failed to handle query: %s", queryId); return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) .ofCategory(DruidException.Category.UNCATEGORIZED) .build(e, "Failed to handle query: [%s]", queryId)); @@ -713,12 +713,13 @@ private Optional> getResultYielder( msqTaskReportPayload)).getStageDefinition(); // get all results - final long selectedPage; + final Long selectedPage; if (page != null) { selectedPage = getPageInformationForPageId(pages, page).getId(); } else { - selectedPage = Long.MIN_VALUE; + selectedPage = null; } + checkForDurableStorageConnectorImpl(); final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( msqControllerTask.getId(), storageConnector, @@ -727,13 +728,13 @@ private Optional> getResultYielder( ); results = Optional.of(Yielders.each( Sequences.concat(pages.stream() - .filter(pageInformation -> pageInformation.getId() == selectedPage) + .filter(pageInformation -> selectedPage == null || selectedPage.equals(page)) .map(pageInformation -> { try { return new FrameChannelSequence(standardImplementation.openChannel( finalStage.getId(), (int) pageInformation.getId(), - 0 // we would always have 0 partition on each worker + (int) pageInformation.getId()// we would always have partition number == worker number )); } catch (Exception e) { @@ -859,19 +860,24 @@ private void contextChecks(QueryContext queryContext) ); } - MSQSelectDestination selectDestination = MultiStageQueryContext.getSelectDestination(queryContext); - if (selectDestination == MSQSelectDestination.DURABLE_STORAGE - && storageConnector instanceof NilStorageConnector) { + if (selectDestination == MSQSelectDestination.DURABLE_STORAGE) { + checkForDurableStorageConnectorImpl(); + } + } + + private void checkForDurableStorageConnectorImpl() + { + if (storageConnector instanceof NilStorageConnector) { throw DruidException.forPersona(DruidException.Persona.USER) .ofCategory(DruidException.Category.INVALID_INPUT) .build( StringUtils.format( - "The statement sql api only supports select destination [%s=%s]. " + "The statement sql api cannot read from select destination [%s=%s] since its not configured. " + "Its recommended to configure durable storage as it allows the user to fetch big results. " + "Please contact your cluster admin to configure durable storage.", MultiStageQueryContext.CTX_SELECT_DESTINATION, - selectDestination.name() + MSQSelectDestination.DURABLE_STORAGE.name() ) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java index 81b9a4e7716b..4211a062b5df 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -61,6 +61,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -232,6 +233,7 @@ public static Optional> populatePageList( pageList.add(new PageInformation(rows, size, counterSnapshots.getKey())); } } + Collections.sort(pageList, PageInformation.getIDComparator()); return Optional.of(pageList); } else { return Optional.empty(); From 6984979d3a6f35261c0f13511c6974e89de8b2eb Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 7 Jul 2023 13:31:38 +0530 Subject: [PATCH 13/15] Reverting debug statement. --- .../msq/querykit/results/QueryResultFrameProcessorFactory.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java index 95a4c7df7d91..30c20fe22dad 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java @@ -80,7 +80,6 @@ public ProcessorsAndChannels, Long> makeProcessors( final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); for (final ReadablePartition partition : slice.getPartitions()) { - System.out.println("partitionId " + partition.getPartitionNumber()); outputChannels.computeIfAbsent( partition.getPartitionNumber(), i -> { From 81a7ecbca2e32feeeb707ab953030bb278f3f262 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 7 Jul 2023 13:37:17 +0530 Subject: [PATCH 14/15] Better naming --- .../druid/msq/sql/resources/SqlStatementResource.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index ab83b71853fb..268cfcb5b0a4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -713,11 +713,11 @@ private Optional> getResultYielder( msqTaskReportPayload)).getStageDefinition(); // get all results - final Long selectedPage; + final Long selectedPageId; if (page != null) { - selectedPage = getPageInformationForPageId(pages, page).getId(); + selectedPageId = getPageInformationForPageId(pages, page).getId(); } else { - selectedPage = null; + selectedPageId = null; } checkForDurableStorageConnectorImpl(); final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( @@ -728,7 +728,8 @@ private Optional> getResultYielder( ); results = Optional.of(Yielders.each( Sequences.concat(pages.stream() - .filter(pageInformation -> selectedPage == null || selectedPage.equals(page)) + .filter(pageInformation -> selectedPageId == null + || selectedPageId.equals(pageInformation.getId())) .map(pageInformation -> { try { return new FrameChannelSequence(standardImplementation.openChannel( From 8f1971e3f61ae96c055af3ac2b42013b60c7b19a Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 7 Jul 2023 18:10:11 +0530 Subject: [PATCH 15/15] Adding durable storage test for async api. --- .../sql/resources/SqlStatementResource.java | 1 - ...a => SqlMSQStatementResourcePostTest.java} | 89 +++++++++++++++++-- .../msq/sql/SqlStatementResourceTest.java | 7 +- 3 files changed, 87 insertions(+), 10 deletions(-) rename extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/{SqlMsqStatementResourcePostTest.java => SqlMSQStatementResourcePostTest.java} (74%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 268cfcb5b0a4..2be55a2f840c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -170,7 +170,6 @@ public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletReques isDebug = queryContext.isDebug(); contextChecks(queryContext); - Thread.currentThread().setName(StringUtils.format("statement_sql[%s]", sqlQueryId)); final DirectStatement.ResultSet plan = stmt.plan(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMSQStatementResourcePostTest.java similarity index 74% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMSQStatementResourcePostTest.java index 0b7732ceba54..e5133ee3d21b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMSQStatementResourcePostTest.java @@ -26,6 +26,7 @@ import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; @@ -35,37 +36,38 @@ import org.apache.druid.msq.sql.resources.SqlStatementResource; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestOverlordServiceClient; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.ExecutionMode; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlQuery; -import org.apache.druid.storage.local.LocalFileStorageConnector; +import org.apache.druid.storage.NilStorageConnector; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import javax.ws.rs.core.Response; import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -public class SqlMsqStatementResourcePostTest extends MSQTestBase +public class SqlMSQStatementResourcePostTest extends MSQTestBase { private SqlStatementResource resource; @Before - public void init() throws IOException + public void init() { resource = new SqlStatementResource( sqlStatementFactory, CalciteTests.TEST_AUTHORIZER_MAPPER, objectMapper, indexingServiceClient, - new LocalFileStorageConnector(tmpFolder.newFolder("results")) - + localFileStorageConnector ); } @@ -247,6 +249,83 @@ public void forbiddenTest() ).getStatus()); } + @Test + public void durableStorageDisabledTest() + { + SqlStatementResource resourceWithDurableStorage = new SqlStatementResource( + sqlStatementFactory, + CalciteTests.TEST_AUTHORIZER_MAPPER, + objectMapper, + indexingServiceClient, + NilStorageConnector.getInstance() + ); + + String errorMessage = StringUtils.format( + "The statement sql api cannot read from select destination [%s=%s] since its not configured. " + + "Its recommended to configure durable storage as it allows the user to fetch big results. " + + "Please contact your cluster admin to configure durable storage.", + MultiStageQueryContext.CTX_SELECT_DESTINATION, + MSQSelectDestination.DURABLE_STORAGE.name() + ); + Map context = defaultAsyncContext(); + context.put(MultiStageQueryContext.CTX_SELECT_DESTINATION, MSQSelectDestination.DURABLE_STORAGE.name()); + + SqlStatementResourceTest.assertExceptionMessage(resourceWithDurableStorage.doPost( + new SqlQuery( + "select * from foo", + null, + false, + false, + false, + context, + null + ), + SqlStatementResourceTest.makeOkRequest() + ), errorMessage, + Response.Status.BAD_REQUEST + ); + } + + @Test + public void testWithDurableStorage() throws IOException + { + Map context = defaultAsyncContext(); + context.put(MultiStageQueryContext.CTX_SELECT_DESTINATION, MSQSelectDestination.DURABLE_STORAGE.name()); + + SqlStatementResult sqlStatementResult = (SqlStatementResult) resource.doPost( + new SqlQuery( + "select cnt,dim1 from foo", + null, + false, + false, + false, + context, + null + ), + SqlStatementResourceTest.makeOkRequest() + ).getEntity(); + + + List> rows = new ArrayList<>(); + rows.add(ImmutableMap.of("cnt", 1, "dim1", "")); + rows.add(ImmutableMap.of("cnt", 1, "dim1", "10.1")); + rows.add(ImmutableMap.of("cnt", 1, "dim1", "2")); + rows.add(ImmutableMap.of("cnt", 1, "dim1", "1")); + rows.add(ImmutableMap.of("cnt", 1, "dim1", "def")); + rows.add(ImmutableMap.of("cnt", 1, "dim1", "abc")); + + Assert.assertEquals(rows, SqlStatementResourceTest.getResultRowsFromResponse(resource.doGetResults( + sqlStatementResult.getQueryId(), + null, + SqlStatementResourceTest.makeOkRequest() + ))); + + Assert.assertEquals(rows, SqlStatementResourceTest.getResultRowsFromResponse(resource.doGetResults( + sqlStatementResult.getQueryId(), + 0L, + SqlStatementResourceTest.makeOkRequest() + ))); + } private static Map defaultAsyncContext() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 1dba4f8bc73a..51b64b7eedd0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -336,6 +336,9 @@ public class SqlStatementResourceTest extends MSQTestBase @Mock private OverlordClient overlordClient; + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + private void setupMocks(OverlordClient indexingServiceClient) throws JsonProcessingException { @@ -404,10 +407,6 @@ private void setupMocks(OverlordClient indexingServiceClient) throws JsonProcess ))); - final ObjectMapper mapper = TestHelper.makeJsonMapper() - .registerModules(new MSQIndexingModule().getJacksonModules()); - - Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(mapper.readValue( mapper.writeValueAsString(TaskReport.buildTaskReports(selectTaskReport)),