Adding Ability for MSQ to write select results to durable storage. (#14527)

One of the most requested features in druid is to have an ability to download big result sets.
As part of #14416 , we added an ability for MSQ to be queried via a query friendly endpoint. This PR builds upon that work and adds the ability for MSQ to write select results to durable storage.

We write the results to the durable storage location <prefix>/results/<queryId> in the druid frame format. This is exposed to users by
/v2/sql/statements/:queryId/results.
This commit is contained in:
Karan Kumar 2023-07-07 20:49:48 +05:30 committed by GitHub
parent 40d0dc9e0e
commit afa8c7b8ab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
62 changed files with 2256 additions and 619 deletions

View File

@ -91,6 +91,13 @@ public class CounterSnapshotsTree
return retVal;
}
public Map<Integer, CounterSnapshots> snapshotForStage(int stageNumber)
{
synchronized (snapshotsMap) {
return snapshotsMap.getOrDefault(stageNumber, null);
}
}
private void putAll(final Map<Integer, Map<Integer, CounterSnapshots>> otherMap)
{
synchronized (snapshotsMap) {

View File

@ -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

View File

@ -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);

View File

@ -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<String, Object> 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<ColumnValueSelector> selectors =
columnMappings.getMappings()
.stream()
.map(
mapping ->
columnSelectorFactory.makeColumnValueSelector(mapping.getQueryColumn())
).collect(Collectors.toList());
final List<SqlTypeName> sqlTypeNames = task.getSqlTypeNames();
Iterable<Object[]> retVal = () -> new Iterator<Object[]>()
{
@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);

View File

@ -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()));

View File

@ -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()
);
}
}

View File

@ -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
}
}

View File

@ -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,

View File

@ -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;

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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<ClusterByStatisticsSnapshot>() {})
holder -> deserialize(holder, new TypeReference<ClusterByStatisticsSnapshot>()
{
})
);
}
@ -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<ClusterByStatisticsSnapshot>() {})
holder -> deserialize(holder, new TypeReference<ClusterByStatisticsSnapshot>()
{
})
);
}
@ -204,7 +212,9 @@ public class IndexerWorkerClient implements WorkerClient
new RequestBuilder(HttpMethod.GET, "/counters"),
new BytesFullResponseHandler()
),
holder -> deserialize(holder, new TypeReference<CounterSnapshotsTree>() {})
holder -> deserialize(holder, new TypeReference<CounterSnapshotsTree>()
{
})
);
}
@ -299,7 +309,7 @@ public class IndexerWorkerClient implements WorkerClient
/**
* Deserialize a {@link BytesFullResponseHolder} as JSON.
*
* <p>
* It would be reasonable to move this to {@link BytesFullResponseHolder} itself, or some shared utility class.
*/
private <T> T deserialize(final BytesFullResponseHolder bytesHolder, final TypeReference<T> typeReference)

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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{}";
}
}

View File

@ -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
{

View File

@ -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.

View File

@ -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{}";
}
}

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -277,7 +277,7 @@ public class StageDefinition
@JsonProperty("shuffleCheckHasMultipleValues")
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
boolean getShuffleCheckHasMultipleValues()
public boolean getShuffleCheckHasMultipleValues()
{
return shuffleCheckHasMultipleValues;
}

View File

@ -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<FrameProcessor<Long>, Long> makeProcessors(
StageDefinition stageDefinition,
int workerNumber,
List<InputSlice> inputSlices,
InputSliceReader inputSliceReader,
@Nullable Object extra,
OutputChannelFactory outputChannelFactory,
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> 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<OutputChannel> 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<ReadableInput> readableInputs =
Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher));
final Sequence<FrameProcessor<Long>> 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()))
);
}
}

View File

@ -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>
{
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<ReadableFrameChannel> inputChannels()
{
return Collections.singletonList(inChannel);
}
@Override
public List<WritableFrameChannel> outputChannels()
{
return Collections.singletonList(outChannel);
}
@Override
public ReturnOrAwait<Long> 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();
}
}

View File

@ -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);
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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);
}

View File

@ -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());
}
}

View File

@ -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();
}
}

View File

@ -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<String, Object> nativeQueryContextOverrides = new HashMap<>();

View File

