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 953f8b718de..8936e104bd6 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 class CounterSnapshotsTree 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 ec187b9827d..8f3e2d79aae 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 3552459f235..b7a93de5ed0 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.indexing.common.TaskReport; import org.apache.druid.msq.counters.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.client.ControllerChatHandler; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; @@ -83,7 +84,7 @@ public interface Controller /** * 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 1a8ef12cb2f..f5f7b0cc1aa 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.KeyOrder; 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.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; -import org.apache.druid.msq.indexing.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.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; 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.QueryNotSupportedFault; 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.QueryKitUtils; import org.apache.druid.msq.querykit.ShuffleSpecFactories; import org.apache.druid.msq.querykit.ShuffleSpecFactory; import org.apache.druid.msq.querykit.groupby.GroupByQueryKit; +import org.apache.druid.msq.querykit.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.server.DruidNode; 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.Collections; 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 @@ public class ControllerImpl implements Controller 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 @@ public class ControllerImpl implements Controller /** * 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 @@ public class ControllerImpl implements Controller 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 @@ public class ControllerImpl implements Controller } ).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,10 @@ public class ControllerImpl implements Controller } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); queryToPlan = querySpec.getQuery(); + } 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(); } else { throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); } @@ -1645,6 +1616,24 @@ public class ControllerImpl implements Controller return builder.build(); } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { return queryDef; + } else if (querySpec.getDestination() instanceof DurableStorageMSQDestination) { + + // attaching new query results stage always. + StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition(); + 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(null) + .processorFactory(new QueryResultFrameProcessorFactory()) + ); + + return builder.build(); } else { throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); } @@ -1754,7 +1743,8 @@ public class ControllerImpl implements Controller private static boolean isInlineResults(final MSQSpec querySpec) { - return querySpec.getDestination() instanceof TaskReportMSQDestination; + return querySpec.getDestination() instanceof TaskReportMSQDestination + || querySpec.getDestination() instanceof DurableStorageMSQDestination; } private static boolean isTimeBucketedIngestion(final MSQSpec querySpec) @@ -2051,7 +2041,12 @@ public class ControllerImpl implements Controller ); } - return MSQResultsReport.createReportAndLimitRowsIfNeeded(mappedSignature.build(), sqlTypeNames, resultsYielder, selectDestination); + return MSQResultsReport.createReportAndLimitRowsIfNeeded( + mappedSignature.build(), + sqlTypeNames, + resultsYielder, + selectDestination + ); } private static MSQStatusReport makeStatusReport( @@ -2597,7 +2592,8 @@ public class ControllerImpl implements Controller 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 44fe0dbce8d..24a3fad8dbf 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.KeyCollectorSnapshotDeserializerModule; import org.apache.druid.msq.statistics.KeyCollectors; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.server.DruidNode; +import org.apache.druid.storage.NilStorageConnector; import org.apache.druid.storage.StorageConnector; import javax.annotation.Nullable; @@ -155,7 +156,11 @@ public class MSQTasks 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 cb5881b3ba4..942d8b44b0c 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.counters.CounterTracker; import org.apache.druid.msq.indexing.CountingOutputChannelFactory; import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.indexing.InputChannelsImpl; -import org.apache.druid.msq.indexing.KeyStatisticsCollectionProcessor; import org.apache.druid.msq.indexing.MSQWorkerTask; +import org.apache.druid.msq.indexing.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.MSQWarningReportLimiterPublisher; 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.StagePartition; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.kernel.worker.WorkerStageKernel; import org.apache.druid.msq.kernel.worker.WorkerStagePhase; -import org.apache.druid.msq.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 class WorkerImpl implements Worker 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 class WorkerImpl implements Worker 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 @@ public class WorkerImpl implements Worker 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 @@ public class WorkerImpl implements Worker 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 @@ public class WorkerImpl implements Worker task().getId(), frameSize, MSQTasks.makeStorageConnector(context.injector()), - tmpDir + tmpDir, + false ) ), frameSize @@ -1014,9 +1031,13 @@ public class WorkerImpl implements Worker 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 @@ public class WorkerImpl implements Worker Futures.immediateFuture(workResultAndOutputChannels.getOutputChannels().readOnly()); } - setUpCompletionCallbacks(); + setUpCompletionCallbacks(isFinalStage); } /** @@ -1072,7 +1093,7 @@ public class WorkerImpl implements Worker ); } - private void makeWorkOutputChannelFactory() + private void makeWorkOutputChannelFactory(boolean isFinalStage) { if (workOutputChannelFactory != null) { throw new ISE("processorOutputChannelFactory already created"); @@ -1096,7 +1117,7 @@ public class WorkerImpl implements Worker } else { // Writing stage output. baseOutputChannelFactory = - makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber()); + makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber(), isFinalStage); } workOutputChannelFactory = new CountingOutputChannelFactory( @@ -1105,11 +1126,11 @@ public class WorkerImpl implements Worker ); } - 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 @@ public class WorkerImpl implements Worker } } - private void setUpCompletionCallbacks() + private void setUpCompletionCallbacks(boolean isFinalStage) { final StageDefinition stageDef = kernel.getStageDefinition(); @@ -1273,7 +1294,7 @@ public class WorkerImpl implements Worker // 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 class WorkerImpl implements Worker /** * 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,11 +1342,7 @@ public class WorkerImpl implements Worker e, "Unable to create the success file [%s] at the location [%s]", DurableStorageUtils.SUCCESS_MARKER_FILENAME, - DurableStorageUtils.getSuccessFilePath( - 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 81c6119b8fa..7139377495a 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.LazySingleton; 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,11 @@ public class MSQDurableStorageModule implements DruidModule .addBinding() .to(DurableStorageCleaner.class); } + } 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/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 59300316d58..4af832705c3 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.SuperSorterProgressTrackerCounter; import org.apache.druid.msq.counters.WarningCounters; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQWorkerTask; -import org.apache.druid.msq.indexing.SegmentGeneratorFrameProcessorFactory; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; @@ -69,6 +68,7 @@ import org.apache.druid.msq.indexing.error.TooManyWorkersFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; +import org.apache.druid.msq.indexing.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.OffsetLimitFrameProcessorFactory; 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 class MSQIndexingModule implements DruidModule 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 cc767626ecb..5d17b005b94 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.Controller; import org.apache.druid.msq.exec.ControllerContext; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerManagerClient; +import org.apache.druid.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 fe50bc19ace..10855094bde 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.Worker; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerContext; import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.indexing.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 30047f227ff..c39b7637dd6 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,9 @@ import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerContext; import org.apache.druid.msq.exec.ControllerImpl; import org.apache.druid.msq.exec.MSQTasks; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +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; @@ -263,4 +266,9 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery { return querySpec.getDestination() instanceof DataSourceMSQDestination; } + + public static boolean writeResultsToDurableStorage(final MSQSpec querySpec) + { + 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 22dba052fa1..065471d32ba 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,8 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.msq.indexing.destination.MSQDestination; +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 195b1e26f89..630499bdd87 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 d6637d0ffbd..d45950f37ed 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 5b36117d3f4..22d3b31cf10 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.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.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 00302ebfc57..493cbeb6242 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.BytesFullResponseHolder; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.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 980c7f97bee..39a9c40e08d 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 class IndexerWorkerClient implements WorkerClient 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 class IndexerWorkerClient implements WorkerClient 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 class IndexerWorkerClient implements WorkerClient new RequestBuilder(HttpMethod.GET, "/counters"), new BytesFullResponseHandler() ), - holder -> deserialize(holder, new TypeReference() {}) + holder -> deserialize(holder, new TypeReference() + { + }) ); } @@ -299,7 +309,7 @@ public class IndexerWorkerClient implements WorkerClient /** * 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 707ade86a52..2940fc4f9c3 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 3eae3b05ccf..0d26a21f3db 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.indexing.common.TaskToolbox; 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 98% 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 6b1a683ff0e..5bf033a1aa0 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; @@ -35,7 +35,7 @@ import java.util.Objects; 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/DurableStorageMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java new file mode 100644 index 00000000000..c401d2eee6e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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; + +public class DurableStorageMSQDestination implements MSQDestination +{ + public static final String TYPE = "durableStorage"; + + public static final DurableStorageMSQDestination INSTANCE = new DurableStorageMSQDestination(); + + + private DurableStorageMSQDestination() + { + // Singleton. + } + + @JsonCreator + public static DurableStorageMSQDestination 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/MSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQDestination.java similarity index 86% 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 d84a1635edc..52489d15a34 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,7 +17,7 @@ * 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; @@ -25,7 +25,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = DataSourceMSQDestination.TYPE, value = DataSourceMSQDestination.class), - @JsonSubTypes.Type(name = TaskReportMSQDestination.TYPE, value = TaskReportMSQDestination.class) + @JsonSubTypes.Type(name = TaskReportMSQDestination.TYPE, value = TaskReportMSQDestination.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/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 9ef8fe2c1cf..d41a8562269 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 92% 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 3038f5d65dd..ac1254abfe3 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,14 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonCreator; 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 +42,5 @@ public class TaskReportMSQDestination implements MSQDestination { 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 f9832670801..2539dafcc9f 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 4952ee1ab2a..78b3e16f670 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 74b7a9aa11d..998e0429557 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.CounterTracker; 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 fd06d1d79f7..c12c5540573 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.Sequences; 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 d333edb7d2d..39843db2ba7 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 class StageDefinition @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 00000000000..30c20fe22da --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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 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; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.BaseFrameProcessorFactory; + +import 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 + ) + { + // 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 Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); + + for (final ReadablePartition partition : slice.getPartitions()) { + outputChannels.computeIfAbsent( + partition.getPartitionNumber(), + i -> { + try { + return outputChannelFactory.openChannel(i); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ); + } + + final Sequence readableInputs = + Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); + + final Sequence> processors = readableInputs.map( + readableInput -> { + final OutputChannel outputChannel = + outputChannels.get(readableInput.getStagePartition().getPartitionNumber()); + + return new QueryResultsFrameProcessor( + readableInput.getChannel(), + outputChannel.getWritableChannel() + ); + } + ); + + return new ProcessorsAndChannels<>( + processors, + OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values())) + ); + + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/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 00000000000..c945fd33e7c --- /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 81% 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 8e8315d9024..c0e892b99bf 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.io.Closer; 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 @@ import java.util.concurrent.Executors; /** * 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 class DurableStorageInputChannelFactory implements InputChannelFactory 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 class DurableStorageInputChannelFactory implements InputChannelFactory 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,30 @@ public class DurableStorageInputChannelFactory implements InputChannelFactory successfulTaskId ); - return DurableStorageUtils.getPartitionOutputsFileNameForPartition( + return getPartitionOutputsFileNameWithPathForPartition( controllerTaskId, stageNumber, workerNo, - successfulTaskId, - partitionNumber + partitionNumber, + successfulTaskId ); } + + /** + * Get the filePath with filename for the partitioned output of the controller, stage, worker + */ + public abstract String getPartitionOutputsFileNameWithPathForPartition( + String controllerTaskId, + int stageNumber, + int workerNo, + int partitionNumber, + 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 new file mode 100644 index 00000000000..b29220e790d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageQueryResultsInputChannelFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.shuffle.input; + +import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.storage.StorageConnector; + +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 +{ + + public DurableStorageQueryResultsInputChannelFactory( + String controllerTaskId, + StorageConnector storageConnector, + ExecutorService remoteInputStreamPool + ) + { + super(controllerTaskId, storageConnector, remoteInputStreamPool); + } + + @Override + public String getPartitionOutputsFileNameWithPathForPartition( + String controllerTaskId, + int stageNumber, + int workerNo, + int partitionNumber, + String successfulTaskId + ) + { + return DurableStorageUtils.getQueryResultsFileNameWithPathForPartition(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 00000000000..1034b402679 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageStageInputChannelFactory.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.shuffle.input; + +import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.storage.StorageConnector; + +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( + String controllerTaskId, + StorageConnector storageConnector, + ExecutorService remoteInputStreamPool + ) + { + super(controllerTaskId, storageConnector, remoteInputStreamPool); + } + + @Override + public String getPartitionOutputsFileNameWithPathForPartition( + String controllerTaskId, + int stageNumber, + int workerNo, + int partitionNumber, + String successfulTaskId + ) + { + return DurableStorageUtils.getPartitionOutputsFileNameWithPathForPartition( + 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 cfb7c64a1a7..f583596bb88 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 00000000000..009bfd973a4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageOutputChannelFactory.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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(DurableStorageOutputChannelFactory.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 DurableStorageQueryResultsOutputChannelFactory( + controllerTaskId, + workerNumber, + stageNumber, + taskId, + frameSize, + storageConnector, + tmpDir + ); + } else { + return new DurableStorageTaskOutputChannelFactory( + 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(); + } + + /** + * Get filepath to write success file in. + */ + public abstract String getSuccessFilePath(); + + @Override + public OutputChannel openNilChannel(int 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()) + .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 + ); + } + } + + /** + * 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 new file mode 100644 index 00000000000..271129aa435 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/DurableStorageQueryResultsOutputChannelFactory.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.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; + +/** + * Used to write out select query results to durable storage. + */ +public class DurableStorageQueryResultsOutputChannelFactory extends DurableStorageOutputChannelFactory +{ + + public DurableStorageQueryResultsOutputChannelFactory( + String controllerTaskId, + int workerNumber, + int stageNumber, + String taskId, + int frameSize, + StorageConnector storageConnector, + File tmpDir + ) + { + super(controllerTaskId, workerNumber, stageNumber, taskId, frameSize, storageConnector, tmpDir); + } + + @Override + public String getSuccessFilePath() + { + return DurableStorageUtils.getQueryResultsSuccessFilePath( + controllerTaskId, + stageNumber, + workerNumber + ); + } + + @Override + protected String getFileNameWithPathForPartition(int partitionNumber) + { + return DurableStorageUtils.getQueryResultsFileNameWithPathForPartition(controllerTaskId, + stageNumber, + workerNumber, + taskId, + partitionNumber); + } + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + final String fileName = getFileNameWithPathForPartition(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", DurableStorageQueryResultsOutputChannelFactory.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/DurableStorageTaskOutputChannelFactory.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/DurableStorageTaskOutputChannelFactory.java index ca7041cfa00..7ece0dd3155 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/DurableStorageTaskOutputChannelFactory.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,67 @@ import org.apache.druid.frame.channel.WritableFrameFileChannel; 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 +/** + * 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 { - 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 DurableStorageTaskOutputChannelFactory( + 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")); - } - - /** - * 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 - ) - { - return new DurableStorageOutputChannelFactory( - controllerTaskId, - workerNumber, - stageNumber, - taskId, - frameSize, - storageConnector, - tmpDir - ); + super(controllerTaskId, workerNumber, stageNumber, taskId, frameSize, storageConnector, tmpDir); } @Override - public OutputChannel openChannel(int partitionNumber) throws IOException + public String getSuccessFilePath() { - final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition( + return DurableStorageUtils.getWorkerOutputSuccessFilePath(controllerTaskId, stageNumber, workerNumber); + } + + @Override + protected String getFileNameWithPathForPartition(int partitionNumber) + { + return DurableStorageUtils.getPartitionOutputsFileNameWithPathForPartition( controllerTaskId, stageNumber, workerNumber, taskId, partitionNumber ); + } + + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + final String fileName = getFileNameWithPathForPartition(partitionNumber); final WritableFrameFileChannel writableChannel = new WritableFrameFileChannel( FrameFileWriter.open( @@ -211,8 +178,8 @@ public class DurableStorageOutputChannelFactory implements OutputChannelFactory 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 +206,4 @@ public class DurableStorageOutputChannelFactory implements OutputChannelFactory ); } - @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 c0de08a809f..a5dc579419a 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.Granularities; 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.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; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.QueryContext; @@ -229,8 +231,14 @@ public class MSQTaskQueryMaker implements QueryMaker if (ctxDestination != null && !DESTINATION_REPORT.equals(ctxDestination)) { throw new IAE("Cannot SELECT with destination [%s]", ctxDestination); } - - destination = TaskReportMSQDestination.instance(); + 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 { + throw new IAE("Cannot SELECT with destination [%s]", msqSelectDestination.name()); + } } final Map nativeQueryContextOverrides = new HashMap<>(); 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 2754c52f1fe..1c212f275ee 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.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.Objects; /** @@ -100,4 +101,18 @@ public class PageInformation ", 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/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java index e131fa85c73..84292d70559 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 class 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 class ResultSetInformation @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 ce30284e593..2be55a2f840 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 @@ package org.apache.druid.msq.sql.resources; 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,27 @@ import org.apache.druid.common.guava.FutureUtils; 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.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; +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 +59,10 @@ import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; 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 +79,8 @@ import org.apache.druid.sql.SqlStatementFactory; 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 +96,13 @@ import javax.ws.rs.core.MediaType; 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 +114,7 @@ public class SqlStatementResource private final AuthorizerMapper authorizerMapper; private final ObjectMapper jsonMapper; private final OverlordClient overlordClient; + private final StorageConnector storageConnector; @Inject @@ -105,15 +122,20 @@ public class 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,27 +164,11 @@ public class SqlStatementResource 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 +207,11 @@ public class SqlStatementResource // 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) @@ -258,7 +267,7 @@ public class SqlStatementResource ); } 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)); @@ -286,13 +295,11 @@ public class SqlStatementResource 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 +313,33 @@ public class SqlStatementResource } MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); - SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(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(); - } - - 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.") - ); - } - - 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(); - } - - 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(); + throwIfQueryIsNotSuccessful(queryId, statusPlus); + 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; + results = getResultYielder(queryId, page, msqControllerTask, closer); + 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); } @@ -388,7 +352,7 @@ public class SqlStatementResource ); } 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)); @@ -455,7 +419,7 @@ public class SqlStatementResource ); } 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)); @@ -562,35 +526,74 @@ public class SqlStatementResource .build(); } + @SuppressWarnings("ReassignedVariable") private Optional getSampleResults( - String asyncResultId, - boolean isSelectQuery, + String queryId, 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 + queryId))); + 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 - ) + + // getting the total number of rows, size from page information. + Long rows = null; + Long size = null; + if (pageList.isPresent()) { + rows = 0L; + size = 0L; + 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 = 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( + rows, + size, + null, + dataSource, + results, + isSelectQuery ? pageList.orElse(null) : null ) - )); + ); } else { return Optional.empty(); } @@ -633,9 +636,9 @@ public class SqlStatementResource taskResponse.getStatus().getDuration(), withResults ? getSampleResults( queryId, - signature.isPresent(), msqControllerTask.getDataSource(), - sqlStatementState + sqlStatementState, + msqControllerTask.getQuerySpec().getDestination() ).orElse(null) : null, null )); @@ -664,6 +667,222 @@ public class SqlStatementResource return msqControllerTask; } + private Optional> getResultYielder( + String queryId, + Long page, + MSQControllerTask msqControllerTask, + Closer closer + ) + { + 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 selectedPageId; + if (page != null) { + selectedPageId = getPageInformationForPageId(pages, page).getId(); + } else { + selectedPageId = null; + } + checkForDurableStorageConnectorImpl(); + final DurableStorageInputChannelFactory standardImplementation = DurableStorageInputChannelFactory.createStandardImplementation( + msqControllerTask.getId(), + storageConnector, + closer, + true + ); + results = Optional.of(Yielders.each( + Sequences.concat(pages.stream() + .filter(pageInformation -> selectedPageId == null + || selectedPageId.equals(pageInformation.getId())) + .map(pageInformation -> { + try { + return new FrameChannelSequence(standardImplementation.openChannel( + finalStage.getId(), + (int) pageInformation.getId(), + (int) pageInformation.getId()// we would always have partition number == worker number + )); + } + 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( + "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) + { + 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) { + 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 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() + ) + ); + } + } + private T contactOverlord(final ListenableFuture future) { try { @@ -675,4 +894,6 @@ public class SqlStatementResource .build("Unable to contact overlord " + e.getMessage()); } } + + } 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 3c951c7d0cc..ca364be4c73 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.RFC4180Parser; 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; @@ -39,6 +39,7 @@ import java.io.IOException; 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,10 +218,20 @@ public class MultiStageQueryContext queryContext.getString( CTX_SELECT_DESTINATION, DEFAULT_SELECT_DESTINATION - ) + ).toUpperCase(Locale.ENGLISH) ); } + @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(Locale.ENGLISH)); + } + 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 4c0474e1f93..4211a062b5d 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,54 @@ 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.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.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +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; +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.Collections; 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 +76,7 @@ public class SqlStatementResourceHelper ) { // 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,49 +163,83 @@ public class SqlStatementResourceHelper return last; } - public static Optional> getRowsAndSizeFromPayload(Map payload, boolean isSelectQuery) + /** + * 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 + ) { - List stages = getList(payload, "stages"); - if (stages == null || stages.isEmpty()) { + if (msqTaskReportPayload.getStages() == null || msqTaskReportPayload.getCounters() == null) { 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) { + } + 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(); } - 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"); - } + } 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(); } } + if (rows != 0L) { + return Optional.of(ImmutableList.of(new PageInformation(rows, size, 0))); + } else { + return Optional.empty(); + } - return Optional.of(new Pair<>(rows == -1L ? null : rows + 1, sizeInBytes == -1L ? null : sizeInBytes + 1)); + } else if (msqDestination instanceof DurableStorageMSQDestination) { + 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(); + } + // do not populate a page if the worker generated 0 rows. + if (rows != 0L) { + pageList.add(new PageInformation(rows, size, counterSnapshots.getKey())); + } + } + Collections.sort(pageList, PageInformation.getIDComparator()); + return Optional.of(pageList); + } else { + return Optional.empty(); } } - public static Optional getExceptionPayload( String queryId, TaskStatusResponse taskResponse, @@ -234,6 +294,74 @@ public class SqlStatementResourceHelper )); } + 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"); @@ -247,35 +375,6 @@ public class SqlStatementResourceHelper 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(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/MSQParseExceptionsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java index 7b0a5e52d10..3f4c953bcb5 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.CsvInputFormat; 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 0f4210e7f59..d751946f24a 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.StringUtils; 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.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; import org.apache.druid.msq.test.CounterSnapshotMatcher; import org.apache.druid.msq.test.MSQTestBase; @@ -91,13 +93,37 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; +import java.util.Locale; 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(Locale.ENGLISH) + ) + .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(Locale.ENGLISH) + ) + .build(); + @Parameterized.Parameters(name = "{index}:with context {0}") public static Collection data() { @@ -105,7 +131,9 @@ public class MSQSelectTest extends MSQTestBase {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 +170,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -171,6 +202,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -226,6 +260,9 @@ public class MSQSelectTest extends MSQTestBase .build()) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -291,6 +328,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(expectedColumnMappings) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -350,6 +390,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -381,6 +424,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -413,6 +459,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -455,6 +504,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) @@ -519,6 +571,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setQueryContext(context) @@ -573,6 +628,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setQueryContext(context) @@ -639,6 +697,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) @@ -666,6 +727,9 @@ public class MSQSelectTest extends MSQTestBase .build()) .columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "EXPR$0")))) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{4L})) @@ -719,6 +783,9 @@ public class MSQSelectTest extends MSQTestBase ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows( @@ -761,6 +828,9 @@ public class MSQSelectTest extends MSQTestBase .query(query) .columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "cnt")))) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -929,6 +999,9 @@ public class MSQSelectTest extends MSQTestBase ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -984,6 +1057,9 @@ public class MSQSelectTest extends MSQTestBase ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(rowSignature) @@ -1060,6 +1136,9 @@ public class MSQSelectTest extends MSQTestBase ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(rowSignature) @@ -1134,6 +1213,9 @@ public class MSQSelectTest extends MSQTestBase ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(rowSignature) @@ -1226,6 +1308,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedCountersForStageWorkerChannel( @@ -1336,6 +1421,9 @@ public class MSQSelectTest extends MSQTestBase ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build() ) .setExpectedRowSignature(resultSignature) @@ -1401,6 +1489,9 @@ public class MSQSelectTest extends MSQTestBase ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(expectedResultSignature) .setQueryContext(context) @@ -1469,6 +1560,9 @@ public class MSQSelectTest extends MSQTestBase new ColumnMapping("a0", "col") ))) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setQueryContext(context) .setExpectedRowSignature(resultSignature) @@ -1520,6 +1614,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(expectedMultiValueFooRowsGroup()) @@ -1590,6 +1687,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows( @@ -1657,6 +1757,9 @@ public class MSQSelectTest extends MSQTestBase ) ) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(expected) @@ -1773,6 +1876,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows( @@ -1824,6 +1930,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) @@ -1831,7 +1940,7 @@ public class MSQSelectTest extends MSQTestBase 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 +1949,7 @@ public class MSQSelectTest extends MSQTestBase } @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 +2004,9 @@ public class MSQSelectTest extends MSQTestBase ) )) .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(isDurableStorageDestination() + ? DurableStorageMSQDestination.INSTANCE + : TaskReportMSQDestination.INSTANCE) .build()) .setQueryContext(context) .setExpectedResultRows(result) @@ -2008,4 +2057,9 @@ public class MSQSelectTest extends MSQTestBase .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 5063c054777..21e2d7ffaa3 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 @@ package org.apache.druid.msq.indexing; 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 7f7bf4fbfd1..3da4adcbc3f 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.TaskMaster; 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 33e0d8c2486..6aaf21ee3bd 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 @@ package org.apache.druid.msq.indexing; 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 b0dbacee242..c88f22d1b74 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.jackson.DefaultObjectMapper; 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/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 00000000000..70700c026d6 --- /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 c17c916a94f..38d99dfe03d 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.DurableStorageTaskOutputChannelFactory; import org.apache.druid.storage.local.LocalFileStorageConnector; import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; @@ -35,7 +36,7 @@ public class DurableStorageOutputChannelFactoryTest extends OutputChannelFactory throws IOException { super( - new DurableStorageOutputChannelFactory( + new DurableStorageTaskOutputChannelFactory( "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 similarity index 72% 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 51e10a93b2d..e5133ee3d21 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,13 +20,13 @@ 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; 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; @@ -36,25 +36,29 @@ import org.apache.druid.msq.sql.entity.SqlStatementResult; 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.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() { @@ -62,14 +66,15 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase sqlStatementFactory, CalciteTests.TEST_AUTHORIZER_MAPPER, objectMapper, - indexingServiceClient + indexingServiceClient, + localFileStorageConnector ); } @Test public void testMSQSelectQueryTest() throws IOException { - List results = ImmutableList.of( + List results = ImmutableList.of( new Object[]{1L, ""}, new Object[]{ 1L, @@ -117,19 +122,16 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase 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()) + ); } @@ -247,6 +249,83 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase ).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 6a19f3f792e..51b64b7eedd 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.StringUtils; 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.server.security.AuthenticationResult; 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 javax.ws.rs.core.Response; 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( @@ -320,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 { @@ -388,10 +407,6 @@ public class SqlStatementResourceTest extends MSQTestBase ))); - 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)), @@ -627,8 +642,9 @@ public class SqlStatementResourceTest extends MSQTestBase resource = new SqlStatementResource( sqlStatementFactory, CalciteTests.TEST_AUTHORIZER_MAPPER, - JSON_MAPPER, - overlordClient + objectMapper, + overlordClient, + new LocalFileStorageConnector(tmpFolder.newFolder("local")) ); } @@ -704,24 +720,22 @@ public class SqlStatementResourceTest extends MSQTestBase { 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 +807,18 @@ public class SqlStatementResourceTest extends MSQTestBase 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 +905,28 @@ public class SqlStatementResourceTest extends MSQTestBase @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 8e40d8daf1a..67b77fa51b3 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 class ResultSetInformationTest 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 1409450c697..88cff552a71 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 class SqlStatementResultTest + " 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 736ec2f430d..68965f40bfe 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.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.frame.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.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.input.InputSourceModule; @@ -85,9 +87,12 @@ import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.guice.MSQSqlModule; import org.apache.druid.msq.guice.MultiStageQuery; -import org.apache.druid.msq.indexing.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.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.error.TooManyAttemptsForWorker; 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.HashMap; 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,53 @@ public class MSQTestBase extends BaseCalciteQueryTest 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(); + Closer closer = Closer.create(); + InputChannelFactory inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation( + controllerId, + localFileStorageConnector, + closer, + true + ); + rows = new FrameChannelSequence(inputChannelFactory.openChannel( + finalStage.getId(), + 0, + 0 + )).flatMap(frame -> SqlStatementResourceHelper.getResultSequence( + msqControllerTask, + finalStage, + frame, + objectMapper + )).withBaggage(closer).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 567e6d8a1f7..16aad3d7b93 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 @@ package org.apache.druid.msq.util; 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 1585b865157..3e5f2fe00a1 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,17 @@ public class DurableStorageUtils 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 +67,24 @@ public class DurableStorageUtils 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 +98,21 @@ public class DurableStorageUtils ); } + + 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,11 +131,25 @@ public class DurableStorageUtils ); } + + 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 */ - public static String getPartitionOutputsFileNameForPartition( + public static String getPartitionOutputsFileNameWithPathForPartition( final String controllerTaskId, final int stageNumber, final int workerNumber, @@ -111,6 +164,21 @@ public class DurableStorageUtils ); } + public static String getQueryResultsFileNameWithPathForPartition( + 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 00000000000..e63d360c2dd --- /dev/null +++ b/processing/src/main/java/org/apache/druid/storage/NilStorageConnector.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.storage; + +import org.apache.druid.error.DruidException; + +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) + { + throw notConfiguredException(); + } + + @Override + public InputStream read(String path) + { + throw notConfiguredException(); + + } + + @Override + public InputStream readRange(String path, long from, long size) + { + throw notConfiguredException(); + + } + + @Override + public OutputStream write(String path) + { + throw notConfiguredException(); + } + + @Override + public void deleteFile(String path) + { + throw notConfiguredException(); + } + + @Override + public void deleteFiles(Iterable paths) + { + throw notConfiguredException(); + } + + @Override + public void deleteRecursively(String path) + { + throw notConfiguredException(); + } + + @Override + public Iterator listDir(String dirName) + { + 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/frame/util/DurableStorageUtilsTest.java b/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java index 71b16633e92..2f01a402db4 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 @@ import org.junit.Test; 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 class DurableStorageUtilsTest 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.getPartitionOutputsFileNameWithPathForPartition( + CONTROLLER_ID, + STAGE_NUMBER, + WORKER_NUMBER, + TASK_ID, + PARTITION_NUMBER + ) + ); + Assert.assertEquals( + DurableStorageUtils.QUERY_RESULTS_DIR + "/" + baseString + "taskId_" + TASK_ID + "/part_3", + DurableStorageUtils.getQueryResultsFileNameWithPathForPartition( + 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 00000000000..8dfcb1b78f1 --- /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.error.DruidException; +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, 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")); + } + +}