@ -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<PageInformation> getIDComparator()
{
return new PageComparator();
}
public static class PageComparator implements Comparator<PageInformation>
{
@Override
public int compare(PageInformation s1, PageInformation s2)
{
return Long.compare(s1.getId(), s2.getId());
}
}
}

View File

@ -38,7 +38,7 @@ public class ResultSetInformation
@Nullable
private final ResultFormat resultFormat;
@Nullable
private final List<Object> records;
private final List<Object[]> 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<Object> records,
@JsonProperty("sampleRecords") @Nullable List<Object[]> records,
@JsonProperty("pages") @Nullable List<PageInformation> pages
)
{
@ -97,7 +97,7 @@ public class ResultSetInformation
@JsonProperty("sampleRecords")
@Nullable
@JsonInclude(JsonInclude.Include.NON_NULL)
public List<Object> getRecords()
public List<Object[]> getRecords()
{
return records;
}

View File

@ -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<List<ColumnNameAndTypes>> 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<List<Object>> 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<ColumnNameAndTypes> 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<List<ColumnNameAndTypes>> 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<Yielder<Object[]>> 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<ResultSetInformation> getSampleResults(
String asyncResultId,
boolean isSelectQuery,
String queryId,
String dataSource,
SqlStatementState sqlStatementState
SqlStatementState sqlStatementState,
MSQDestination msqDestination
)
{
if (sqlStatementState == SqlStatementState.SUCCESS) {
Map<String, Object> payload = SqlStatementResourceHelper.getPayload(contactOverlord(overlordClient.taskReportAsMap(
asyncResultId)));
Optional<Pair<Long, Long>> rowsAndSize = SqlStatementResourceHelper.getRowsAndSizeFromPayload(
payload,
isSelectQuery
queryId)));
MSQTaskReportPayload msqTaskReportPayload = jsonMapper.convertValue(payload, MSQTaskReportPayload.class);
Optional<List<PageInformation>> 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<Object[]> results = null;
if (isSelectQuery) {
results = new ArrayList<>();
Yielder<Object[]> 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<Yielder<Object[]>> getResultYielder(
String queryId,
Long page,
MSQControllerTask msqControllerTask,
Closer closer
)
{
final Optional<Yielder<Object[]>> 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<PageInformation> 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<PageInformation> 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<List<ColumnNameAndTypes>> signature,
Closer closer,
Optional<Yielder<Object[]>> results,
CountingOutputStream os
) throws IOException
{
try {
try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) {
Yielder<Object[]> yielder = results.get();
List<ColumnNameAndTypes> 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> T contactOverlord(final ListenableFuture<T> future)
{
try {
@ -675,4 +894,6 @@ public class SqlStatementResource
.build("Unable to contact overlord " + e.getMessage());
}
}
}

View File

@ -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);

View File

@ -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<SqlTypeName> sqlTypeNames = msqControllerTask.getSqlTypeNames();
if (sqlTypeNames == null || sqlTypeNames.size() != columnMappings.size()) {
@ -137,49 +163,83 @@ public class SqlStatementResourceHelper
return last;
}
public static Optional<Pair<Long, Long>> getRowsAndSizeFromPayload(Map<String, Object> payload, boolean isSelectQuery)
/**
* Populates pages list from the {@link CounterSnapshotsTree}.
* <br>
* The number of pages changes with respect to the destination
* <ol>
* <li>{@link DataSourceMSQDestination} a single page is returned which adds all the counters of {@link SegmentGenerationProgressCounter.Snapshot}</li>
* <li>{@link TaskReportMSQDestination} a single page is returned which adds all the counters of {@link ChannelCounters}</li>
* <li>{@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</li>
* </ol>
*/
public static Optional<List<PageInformation>> 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<String, Object> 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<Integer, CounterSnapshots> 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<String, Object> 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<Integer> rowsPerChannel = (List<Integer>) ((Map<String, Object>) output).get("rows");
List<Integer> bytesPerChannel = (List<Integer>) ((Map<String, Object>) 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<String, Object>) 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<PageInformation> pageList = new ArrayList<>();
for (Map.Entry<Integer, CounterSnapshots> 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<SqlStatementResult> getExceptionPayload(
String queryId,
TaskStatusResponse taskResponse,
@ -234,6 +294,74 @@ public class SqlStatementResourceHelper
));
}
public static Sequence<Object[]> 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<ColumnValueSelector> selectors = columnMappings.getMappings()
.stream()
.map(mapping -> columnSelectorFactory.makeColumnValueSelector(
mapping.getQueryColumn()))
.collect(Collectors.toList());
final List<SqlTypeName> sqlTypeNames = msqControllerTask.getSqlTypeNames();
Iterable<Object[]> retVal = () -> new Iterator<Object[]>()
{
@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<String, Object> getQueryExceptionDetails(Map<String, Object> payload)
{
return getMap(getMap(payload, "status"), "errorReport");
@ -247,35 +375,6 @@ public class SqlStatementResourceHelper
return (Map<String, Object>) map.get(key);
}
@SuppressWarnings("rawtypes")
public static List getList(Map<String, Object> map, String key)
{
if (map == null) {
return null;
}
return (List) map.get(key);
}
/**
* Get results from report
*/
@SuppressWarnings("unchecked")
public static Optional<List<Object>> getResults(Map<String, Object> payload)
{
Map<String, Object> resultsHolder = getMap(payload, "results");
if (resultsHolder == null) {
return Optional.empty();
}
List<Object> data = (List<Object>) resultsHolder.get("results");
List<Object> rows = new ArrayList<>();
if (data != null) {
rows.addAll(data);
}
return Optional.of(rows);
}
public static Map<String, Object> getPayload(Map<String, Object> results)
{
Map<String, Object> msqReport = getMap(results, "multiStageQuery");

View File

@ -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;

View File

@ -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<String, Object> QUERY_RESULTS_WITH_DURABLE_STORAGE_CONTEXT =
ImmutableMap.<String, Object>builder()
.putAll(DURABLE_STORAGE_MSQ_CONTEXT)
.put(
MultiStageQueryContext.CTX_SELECT_DESTINATION,
MSQSelectDestination.DURABLE_STORAGE.name().toLowerCase(Locale.ENGLISH)
)
.build();
public static final Map<String, Object> QUERY_RESULTS_WITH_DEFAULT_CONTEXT =
ImmutableMap.<String, Object>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<Object[]> 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<Object[]> result = new ArrayList<>();
for (int i = 0; i < Limits.MAX_SELECT_RESULT_ROWS; ++i) {
result.add(new Object[]{1});
}
Map<String, Object> 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);
}
}

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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<Frame> 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<Long> retVal = exec.runFully(processor, null);
final Sequence<List<Object>> rowsFromProcessor = FrameTestUtil.readRowsFromFrameChannel(
outputChannel.readable(),
FrameReader.create(signature)
);
FrameTestUtil.assertRowsEqual(
FrameTestUtil.readRowsFromAdapter(adapter, signature, false),
rowsFromProcessor
);
Assert.assertEquals(adapter.getNumRows(), (long) retVal.get());
}
}

View File

@ -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,

View File

@ -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<Object> results = ImmutableList.of(
List<Object[]> 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<List<Object>>()
{
}
),
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<String, Object> 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<String, Object> 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<Map<String, Object>> 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<String, Object> defaultAsyncContext()
{

View File

@ -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

View File

@ -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<Object[]> as a field, we cannot call equals method after deserialization.
Assert.assertEquals(JSON_STRING_1, MAPPER.writeValueAsString(RESULTS_1));
}
}

View File

@ -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()
);

View File

@ -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<Object[]> rows = getRows(payload.getResults());
MSQControllerTask msqControllerTask = indexingServiceClient.getMSQControllerTask(controllerId);
final MSQSpec spec = msqControllerTask.getQuerySpec();
final List<Object[]> 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);

View File

@ -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;

View File

@ -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

View File

@ -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<String> paths)
{
throw notConfiguredException();
}
@Override
public void deleteRecursively(String path)
{
throw notConfiguredException();
}
@Override
public Iterator<String> listDir(String dirName)
{
throw notConfiguredException();
}
private DruidException notConfiguredException()
{
return DruidException.forPersona(DruidException.Persona.DEVELOPER)
.ofCategory(DruidException.Category.DEFENSIVE)
.build("Please configure durable storage.");
}
}

View File

@ -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
)
);
}
}

View File

@ -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"));
}
}