From 41bb6609880c51e78e1ed7b92c3d37df7f1441eb Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 7 Nov 2022 19:43:17 +0530 Subject: [PATCH 01/27] Initial commit. --- .../apache/druid/msq/exec/ControllerImpl.java | 274 ++++++++++++++++-- .../org/apache/druid/msq/exec/Limits.java | 10 + .../org/apache/druid/msq/exec/MSQTasks.java | 7 +- .../org/apache/druid/msq/exec/WorkerImpl.java | 20 +- .../druid/msq/exec/WorkerManagerClient.java | 2 +- .../indexing/IndexerWorkerManagerClient.java | 6 +- .../druid/msq/indexing/MSQWorkerTask.java | 45 ++- .../msq/indexing/MSQWorkerTaskLauncher.java | 204 ++++++++++++- .../apache/druid/msq/indexing/RetryTask.java | 28 ++ .../msq/indexing/error/BaseMSQFault.java | 8 +- .../msq/indexing/error/CanceledFault.java | 2 +- .../msq/indexing/error/MSQException.java | 6 +- .../druid/msq/indexing/error/MSQFault.java | 10 - .../msq/indexing/error/MSQFaultUtils.java | 46 +++ .../error/TooManyWorkerRetriesFault.java | 113 ++++++++ .../error/TotalRetryLimitExceededFault.java | 114 ++++++++ .../indexing/error/WorkerRpcFailedFault.java | 2 +- .../controller/ControllerQueryKernel.java | 104 ++++++- .../controller/ControllerStagePhase.java | 17 +- .../controller/ControllerStageTracker.java | 188 +++++++++++- .../apache/druid/msq/exec/MSQTasksTest.java | 8 +- .../msq/indexing/error/MSQFaultSerdeTest.java | 2 + .../BaseControllerQueryKernelTest.java | 45 ++- .../ControllerQueryKernelTests.java | 36 ++- .../apache/druid/msq/test/MSQTestBase.java | 9 +- .../msq/test/MSQTestControllerContext.java | 6 +- .../druid/indexing/common/task/Task.java | 2 +- 27 files changed, 1192 insertions(+), 122 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriesFault.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRetryLimitExceededFault.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 3aa346c38463..5e392a67bace 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -26,12 +26,16 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntArraySet; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.StringTuple; @@ -101,11 +105,13 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; import org.apache.druid.msq.indexing.error.TooManyWarningsFault; import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQStagesReport; import org.apache.druid.msq.indexing.report.MSQStatusReport; @@ -215,11 +221,13 @@ public class ControllerImpl implements Controller private final MSQControllerTask task; private final ControllerContext context; + private final boolean isDurableStorageEnabled; + /** * Queue of "commands" to run on the {@link ControllerQueryKernel}. Various threads insert into the queue * using {@link #addToKernelManipulationQueue}. The main thread running {@link RunQueryUntilDone#run()} reads * from the queue and executes the commands. - * + *

* This ensures that all manipulations on {@link ControllerQueryKernel}, and all core logic, are run in * a single-threaded manner. */ @@ -258,6 +266,9 @@ public class ControllerImpl implements Controller // Time at which the query started. // For live reports. Written by the main controller thread, read by HTTP threads. + + // WorkerNumber -> WorkOrders + private final ConcurrentHashMap> workOrdersToRetry = new ConcurrentHashMap<>(); private volatile DateTime queryStartTime = null; private volatile DruidNode selfDruidNode; @@ -273,6 +284,9 @@ public ControllerImpl( { this.task = task; this.context = context; + this.isDurableStorageEnabled = + MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext()); + } @Override @@ -490,14 +504,14 @@ public TaskStatus runTask(final Closer closer) return TaskStatus.success(id()); } else { // errorForReport is nonnull when taskStateForReport != SUCCESS. Use that message. - return TaskStatus.failure(id(), errorForReport.getFault().getCodeWithMessage()); + return TaskStatus.failure(id(), MSQFaultUtils.generateMessageWithErrorCode(errorForReport.getFault())); } } /** * Adds some logic to {@link #kernelManipulationQueue}, where it will, in due time, be executed by the main * controller loop in {@link RunQueryUntilDone#run()}. - * + *

* If the consumer throws an exception, the query fails. */ private void addToKernelManipulationQueue(Consumer kernelConsumer) @@ -517,8 +531,6 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); closer.register(netClient::close); - final boolean isDurableStorageEnabled = - MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext()); final QueryDefinition queryDef = makeQueryDefinition( id(), @@ -531,10 +543,20 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) log.debug("Query [%s] durable storage mode is set to %s.", queryDef.getQueryId(), isDurableStorageEnabled); + this.workerTaskLauncher = new MSQWorkerTaskLauncher( id(), task.getDataSource(), context, + (failedTask, fault) -> { + addToKernelManipulationQueue((kernel) -> { + if (isDurableStorageEnabled) { + addToRetryQueue(kernel, failedTask.getWorkerNumber(), fault); + } else { + throw new MSQException(fault); + } + }); + }, isDurableStorageEnabled, // 10 minutes +- 2 minutes jitter @@ -557,6 +579,25 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) return queryDef; } + private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault fault) + { + List retriableWorkOrders = kernel.getRetriableWorkOrders(worker, fault); + if (retriableWorkOrders.size() != 0) { + log.info("Submitting worker[%s] for relaunch because of fault[%s]", worker, fault); + workerTaskLauncher.submitForRelaunch(worker); + workOrdersToRetry.compute(worker, (workerNumber, workOrders) -> { + if (workOrders == null) { + Set orders = ConcurrentHashMap.newKeySet(); + orders.addAll(retriableWorkOrders); + return orders; + } else { + workOrders.addAll(retriableWorkOrders); + return workOrders; + } + }); + } + } + /** * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages. */ @@ -596,6 +637,7 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics @Override public void workerError(MSQErrorReport errorReport) { + // move inside kernel if (!workerTaskLauncher.isTaskCanceledByController(errorReport.getTaskId())) { workerErrorRef.compareAndSet(null, errorReport); } @@ -897,7 +939,7 @@ private List generateSegmentIdsWithShardSpecsForReplace( /** * Returns a complete list of task ids, ordered by worker number. The Nth task has worker number N. - * + *

* If the currently-running set of tasks is incomplete, returns an absent Optional. */ @Override @@ -907,7 +949,7 @@ public List getTaskIds() return Collections.emptyList(); } - return workerTaskLauncher.getTaskList(); + return workerTaskLauncher.getActiveTasks(); } @SuppressWarnings({"unchecked", "rawtypes"}) @@ -982,6 +1024,13 @@ private void contactWorkersForStage(final TaskContactFn contactFn, final IntSet final List taskIds = getTaskIds(); final List> taskFutures = new ArrayList<>(workers.size()); + try { + workerTaskLauncher.waitUntilWorkersReady(workers); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + for (int workerNumber : workers) { final String taskId = taskIds.get(workerNumber); taskFutures.add(contactFn.contactTask(netClient, taskId, workerNumber)); @@ -990,6 +1039,7 @@ private void contactWorkersForStage(final TaskContactFn contactFn, final IntSet FutureUtils.getUnchecked(MSQFutureUtils.allAsList(taskFutures, true), true); } + private void startWorkForStage( final QueryDefinition queryDef, final ControllerQueryKernel queryKernel, @@ -1005,35 +1055,100 @@ private void startWorkForStage( ); final Int2ObjectMap workOrders = queryKernel.createWorkOrders(stageNumber, extraInfos); + final StageId stageId = new StageId(queryDef.getQueryId(), stageNumber); + + Set> workOrdersNotSent = ConcurrentHashMap.newKeySet(); contactWorkersForStage( - (netClient, taskId, workerNumber) -> netClient.postWorkOrder(taskId, workOrders.get(workerNumber)), + (netClient, taskId, workerNumber) -> { + queryKernel.workOrdersSentForWorker(stageId, workerNumber); + SettableFuture settableFuture = SettableFuture.create(); + ListenableFuture future = netClient.postWorkOrder(taskId, workOrders.get(workerNumber)); + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Void result) + { + settableFuture.set(true); + } + + @Override + public void onFailure(Throwable t) + { + if (isDurableStorageEnabled) { + settableFuture.setException(new MSQException(t, new WorkerRpcFailedFault(taskId))); + } else { + workOrdersNotSent.add(new Pair<>(workerNumber, taskId)); + settableFuture.set(false); + } + } + }); + + return settableFuture; + }, workOrders.keySet() ); + + + for (Pair workerIdTask : workOrdersNotSent) { + addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); + } } private void postResultPartitionBoundariesForStage( + final ControllerQueryKernel queryKernel, final QueryDefinition queryDef, final int stageNumber, final ClusterByPartitions resultPartitionBoundaries, final IntSet workers ) { + final StageId stageId = new StageId(queryDef.getQueryId(), stageNumber); + + Set> partitionBoundariesNotSent = ConcurrentHashMap.newKeySet(); contactWorkersForStage( - (netClient, taskId, workerNumber) -> - netClient.postResultPartitionBoundaries( - taskId, - new StageId(queryDef.getQueryId(), stageNumber), - resultPartitionBoundaries - ), + (netClient, taskId, workerNumber) -> { + queryKernel.partitionBoundariesSentForWorker(stageId, workerNumber); + SettableFuture settableFuture = SettableFuture.create(); + ListenableFuture future = netClient.postResultPartitionBoundaries( + taskId, + new StageId(queryDef.getQueryId(), stageNumber), + resultPartitionBoundaries + ); + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Void result) + { + settableFuture.set(true); + } + + @Override + public void onFailure(Throwable t) + { + if (isDurableStorageEnabled) { + settableFuture.setException(new MSQException(t, new WorkerRpcFailedFault(taskId))); + } else { + partitionBoundariesNotSent.add(new Pair<>(workerNumber, taskId)); + settableFuture.set(false); + } + + } + }); + return settableFuture; + }, workers ); + + for (Pair workerIdTask : partitionBoundariesNotSent) { + addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); + } } /** * Publish the list of segments. Additionally, if {@link DataSourceMSQDestination#isReplaceTimeChunks()}, * also drop all other segments within the replacement intervals. - * + *

* If any existing segments cannot be dropped because their intervals are not wholly contained within the * replacement parameter, throws a {@link MSQException} with {@link InsertCannotReplaceExistingSegmentFault}. */ @@ -1128,10 +1243,9 @@ private List findIntervalsToDrop(final Set publishedSegme private CounterSnapshotsTree getCountersFromAllTasks() { final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); - final List taskList = workerTaskLauncher.getTaskList(); + final List taskList = getTaskIds(); final List> futures = new ArrayList<>(); - for (String taskId : taskList) { futures.add(netClient.getCounters(taskId)); } @@ -1148,7 +1262,7 @@ private CounterSnapshotsTree getCountersFromAllTasks() private void postFinishToAllTasks() { - final List taskList = workerTaskLauncher.getTaskList(); + final List taskList = getTaskIds(); final List> futures = new ArrayList<>(); @@ -1287,7 +1401,7 @@ private void publishSegmentsIfNeeded( /** * Clean up durable storage, if used for stage output. - * + *

* Note that this is only called by the controller task itself. It isn't called automatically by anything in * particular if the controller fails early without being able to run its cleanup routines. This can cause files * to be left in durable storage beyond their useful life. @@ -1471,7 +1585,7 @@ private static GranularitySpec makeGranularitySpecForIngestion( /** * Checks that a {@link GroupByQuery} is grouping on the primary time column. - * + *

* The logic here is roundabout. First, we check which column in the {@link GroupByQuery} corresponds to the * output column {@link ColumnHolder#TIME_COLUMN_NAME}, using our {@link ColumnMappings}. Then, we check for the * presence of an optimization done in {@link DruidQuery#toGroupByQuery()}, where the context parameter @@ -1491,9 +1605,9 @@ private static boolean timeIsGroupByDimension(GroupByQuery groupByQuery, ColumnM /** * Whether a native query represents an ingestion with rollup. - * + *

* Checks for three things: - * + *

* - The query must be a {@link GroupByQuery}, because rollup requires columns to be split into dimensions and * aggregations. * - The query must not finalize aggregations, because rollup requires inserting the intermediate type of @@ -1832,7 +1946,7 @@ private static Map copyOfStageRuntimesEndingAtCurrentTime( * Method that determines whether an exception was raised due to the task lock for the controller task being * preempted. Uses string comparison, because the relevant Overlord APIs do not have a more reliable way of * discerning the cause of errors. - * + *

* Error strings are taken from {@link org.apache.druid.indexing.common.actions.TaskLocks} * and {@link SegmentAllocateAction}. */ @@ -1922,6 +2036,7 @@ private Pair> run() throws IOExceptio sendPartitionBoundaries(); updateLiveReportMaps(); cleanUpEffectivelyFinishedStages(); + retryFailedTasks(); runKernelCommands(); } @@ -1933,6 +2048,95 @@ private Pair> run() throws IOExceptio return Pair.of(queryKernel, workerTaskLauncherFuture); } + private void retryFailedTasks() throws InterruptedException + { + if (workOrdersToRetry.size() == 0) { + return; + } + Set workersNeedToBeFullyStarted = new HashSet<>(); + + Map> stageWorkerOrders = new HashMap<>(); + + for (Map.Entry> workerStages : workOrdersToRetry.entrySet()) { + workersNeedToBeFullyStarted.add(workerStages.getKey()); + for (WorkOrder workOrder : workerStages.getValue()) { + stageWorkerOrders.compute( + new StageId(queryDef.getQueryId(), workOrder.getStageNumber()), + (stageId, workOrders) -> { + if (workOrders == null) { + workOrders = new HashMap(); + } + workOrders.put(workerStages.getKey(), workOrder); + return workOrders; + } + ); + } + } + + workerTaskLauncher.waitUntilWorkersReady(workersNeedToBeFullyStarted); + + for (Map.Entry> stageWorkOrders : stageWorkerOrders.entrySet()) { + + + Set> workOrdersNotSent = ConcurrentHashMap.newKeySet(); + contactWorkersForStage( + (netClient, taskId, workerNumber) -> { + SettableFuture settableFuture = SettableFuture.create(); + queryKernel.workOrdersSentForWorker(stageWorkOrders.getKey(), workerNumber); + ListenableFuture future = netClient.postWorkOrder( + taskId, + stageWorkOrders.getValue().get(workerNumber) + ); + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Void result) + { + settableFuture.set(true); + } + + @Override + public void onFailure(Throwable t) + { + // durable storage flag will always be set if code reaches here. Skipping the check. + workOrdersNotSent.add(new Pair<>(workerNumber, taskId)); + settableFuture.set(false); + } + }); + + return settableFuture; + }, + new IntArraySet(stageWorkOrders.getValue().keySet()) + ); + + +// for (int worker : workOrdersSent) { +// queryKernel.workOrdersSentForWorker(stageWorkOrders.getKey(), worker); +// } + + + // remove worker orders from retryQueue + for (Integer workerNumber : workersNeedToBeFullyStarted) { + workOrdersToRetry.compute(workerNumber, (task, workOrderSet) -> { + if (workOrderSet == null || workOrderSet.size() == 0 || !workOrderSet.remove(stageWorkOrders.getValue() + .get( + workerNumber))) { + throw new ISE("Worker[%d] orders not found", workerNumber); + } + if (workOrderSet.size() == 0) { + return null; + } + return workOrderSet; + }); + } + + for (Pair workerIdTask : workOrdersNotSent) { + addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); + } + + } + } + /** * Run at least one command from {@link #kernelManipulationQueue}, waiting for it if necessary. */ @@ -2054,31 +2258,37 @@ private void sendPartitionBoundaries() for (final StageId stageId : queryKernel.getActiveStages()) { if (queryKernel.getStageDefinition(stageId).mustGatherResultKeyStatistics() - && queryKernel.doesStageHaveResultPartitions(stageId) - && stageResultPartitionBoundariesSent.add(stageId)) { + && queryKernel.doesStageHaveResultPartitions(stageId)) { + IntSet workersToSendPartitionBoundaries = queryKernel.getWorkersToSendPartitionBoundaries(stageId); + if (workersToSendPartitionBoundaries.isEmpty()) { + return; + } if (log.isDebugEnabled()) { final ClusterByPartitions partitions = queryKernel.getResultPartitionBoundariesForStage(stageId); log.debug( - "Query [%s] sending out partition boundaries for stage %d: %s", + "Query [%s] sending out partition boundaries for stage %d: %s for workers %s", stageId.getQueryId(), stageId.getStageNumber(), IntStream.range(0, partitions.size()) .mapToObj(i -> StringUtils.format("%s:%s", i, partitions.get(i))) - .collect(Collectors.joining(", ")) + .collect(Collectors.joining(", ")), + workersToSendPartitionBoundaries.toString() ); } else { log.info( - "Query [%s] sending out partition boundaries for stage %d.", + "Query [%s] sending out partition boundaries for stage %d for workers %s", stageId.getQueryId(), - stageId.getStageNumber() + stageId.getStageNumber(), + workersToSendPartitionBoundaries.toString() ); } postResultPartitionBoundariesForStage( + queryKernel, queryDef, stageId.getStageNumber(), queryKernel.getResultPartitionBoundariesForStage(stageId), - queryKernel.getWorkerInputsForStage(stageId).workers() + workersToSendPartitionBoundaries ); } } @@ -2164,8 +2374,8 @@ private void throwKernelExceptionIfNotUnknown() /** * Interface used by {@link #contactWorkersForStage}. */ - private interface TaskContactFn + private interface TaskContactFn { - ListenableFuture contactTask(WorkerClient client, String taskId, int workerNumber); + ListenableFuture contactTask(WorkerClient client, String taskId, int workerNumber); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index ac5cf95d1dd4..db6fe8463ead 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -58,4 +58,14 @@ public class Limits * Maximum size of the kernel manipulation queue in {@link org.apache.druid.msq.indexing.MSQControllerTask}. */ public static final int MAX_KERNEL_MANIPULATION_QUEUE_SIZE = 100_000; + + /** + * Maximum retries across all workers. + */ + public static final int TOTAL_RETRY_LIMIT = 30; + + /** + * Maximum worker retries. Initial run is not a retry. The worker will be spawned 1 + retryNumber of times before erroring out; + */ + public static int WORKER_RETRY_LIMIT = 2; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java index fa1371f132ed..bceace299169 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java @@ -31,6 +31,7 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; @@ -65,9 +66,9 @@ public static String controllerTaskId(@Nullable final String queryId) /** * Returns a worker task ID given a SQL query id. */ - public static String workerTaskId(final String controllerTaskId, final int workerNumber) + public static String workerTaskId(final String controllerTaskId, final int workerNumber, final int retryCount) { - return StringUtils.format("%s-worker%d", controllerTaskId, workerNumber); + return StringUtils.format("%s-worker%d_%d", controllerTaskId, workerNumber, retryCount); } /** @@ -194,7 +195,7 @@ static String errorReportToLogMessage(final MSQErrorReport errorReport) logMessage.append("; host ").append(errorReport.getHost()); } - logMessage.append(": ").append(errorReport.getFault().getCodeWithMessage()); + logMessage.append(": ").append(MSQFaultUtils.generateMessageWithErrorCode(errorReport.getFault())); if (errorReport.getExceptionStackTrace() != null) { if (errorReport.getFault() instanceof UnknownFault) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 275965abbe40..8d4cccbdf43a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -69,6 +69,7 @@ import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; import org.apache.druid.msq.indexing.error.MSQWarningReportPublisher; import org.apache.druid.msq.indexing.error.MSQWarningReportSimplePublisher; @@ -218,7 +219,7 @@ public TaskStatus run() throws Exception } }); - return TaskStatus.failure(id(), errorReport.getFault().getCodeWithMessage()); + return TaskStatus.failure(id(), MSQFaultUtils.generateMessageWithErrorCode(errorReport.getFault())); } else { return TaskStatus.success(id()); } @@ -471,11 +472,12 @@ public void postWorkOrder(final WorkOrder workOrder) throw new ISE("Worker number mismatch: expected [%d]", task.getWorkerNumber()); } + // Dont add to queue if workerOrder already present. kernelManipulationQueue.add( kernelHolder -> - kernelHolder.getStageKernelMap().computeIfAbsent( + kernelHolder.getStageKernelMap().putIfAbsent( workOrder.getStageDefinition().getId(), - ignored -> WorkerStageKernel.create(workOrder) + WorkerStageKernel.create(workOrder) ) ); } @@ -493,10 +495,18 @@ public boolean postResultPartitionBoundaries( kernelHolder -> { final WorkerStageKernel stageKernel = kernelHolder.getStageKernelMap().get(stageId); - // Ignore the update if we don't have a kernel for this stage. if (stageKernel != null) { - stageKernel.setResultPartitionBoundaries(stagePartitionBoundaries); + if (!stageKernel.hasResultPartitionBoundaries()) { + stageKernel.setResultPartitionBoundaries(stagePartitionBoundaries); + } else { + // Ignore if partition boundaries are already set. + log.warn( + "Stage[%s] already has result partition boundaries set. Ignoring the latest partition boundaries recieved.", + stageId + ); + } } else { + // Ignore the update if we don't have a kernel for this stage. log.warn("Ignored result partition boundaries call for unknown stage [%s]", stageId); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java index 7d320b111ff8..415c93a85999 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java @@ -32,7 +32,7 @@ */ public interface WorkerManagerClient extends Closeable { - String run(String controllerId, MSQWorkerTask task); + String run(String taskId, MSQWorkerTask task); /** * @param workerId the task ID diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java index ac1e71a55258..707ade86a52b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java @@ -43,10 +43,10 @@ public IndexerWorkerManagerClient(final OverlordClient overlordClient) } @Override - public String run(String controllerId, MSQWorkerTask task) + public String run(String taskId, MSQWorkerTask task) { - FutureUtils.getUnchecked(overlordClient.runTask(controllerId, task), true); - return controllerId; + FutureUtils.getUnchecked(overlordClient.runTask(taskId, task), true); + return taskId; } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java index 5ce5b2290604..0930eb2b45d3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java @@ -36,6 +36,7 @@ import org.apache.druid.msq.exec.WorkerImpl; import java.util.Map; +import java.util.Objects; @JsonTypeName(MSQWorkerTask.TYPE) public class MSQWorkerTask extends AbstractTask @@ -44,8 +45,10 @@ public class MSQWorkerTask extends AbstractTask private final String controllerTaskId; private final int workerNumber; + private final int retryCount; // Using an Injector directly because tasks do not have a way to provide their own Guice modules. + // Not part of equals and hashcode implementation @JacksonInject private Injector injector; @@ -57,11 +60,12 @@ public MSQWorkerTask( @JsonProperty("controllerTaskId") final String controllerTaskId, @JsonProperty("dataSource") final String dataSource, @JsonProperty("workerNumber") final int workerNumber, - @JsonProperty("context") final Map context + @JsonProperty("context") final Map context, + @JsonProperty(value = "retry", defaultValue = "0") final int retryCount ) { super( - MSQTasks.workerTaskId(controllerTaskId, workerNumber), + MSQTasks.workerTaskId(controllerTaskId, workerNumber, retryCount), controllerTaskId, null, dataSource, @@ -70,6 +74,7 @@ public MSQWorkerTask( this.controllerTaskId = controllerTaskId; this.workerNumber = workerNumber; + this.retryCount = retryCount; } @JsonProperty @@ -84,6 +89,17 @@ public int getWorkerNumber() return workerNumber; } + @JsonProperty("retry") + public int getRetryCount() + { + return retryCount; + } + + public MSQWorkerTask getRetryTask() + { + return new MSQWorkerTask(controllerTaskId, getDataSource(), workerNumber, getContext(), retryCount + 1); + } + @Override public String getType() { @@ -112,4 +128,29 @@ public void stopGracefully(TaskConfig taskConfig) worker.stopGracefully(); } } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + MSQWorkerTask that = (MSQWorkerTask) o; + return workerNumber == that.workerNumber + && retryCount == that.retryCount + && Objects.equals(controllerTaskId, that.controllerTaskId) + && Objects.equals(worker, that.worker); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), controllerTaskId, workerNumber, retryCount, worker); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 15c260bdfb22..05b728d95f95 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -35,9 +35,12 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.ControllerContext; import org.apache.druid.msq.exec.ControllerImpl; +import org.apache.druid.msq.exec.Limits; import org.apache.druid.msq.exec.WorkerManagerClient; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; +import org.apache.druid.msq.indexing.error.TooManyWorkerRetriesFault; +import org.apache.druid.msq.indexing.error.TotalRetryLimitExceededFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; import org.apache.druid.msq.util.MultiStageQueryContext; @@ -46,6 +49,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -54,6 +58,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.IntStream; @@ -67,6 +72,7 @@ public class MSQWorkerTaskLauncher private static final long LOW_FREQUENCY_CHECK_MILLIS = 2000; private static final long SWITCH_TO_LOW_FREQUENCY_CHECK_AFTER_MILLIS = 10000; private static final long SHUTDOWN_TIMEOUT_MS = Duration.ofMinutes(1).toMillis(); + private AtomicInteger currentRetryCount = new AtomicInteger(); // States for "state" variable. private enum State @@ -101,15 +107,30 @@ private enum State // Mutable state accessible only to the main loop. LinkedHashMap since order of key set matters. Tasks are added // here once they are submitted for running, but before they are fully started up. + // taskId -> taskTracker private final Map taskTrackers = new LinkedHashMap<>(); // Set of tasks which are issued a cancel request by the controller. private final Set canceledWorkerTasks = ConcurrentHashMap.newKeySet(); + + // tasks to clean up due to retries + private final Set tasksToCleanup = ConcurrentHashMap.newKeySet(); + + // retry worker set + private final Set retryWorkerSet = ConcurrentHashMap.newKeySet(); + + private final Set retryingWorkerSet = ConcurrentHashMap.newKeySet(); + + + private final Map> workerToTaskIds = new ConcurrentHashMap<>(); + private final RetryTask retryTask; + public MSQWorkerTaskLauncher( final String controllerTaskId, final String dataSource, final ControllerContext context, + final RetryTask retryTask, final boolean durableStageStorageEnabled, final long maxTaskStartDelayMillis ) @@ -120,6 +141,8 @@ public MSQWorkerTaskLauncher( this.exec = Execs.singleThreaded( "multi-stage-query-task-launcher[" + StringUtils.encodeForFormat(controllerTaskId) + "]-%s" ); + + this.retryTask = retryTask; this.durableStageStorageEnabled = durableStageStorageEnabled; this.maxTaskStartDelayMillis = maxTaskStartDelayMillis; } @@ -185,7 +208,7 @@ public void stop(final boolean interrupt) /** * Get the list of currently-active tasks. */ - public List getTaskList() + public List getActiveTasks() { synchronized (taskIds) { return ImmutableList.copyOf(taskIds); @@ -214,6 +237,26 @@ public void launchTasksIfNeeded(final int taskCount) throws InterruptedException } } + + public void submitForRelaunch(int workerNumber) + { + retryWorkerSet.add(workerNumber); + } + + public void waitUntilWorkersReady(Set workerSet) throws InterruptedException + { + synchronized (taskIds) { + while (!fullyStartedTasks.containsAll(workerSet)) { + if (stopFuture.isDone() || stopFuture.isCancelled()) { + FutureUtils.getUnchecked(stopFuture, false); + throw new ISE("Stopped"); + } + taskIds.wait(); + } + } + } + + /** * Checks if the controller has canceled the input taskId. This method is used in {@link ControllerImpl} * to figure out if the worker taskId is canceled by the controller. If yes, the errors from that worker taskId @@ -238,6 +281,8 @@ private void mainLoop() runNewTasks(); updateTaskTrackersAndTaskIds(); checkForErroneousTasks(); + relaunchTasks(); + cleanFailedTasks(); } catch (Throwable e) { state.set(State.STOPPED); @@ -321,10 +366,19 @@ private void runNewTasks() controllerTaskId, dataSource, i, - taskContext + taskContext, + 0 ); - taskTrackers.put(task.getId(), new TaskTracker(i)); + taskTrackers.put(task.getId(), new TaskTracker(i, task)); + workerToTaskIds.compute(i, (workerId, taskIds) -> { + if (taskIds == null) { + taskIds = new ArrayList<>(); + } + taskIds.add(task.getId()); + return taskIds; + }); + context.workerManager().run(task.getId(), task); synchronized (taskIds) { @@ -374,33 +428,120 @@ private void updateTaskTrackersAndTaskIds() /** * Used by the main loop to generate exceptions if any tasks have failed, have taken too long to start up, or * have gone inexplicably missing. - * + *

* Throws an exception if some task is erroneous. */ private void checkForErroneousTasks() { final int numTasks = taskTrackers.size(); - for (final Map.Entry taskEntry : taskTrackers.entrySet()) { + Iterator> taskTrackerIterator = taskTrackers.entrySet().iterator(); + + while (taskTrackerIterator.hasNext()) { + final Map.Entry taskEntry = taskTrackerIterator.next(); final String taskId = taskEntry.getKey(); final TaskTracker tracker = taskEntry.getValue(); - - if (tracker.status == null) { - throw new MSQException(UnknownFault.forMessage(StringUtils.format("Task [%s] status missing", taskId))); + if (tracker.toRetry()) { + continue; } - if (tracker.didRunTimeOut(maxTaskStartDelayMillis) && !canceledWorkerTasks.contains(taskId)) { + if (tracker.status == null) { + removeWorkerFromFullyStartedWorkers(tracker); + final String errorMessage = StringUtils.format("Task [%s] status missing", taskId); + log.info(errorMessage + ". Trying to relaunch the worker"); + retryTask.shouldRetry( + tracker.msqWorkerTask, + UnknownFault.forMessage(errorMessage) + ); + tracker.retry(); + } else if (tracker.didRunTimeOut(maxTaskStartDelayMillis) && !canceledWorkerTasks.contains(taskId)) { + removeWorkerFromFullyStartedWorkers(tracker); throw new MSQException(new TaskStartTimeoutFault(numTasks + 1)); + } else if (tracker.didFail() && !canceledWorkerTasks.contains(taskId)) { + removeWorkerFromFullyStartedWorkers(tracker); + log.info("Task[%s] failed because %s", taskId, tracker.status.getErrorMsg()); + retryTask.shouldRetry(tracker.msqWorkerTask, new WorkerFailedFault(taskId, tracker.status.getErrorMsg())); + tracker.retry(); } + } + } - if (tracker.didFail() && !canceledWorkerTasks.contains(taskId)) { - throw new MSQException(new WorkerFailedFault(taskId, tracker.status.getErrorMsg())); - } + private void removeWorkerFromFullyStartedWorkers(TaskTracker tracker) + { + synchronized (taskIds) { + fullyStartedTasks.remove(tracker.msqWorkerTask.getWorkerNumber()); + } + } + + + private void relaunchTasks() + { + Iterator iterator = retryWorkerSet.iterator(); + + while (iterator.hasNext()) { + int worker = iterator.next(); + workerToTaskIds.compute(worker, (workerId, taskHistory) -> { + + if (retryingWorkerSet.contains(worker)) { + return taskHistory; + } + + if (taskHistory == null || taskHistory.isEmpty()) { + throw new ISE("TaskHistory cannot by null for worker %d", workerId); + } + String latestTaskId = taskHistory.get(taskHistory.size() - 1); + + TaskTracker tracker = taskTrackers.get(latestTaskId); + if (tracker == null) { + throw new ISE("Did not find taskTracker for latest taskId[%s]", latestTaskId); + } + if (!tracker.isComplete()) { + // if task is not failed donot retry + return taskHistory; + } + + MSQWorkerTask toRetry = tracker.msqWorkerTask; + + MSQWorkerTask retryTask = toRetry.getRetryTask(); + + if (toRetry.getRetryCount() > Limits.WORKER_RETRY_LIMIT) { + throw new MSQException(new TooManyWorkerRetriesFault( + Limits.WORKER_RETRY_LIMIT, + toRetry.getId(), + toRetry.getWorkerNumber(), + "123" + )); + } + if (currentRetryCount.get() > Limits.TOTAL_RETRY_LIMIT) { + throw new MSQException(new TotalRetryLimitExceededFault( + Limits.TOTAL_RETRY_LIMIT, + currentRetryCount.get(), + toRetry.getId(), + "null" + )); + } + taskTrackers.remove(latestTaskId); + currentRetryCount.addAndGet(1); + taskTrackers.put(retryTask.getId(), new TaskTracker(retryTask.getWorkerNumber(), retryTask)); + context.workerManager().run(retryTask.getId(), retryTask); + taskHistory.add(retryTask.getId()); + synchronized (taskIds) { + // replace taskId with the retry taskID for the same worker number + taskIds.set(toRetry.getWorkerNumber(), retryTask.getId()); + fullyStartedTasks.remove(retryTask.getWorkerNumber()); + taskIds.notifyAll(); + } + return taskHistory; + + }); + iterator.remove(); } } private void shutDownTasks() { + + cleanFailedTasks(); for (final Map.Entry taskEntry : taskTrackers.entrySet()) { final String taskId = taskEntry.getKey(); final TaskTracker tracker = taskEntry.getValue(); @@ -410,6 +551,29 @@ private void shutDownTasks() context.workerManager().cancel(taskId); } } + + } + + private void cleanFailedTasks() + { + Iterator tasksToCancel = tasksToCleanup.iterator(); + while (tasksToCancel.hasNext()) { + String taskId = tasksToCancel.next(); + try { + if (canceledWorkerTasks.add(taskId)) { + try { + context.workerManager().cancel(taskId); + } + catch (Exception ignore) { + //ignoring cancellation exception + } + } + } + finally { + tasksToCancel.remove(); + } + + } } /** @@ -454,12 +618,16 @@ private static class TaskTracker { private final int workerNumber; private final long startTimeMs = System.currentTimeMillis(); + private final MSQWorkerTask msqWorkerTask; private TaskStatus status; private TaskLocation initialLocation; - public TaskTracker(int workerNumber) + private boolean isRetrying = false; + + public TaskTracker(int workerNumber, MSQWorkerTask msqWorkerTask) { this.workerNumber = workerNumber; + this.msqWorkerTask = msqWorkerTask; } public boolean unknownLocation() @@ -483,5 +651,15 @@ public boolean didRunTimeOut(final long maxTaskStartDelayMillis) && unknownLocation() && System.currentTimeMillis() - startTimeMs > maxTaskStartDelayMillis; } + + public void retry() + { + isRetrying = true; + } + + public boolean toRetry() + { + return isRetrying; + } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java new file mode 100644 index 000000000000..fd0602d72c42 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing; + +import org.apache.druid.msq.indexing.error.MSQFault; + +public interface RetryTask +{ + + void shouldRetry(MSQWorkerTask workerTask, MSQFault msqFault); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java index a510852c95ad..18035a2df172 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/BaseMSQFault.java @@ -44,6 +44,12 @@ public abstract class BaseMSQFault implements MSQFault BaseMSQFault(final String errorCode, @Nullable final String errorMessage) { this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode"); + Preconditions.checkArgument( + !errorCode.contains(MSQFaultUtils.ERROR_CODE_DELIMITER), + "Error code[%s] contains restricted characters[%s]", + errorCode, + MSQFaultUtils.ERROR_CODE_DELIMITER + ); this.errorMessage = errorMessage; } @@ -99,7 +105,7 @@ public int hashCode() @Override public String toString() { - return getCodeWithMessage(); + return MSQFaultUtils.generateMessageWithErrorCode(this); } private static String format( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java index 4f3508cbde12..c81572a88165 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/CanceledFault.java @@ -26,7 +26,7 @@ public class CanceledFault extends BaseMSQFault { public static final CanceledFault INSTANCE = new CanceledFault(); - static final String CODE = "Canceled"; + public static final String CODE = "Canceled"; CanceledFault() { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.java index b50e430f18ca..06828610e0b1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQException.java @@ -19,8 +19,6 @@ package org.apache.druid.msq.indexing.error; -import com.google.common.base.Preconditions; - import javax.annotation.Nullable; /** @@ -35,8 +33,8 @@ public MSQException( final MSQFault fault ) { - super(fault.getCodeWithMessage(), cause); - this.fault = Preconditions.checkNotNull(fault, "fault"); + super(MSQFaultUtils.generateMessageWithErrorCode(fault), cause); + this.fault = fault; } public MSQException(final MSQFault fault) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java index 36f5910d3eec..c36157e0ddca 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFault.java @@ -36,14 +36,4 @@ public interface MSQFault @Nullable String getErrorMessage(); - default String getCodeWithMessage() - { - final String message = getErrorMessage(); - - if (message != null && !message.isEmpty()) { - return getErrorCode() + ": " + message; - } else { - return getErrorCode(); - } - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java new file mode 100644 index 000000000000..f6cb04629a08 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing.error; + +public class MSQFaultUtils +{ + + public static final String ERROR_CODE_DELIMITER = ": "; + + public static String generateMessageWithErrorCode(MSQFault msqFault) + { + final String message = msqFault.getErrorMessage(); + + if (message != null && !message.isEmpty()) { + return msqFault.getErrorCode() + ERROR_CODE_DELIMITER + message; + } else { + return msqFault.getErrorCode(); + } + } + + public static String getErrorCodeFromMessage(String message) + { + if (message == null || message.isEmpty()) { + return UnknownFault.CODE; + } + return message.split(ERROR_CODE_DELIMITER, 2)[0]; + } + +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriesFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriesFault.java new file mode 100644 index 000000000000..e1a301ed0efc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriesFault.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(TotalRetryLimitExceededFault.CODE) +public class TooManyWorkerRetriesFault extends BaseMSQFault +{ + static final String CODE = "TooManyWorkerRetries"; + + + private final int maxRetryCount; + + private final String taskId; + + private final int workerNumber; + + + private final String rootErrorMessage; + + @JsonCreator + public TooManyWorkerRetriesFault( + @JsonProperty("maxRetryCount") int maxRetryCount, + @JsonProperty("taskId") String taskId, + @JsonProperty("workerNumber") int workerNumber, + @JsonProperty("rootErrorMessage") String rootErrorMessage + ) + { + super( + CODE, + "Worker[%d] exceeded max retry count of %d for task[%s]. Latest failure reason: %s.", + workerNumber, + maxRetryCount, + taskId, + rootErrorMessage + ); + this.maxRetryCount = maxRetryCount; + this.taskId = taskId; + this.workerNumber = workerNumber; + this.rootErrorMessage = rootErrorMessage; + } + + @JsonProperty + public int getMaxRetryCount() + { + return maxRetryCount; + } + + @JsonProperty + public int getWorkerNumber() + { + return workerNumber; + } + + @JsonProperty + public String getTaskId() + { + return taskId; + } + + @JsonProperty + public String getRootErrorMessage() + { + return rootErrorMessage; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TooManyWorkerRetriesFault that = (TooManyWorkerRetriesFault) o; + return maxRetryCount == that.maxRetryCount && workerNumber == that.workerNumber && Objects.equals( + taskId, + that.taskId + ) && Objects.equals(rootErrorMessage, that.rootErrorMessage); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), maxRetryCount, taskId, workerNumber, rootErrorMessage); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRetryLimitExceededFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRetryLimitExceededFault.java new file mode 100644 index 000000000000..003f0157b1dd --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRetryLimitExceededFault.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName(TotalRetryLimitExceededFault.CODE) +public class TotalRetryLimitExceededFault extends BaseMSQFault +{ + static final String CODE = "TotalRetryLimitExceededFault"; + + + private final int maxRetryCount; + + private final String taskId; + + private final int retryCount; + + + private final String rootErrorMessage; + + @JsonCreator + public TotalRetryLimitExceededFault( + @JsonProperty("maxRetryCount") int maxRetryCount, + @JsonProperty("retryCount") int retryCount, + @JsonProperty("taskId") String taskId, + @JsonProperty("rootErrorMessage") String rootErrorMessage + ) + { + super( + CODE, + "Retry count %d exceeded total retry limit %d .Latest task[%s] failure reason: %s", + retryCount, + maxRetryCount, + taskId, + rootErrorMessage + ); + this.maxRetryCount = maxRetryCount; + this.retryCount = retryCount; + this.taskId = taskId; + + this.rootErrorMessage = rootErrorMessage; + } + + @JsonProperty + public int getMaxRetryCount() + { + return maxRetryCount; + } + + @JsonProperty + public String getTaskId() + { + return taskId; + } + + @JsonProperty + public int getRetryCount() + { + return retryCount; + } + + @JsonProperty + public String getRootErrorMessage() + { + return rootErrorMessage; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TotalRetryLimitExceededFault that = (TotalRetryLimitExceededFault) o; + return maxRetryCount == that.maxRetryCount && retryCount == that.retryCount && Objects.equals( + taskId, + that.taskId + ) && Objects.equals(rootErrorMessage, that.rootErrorMessage); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), maxRetryCount, taskId, retryCount, rootErrorMessage); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java index 0fb888d8fc09..5c50280fac32 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRpcFailedFault.java @@ -28,7 +28,7 @@ @JsonTypeName(WorkerRpcFailedFault.CODE) public class WorkerRpcFailedFault extends BaseMSQFault { - static final String CODE = "WorkerRpcFailed"; + public static final String CODE = "WorkerRpcFailed"; private final String workerTaskId; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 9c02beea88d1..cdda7bc3a072 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -23,15 +23,23 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import it.unimi.dsi.fastutil.ints.Int2IntAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2IntMap; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.exec.QueryValidator; +import org.apache.druid.msq.indexing.error.CanceledFault; +import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; +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.input.InputSpecSlicer; import org.apache.druid.msq.input.InputSpecSlicerFactory; import org.apache.druid.msq.input.stage.ReadablePartitions; @@ -44,6 +52,7 @@ import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -51,11 +60,12 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; /** * Kernel for the controller of a multi-stage query. - * + *

* Instances of this class are state machines for query execution. Kernels do not do any RPC or deal with any data. * This separation of decision-making from the "real world" allows the decision-making to live in one, * easy-to-follow place. @@ -106,6 +116,13 @@ public class ControllerQueryKernel */ private final Set effectivelyFinishedStages = new HashSet<>(); + + private final Map> stageWorkOrders; + + private final Set retriableErrorCodes = ImmutableSet.of(CanceledFault.CODE, UnknownFault.CODE, + WorkerRpcFailedFault.CODE + ); + public ControllerQueryKernel(final QueryDefinition queryDef) { this.queryDef = queryDef; @@ -116,6 +133,8 @@ public ControllerQueryKernel(final QueryDefinition queryDef) this.pendingInflowMap = computeStageInflowMap(queryDef); this.pendingOutflowMap = computeStageOutflowMap(queryDef); + stageWorkOrders = new ConcurrentHashMap<>(); + initializeReadyToRunStages(); } @@ -247,7 +266,7 @@ public Int2ObjectMap createWorkOrders( QueryValidator.validateWorkOrder(workOrder); retVal.put(workerNumber, workOrder); } - + stageWorkOrders.put(new StageId(queryDef.getQueryId(), stageNumber), retVal); return retVal; } @@ -323,6 +342,22 @@ public ReadablePartitions getResultPartitionsForStage(final StageId stageId) return getStageKernelOrThrow(stageId).getResultPartitions(); } + public IntSet getWorkersToSendPartitionBoundaries(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getWorkersForPartitionBoundaries(); + } + + + public void partitionBoundariesSentForWorker(final StageId stageId, int worker) + { + getStageKernelOrThrow(stageId).partitionBoundariesSentForWorker(worker); + } + + public void workOrdersSentForWorker(final StageId stageId, int worker) + { + getStageKernelOrThrow(stageId).workOrderSentForWorker(worker); + } + /** * Delegates call to {@link ControllerStageTracker#getResultPartitionBoundaries()} */ @@ -364,7 +399,7 @@ public void startStage(final StageId stageId) /** * Checks if the stage can be finished, delegates call to {@link ControllerStageTracker#finish()} for internal phase * transition and registers the transition in this query kernel - * + *

* If the method is called with strict = true, we confirm if the stage can be marked as finished or else * throw illegal argument exception */ @@ -376,6 +411,7 @@ public void finishStage(final StageId stageId, final boolean strict) getStageKernelOrThrow(stageId).finish(); effectivelyFinishedStages.remove(stageId); transitionStageKernel(stageId, ControllerStagePhase.FINISHED); + stageWorkOrders.remove(stageId); } /** @@ -403,7 +439,6 @@ public void addResultKeyStatisticsForStageAndWorker( // If the phase is POST_READING or FAILED, that implies the kernel has transitioned. We need to account for that switch (newPhase) { - case POST_READING: case FAILED: transitionStageKernel(stageId, newPhase); break; @@ -454,11 +489,26 @@ private ControllerStageTracker getStageKernelOrThrow(StageId stageId) return stageKernel; } + private WorkOrder getWorkOrder(int workerNumber, StageId stageId) + { + Int2ObjectMap stageWorkOrder = stageWorkOrders.get(stageId); + + if (stageWorkOrder == null) { + throw new ISE("Stage[%d] work orders not found", stageId.getStageNumber()); + } + + WorkOrder workOrder = stageWorkOrder.get(workerNumber); + if (workOrder == null) { + throw new ISE("Work order for worker[%d] not found for stage[%d]", workerNumber, stageId.getStageNumber()); + } + return workOrder; + } + /** - * Whenever a stage kernel changes it phase, the change must be "registered" by calling this method with the stageId + * Whenever a stage kernel changes its phase, the change must be "registered" by calling this method with the stageId * and the new phase */ - public void transitionStageKernel(StageId stageId, ControllerStagePhase newPhase) + private void transitionStageKernel(StageId stageId, ControllerStagePhase newPhase) { Preconditions.checkArgument( stageTracker.containsKey(stageId), @@ -543,4 +593,46 @@ private static Map> computeStageOutflowMap(final QueryDefi return retVal; } + + public List getRetriableWorkOrders(int workerNumber, MSQFault msqFault) + { + + final String errorCode; + if (msqFault instanceof WorkerFailedFault) { + errorCode = MSQFaultUtils.getErrorCodeFromMessage((msqFault.getErrorMessage())); + } else { + errorCode = msqFault.getErrorCode(); + } + + if (retriableErrorCodes.contains(errorCode)) { + return getRetriableWorkOrders(workerNumber); + + } else { + throw new MSQException(msqFault); + } + } + + private List getRetriableWorkOrders(int worker) + { + List trackedSet = new ArrayList<>(getActiveStages()); + // no need to retry effectively finished stages + List getEffictivelyFinishedStages = getEffectivelyFinishedStageIds(); + trackedSet.removeAll(getEffictivelyFinishedStages); + + List workOrders = new ArrayList<>(); + for (StageId stageId : trackedSet) { + + ControllerStageTracker controllerStageTracker = getStageKernelOrThrow(stageId); + if (ControllerStagePhase.RETRYING.canTransitionFrom(controllerStageTracker.getPhase()) + && controllerStageTracker.retryIfNeeded(worker)) { + workOrders.add(getWorkOrder(worker, stageId)); + // should be a no-op. Calling for code patterns. + transitionStageKernel(stageId, ControllerStagePhase.RETRYING); + } + + } + return workOrders; + } + } + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index 6686292c1d85..f5dd20c8df59 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java @@ -25,7 +25,7 @@ /** * Phases that a stage can be in, as far as the controller is concerned. - * + *

* Used by {@link ControllerStageTracker}. */ public enum ControllerStagePhase @@ -35,7 +35,16 @@ public enum ControllerStagePhase @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return false; + return true; + } + }, + + + RETRYING { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == READING_INPUT || priorPhase == POST_READING || priorPhase == RETRYING; } }, @@ -44,7 +53,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return priorPhase == NEW; + return priorPhase == RETRYING || priorPhase == NEW; } }, @@ -55,7 +64,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return priorPhase == READING_INPUT; + return priorPhase == RETRYING || priorPhase == READING_INPUT; } }, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index 1b32deb5341f..e857ffdf6aab 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -22,6 +22,8 @@ import it.unimi.dsi.fastutil.ints.Int2IntAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2IntMap; import it.unimi.dsi.fastutil.ints.Int2IntSortedMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.frame.key.ClusterByPartitions; @@ -38,16 +40,18 @@ import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.kernel.worker.WorkerStagePhase; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import javax.annotation.Nullable; import java.util.List; +import java.util.stream.IntStream; /** * Controller-side state machine for each stage. Used by {@link ControllerQueryKernel} to form the overall state * machine for an entire query. - * + *

* Package-private: stage trackers are an internal implementation detail of {@link ControllerQueryKernel}, not meant * for separate use. */ @@ -57,8 +61,8 @@ class ControllerStageTracker private final int workerCount; private final WorkerInputs workerInputs; - private final IntSet workersWithResultKeyStatistics = new IntAVLTreeSet(); - private final IntSet workersWithResultsComplete = new IntAVLTreeSet(); + private final Int2ObjectMap workerStagePhases = new Int2ObjectOpenHashMap<>(); + private ControllerStagePhase phase = ControllerStagePhase.NEW; @@ -88,6 +92,8 @@ private ControllerStageTracker( this.workerCount = workerInputs.workerCount(); this.workerInputs = workerInputs; + initializeWorkerState(workerCount); + if (stageDef.mustGatherResultKeyStatistics()) { this.resultKeyStatisticsCollector = stageDef.createResultKeyStatisticsCollector(); } else { @@ -96,6 +102,11 @@ private ControllerStageTracker( } } + private void initializeWorkerState(int workerCount) + { + IntStream.range(0, workerCount).forEach(wokerNumber -> workerStagePhases.put(wokerNumber, WorkerStagePhase.NEW)); + } + /** * Given a stage definition and number of workers to available per stage, this method creates a stage tracker. * This method determines the actual number of workers to use (which in turn depends on the input slices and @@ -162,10 +173,73 @@ ClusterByPartitions getResultPartitionBoundaries() } } + + IntSet getWorkersForPartitionBoundaries() + { + if (!getStageDefinition().doesShuffle()) { + throw new ISE("Result partition information is not relevant to this stage because it does not shuffle"); + } + IntAVLTreeSet workers = new IntAVLTreeSet(); + for (Integer worker : workerStagePhases.keySet()) { + if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.equals(workerStagePhases.get(worker))) { + workers.add(worker); + } + } + return workers; + } + + void workOrderSentForWorker(int worker) + { + + workerStagePhases.compute(worker, (wk, state) -> { + if (state == null) { + throw new ISE("Worker[%d] not found for stage[%s]", wk, stageDef.getStageNumber()); + } + if (!WorkerStagePhase.READING_INPUT.canTransitionFrom(state)) { + throw new ISE( + "Worker[%d] cannot transistion from state[%s] to state[%s] while sending work order", + worker, + state, + WorkerStagePhase.READING_INPUT + ); + } + return WorkerStagePhase.READING_INPUT; + }); + if (phase != ControllerStagePhase.READING_INPUT) { + if (allWorkOrderSent()) { + // if all the work orders are sent, change state to reading input from retrying + transitionTo(ControllerStagePhase.READING_INPUT); + } + } + + } + + + void partitionBoundariesSentForWorker(int worker) + { + + workerStagePhases.compute(worker, (wk, state) -> { + if (state == null) { + throw new ISE("Worker[%d] not found for stage[%s]", wk, stageDef.getStageNumber()); + } + if (!WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT.canTransitionFrom(state)) { + throw new ISE( + "Worker[%d] cannot transistion from state[%s] to state[%s] while sending partition boundaries", + worker, + state, + WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT + ); + } + return WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT; + }); + + } + + /** * Whether the result key statistics collector for this stage has encountered any multi-valued input at * any key position. - * + *

* This method exists because {@link org.apache.druid.timeline.partition.DimensionRangeShardSpec} does not * support partitioning on multi-valued strings, so we need to know if any multi-valued strings exist in order * to decide whether we can use this kind of shard spec. @@ -247,22 +321,42 @@ ControllerStagePhase addResultKeyStatisticsForWorker( throw new IAE("Invalid workerNumber [%s]", workerNumber); } - if (phase != ControllerStagePhase.READING_INPUT) { + if (phase != ControllerStagePhase.READING_INPUT && phase != ControllerStagePhase.RETRYING) { throw new ISE("Cannot add result key statistics from stage [%s]", phase); } + WorkerStagePhase currentPhase = workerStagePhases.get(workerNumber); + + if (currentPhase == null) { + throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); + } + try { - if (workersWithResultKeyStatistics.add(workerNumber)) { + if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.canTransitionFrom(currentPhase)) { + workerStagePhases.put(workerNumber, WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); resultKeyStatisticsCollector.addAll(snapshot); - if (workersWithResultKeyStatistics.size() == workerCount) { + if (allPartitionStatisticsPresent()) { generateResultPartitionsAndBoundaries(); +// for (int worker : workerStagePhases.keySet()) { +// workerStagePhases.compute(worker, (wk, state) -> WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT); +// } + // Phase can become FAILED after generateResultPartitionsAndBoundaries, if there were too many partitions. if (phase != ControllerStagePhase.FAILED) { transitionTo(ControllerStagePhase.POST_READING); } } + } else { + throw new ISE( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + workerNumber, + (stageDef.getStageNumber()), + WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES, + currentPhase + + ); } } catch (Exception e) { @@ -289,12 +383,20 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO throw new NullPointerException("resultObject must not be null"); } + WorkerStagePhase currentPhase = workerStagePhases.get(workerNumber); + if (currentPhase == null) { + throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); + } + // This is unidirectional flow of data. While this works in the current state of MSQ where partial fault tolerance // is implemented and a query flows in one direction only, rolling back of workers' state and query kernel's // phase should be allowed to fully support fault tolerance in cases such as: // 1. Rolling back worker's state in case it fails (and then retries) // 2. Rolling back query kernel's phase in case the results are lost (and needs workers to retry the computation) - if (workersWithResultsComplete.add(workerNumber)) { + + + if (WorkerStagePhase.RESULTS_READY.canTransitionFrom(currentPhase)) { + workerStagePhases.put(workerNumber, WorkerStagePhase.RESULTS_READY); if (this.resultObject == null) { this.resultObject = resultObject; } else { @@ -302,15 +404,30 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO this.resultObject = getStageDefinition().getProcessorFactory() .mergeAccumulatedResult(this.resultObject, resultObject); } + } else { + throw new ISE( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + workerNumber, + (stageDef.getStageNumber()), WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT, currentPhase + + ); } - if (workersWithResultsComplete.size() == workerCount) { + if (allResultsPresent()) { transitionTo(ControllerStagePhase.RESULTS_READY); return true; } return false; } + private boolean allResultsPresent() + { + return workerStagePhases.values() + .stream() + .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.RESULTS_READY)) + .count() == workerCount; + } + /** * Reason for failure of this stage. */ @@ -333,7 +450,7 @@ void fail() /** * Sets {@link #resultPartitions} (always) and {@link #resultPartitionBoundaries}. - * + *

* If {@link StageDefinition#mustGatherResultKeyStatistics()} is true, this method cannot be called until after * statistics have been provided to {@link #addResultKeyStatisticsForWorker} for all workers. */ @@ -346,7 +463,7 @@ private void generateResultPartitionsAndBoundaries() final int stageNumber = stageDef.getStageNumber(); if (stageDef.doesShuffle()) { - if (stageDef.mustGatherResultKeyStatistics() && workersWithResultKeyStatistics.size() != workerCount) { + if (stageDef.mustGatherResultKeyStatistics() && !allPartitionStatisticsPresent()) { throw new ISE("Cannot generate result partitions without all worker statistics"); } @@ -385,6 +502,25 @@ private void generateResultPartitionsAndBoundaries() } } + private boolean allPartitionStatisticsPresent() + { + return workerStagePhases.values() + .stream() + .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES)) + .count() + == workerCount; + } + + private boolean allWorkOrderSent() + { + return workerStagePhases.values() + .stream() + .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) + || stagePhase.equals(WorkerStagePhase.READING_INPUT)) + .count() + == workerCount; + } + /** * Marks the stage as failed and sets the reason for the same. * @@ -401,7 +537,7 @@ private void failForReason(final MSQFault fault) } } - void transitionTo(final ControllerStagePhase newPhase) + private void transitionTo(final ControllerStagePhase newPhase) { if (newPhase.canTransitionFrom(phase)) { phase = newPhase; @@ -409,4 +545,32 @@ void transitionTo(final ControllerStagePhase newPhase) throw new IAE("Cannot transition from [%s] to [%s]", phase, newPhase); } } + + public boolean retryIfNeeded(int workerNumber) + { + if (phase.equals(ControllerStagePhase.FINISHED) || phase.equals(ControllerStagePhase.RESULTS_READY)) { + // do nothing + return false; + } + if (!isTrackingWorker(workerNumber)) { + // not tracking this worker + return false; + } + + if (workerStagePhases.get(workerNumber).equals(WorkerStagePhase.RESULTS_READY) + || workerStagePhases.get(workerNumber).equals(WorkerStagePhase.FINISHED)) { + // do nothing + return false; + } + workerStagePhases.put(workerNumber, WorkerStagePhase.NEW); + transitionTo(ControllerStagePhase.RETRYING); + return true; + } + + + public boolean isTrackingWorker(int workerNumber) + { + return workerStagePhases.get(workerNumber) != null; + } + } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java index 1321d1362f73..b8a92c308985 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java @@ -27,6 +27,7 @@ import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; import org.apache.druid.msq.indexing.error.TooManyColumnsFault; import org.apache.druid.msq.indexing.error.TooManyWorkersFault; @@ -150,6 +151,7 @@ public void test_queryWithoutEnoughSlots_shouldThrowException() CONTROLLER_ID, "foo", controllerContext, + (task, fault) -> {}, false, TimeUnit.SECONDS.toMillis(5) ); @@ -161,8 +163,8 @@ public void test_queryWithoutEnoughSlots_shouldThrowException() } catch (Exception e) { Assert.assertEquals( - new TaskStartTimeoutFault(numTasks + 1).getCodeWithMessage(), - ((MSQException) e.getCause()).getFault().getCodeWithMessage() + MSQFaultUtils.generateMessageWithErrorCode(new TaskStartTimeoutFault(numTasks + 1)), + MSQFaultUtils.generateMessageWithErrorCode(((MSQException) e.getCause()).getFault()) ); } } @@ -220,7 +222,7 @@ public synchronized TaskLocation location(String workerId) } @Override - public synchronized String run(String controllerId, MSQWorkerTask task) + public synchronized String run(String taskId, MSQWorkerTask task) { allTasks.add(task.getId()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index 77abdfc8356c..2b400f2379a3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -74,6 +74,8 @@ public void testFaultSerde() throws IOException assertFaultSerde(new TooManyPartitionsFault(10)); assertFaultSerde(new TooManyWarningsFault(10, "the error")); assertFaultSerde(new TooManyWorkersFault(10, 5)); + assertFaultSerde(new TooManyWorkerRetriesFault(2, "taskId", 1, "rootError")); + assertFaultSerde(new TotalRetryLimitExceededFault(2, 2, "taskId", "rootError")); assertFaultSerde(UnknownFault.forMessage(null)); assertFaultSerde(UnknownFault.forMessage("the message")); assertFaultSerde(new WorkerFailedFault("the worker task", "the error msg")); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java index 55fd6b75d096..1ee240d68300 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.IntStream; public class BaseControllerQueryKernelTest extends InitializedNullHandlingTest { @@ -108,21 +109,25 @@ public ControllerQueryKernelTester setupStage( createAndGetNewStageNumbers(false); // Initial phase would always be new as we can call this method only once for each + StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); switch (controllerStagePhase) { case NEW: break; case READING_INPUT: - controllerQueryKernel.startStage(new StageId(queryDefinition.getQueryId(), stageNumber)); + controllerQueryKernel.startStage(stageId); + for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { + controllerQueryKernel.workOrdersSentForWorker(stageId, i); + } break; case POST_READING: setupStage(stageNumber, ControllerStagePhase.READING_INPUT, true); if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { - for (int i = 0; i < numWorkers; ++i) { + for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( - new StageId(queryDefinition.getQueryId(), stageNumber), + stageId, i, ClusterByStatisticsSnapshot.empty() ); @@ -139,9 +144,9 @@ public ControllerQueryKernelTester setupStage( } else { setupStage(stageNumber, ControllerStagePhase.READING_INPUT, true); } - for (int i = 0; i < numWorkers; ++i) { + for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { controllerQueryKernel.setResultsCompleteForStageAndWorker( - new StageId(queryDefinition.getQueryId(), stageNumber), + stageId, i, new Object() ); @@ -150,11 +155,11 @@ public ControllerQueryKernelTester setupStage( case FINISHED: setupStage(stageNumber, ControllerStagePhase.RESULTS_READY, true); - controllerQueryKernel.finishStage(new StageId(queryDefinition.getQueryId(), stageNumber), false); + controllerQueryKernel.finishStage(stageId, false); break; case FAILED: - controllerQueryKernel.failStage(new StageId(queryDefinition.getQueryId(), stageNumber)); + controllerQueryKernel.failStage(stageId); break; } if (!recursiveCall) { @@ -232,6 +237,15 @@ public void startStage(int stageNumber) controllerQueryKernel.startStage(new StageId(queryDefinition.getQueryId(), stageNumber)); } + public void startWorkOrder(int stageNumber) + { + StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + Preconditions.checkArgument(initialized); + IntStream.range(0, queryDefinition.getStageDefinition(stageId).getMaxWorkerCount()) + .forEach(n -> controllerQueryKernel.workOrdersSentForWorker(stageId, n)); + + } + public void finishStage(int stageNumber) { @@ -270,8 +284,9 @@ public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerN public void setResultsCompleteForStageAndWorker(int stageNumber, int workerNumber) { Preconditions.checkArgument(initialized); + final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); controllerQueryKernel.setResultsCompleteForStageAndWorker( - new StageId(queryDefinition.getQueryId(), stageNumber), + stageId, workerNumber, new Object() ); @@ -316,5 +331,19 @@ private Set mapStageIdsToStageNumbers(List stageIds) .map(StageId::getStageNumber) .collect(Collectors.toSet()); } + + public void sendWorkOrdersForWorker(int stageNumber, int worker) + { + Preconditions.checkArgument(initialized); + final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + controllerQueryKernel.workOrdersSentForWorker(stageId, worker); + } + + public void sendPartitionBoundariesForStageAndWorker(int stageNumber, int worker) + { + Preconditions.checkArgument(initialized); + final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + controllerQueryKernel.partitionBoundariesSentForWorker(stageId, worker); + } } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java index 3094bf344d2f..b1330e16aee7 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java @@ -20,6 +20,9 @@ package org.apache.druid.msq.kernel.controller; import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.kernel.worker.WorkerStagePhase; import org.junit.Assert; import org.junit.Test; @@ -146,8 +149,10 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(0), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(0); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); @@ -156,12 +161,16 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(1), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(1); + controllerQueryKernelTester.sendWorkOrdersForWorker(1, 0); controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.sendWorkOrdersForWorker(1, 1); controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 1); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(1, 0); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(1, 1); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 1); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.RESULTS_READY); @@ -171,6 +180,7 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(0), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(2); controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.sendWorkOrdersForWorker(2, 0); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(2, 0); controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(0); @@ -182,13 +192,18 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(1), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(3); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 0); - controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.startWorkOrder(3); controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 1); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 0); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(3, 0); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 0); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(3, 1); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 1); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(1); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.FINISHED); @@ -216,16 +231,19 @@ public void testTransitionsInShufflingStagesAndMultipleWorkers() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); @@ -255,8 +273,15 @@ public void testPrematureResultsComplete() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + Assert.assertThrows(StringUtils.format( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + 1, + 0, + WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES, + WorkerStagePhase.NEW + ) + , ISE.class, () -> controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0)); + } @Test @@ -377,6 +402,7 @@ public void testMarkSuccessfulTerminalStagesAsFinished() private static void transitionNewToResultsComplete(ControllerQueryKernelTester queryKernelTester, int stageNumber) { queryKernelTester.startStage(stageNumber); + queryKernelTester.startWorkOrder(stageNumber); queryKernelTester.setResultsCompleteForStageAndWorker(stageNumber, 0); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 880c6f4adead..6a01f529b5f4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -79,6 +79,7 @@ import org.apache.druid.msq.indexing.error.InsertLockPreemptedFaultTest; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; @@ -861,8 +862,8 @@ public void verifyResults() if (expectedMSQFault != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); Assert.assertEquals( - expectedMSQFault.getCodeWithMessage(), - msqErrorReport.getFault().getCodeWithMessage() + MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), + MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()) ); return; } @@ -1020,8 +1021,8 @@ public Pair>> runQueryWithResult() if (expectedMSQFault != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); Assert.assertEquals( - expectedMSQFault.getCodeWithMessage(), - msqErrorReport.getFault().getCodeWithMessage() + MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), + MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()) ); return null; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index ef17a8d32371..efd3575c6178 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -111,7 +111,7 @@ public MSQTestControllerContext( WorkerManagerClient workerManagerClient = new WorkerManagerClient() { @Override - public String run(String controllerId, MSQWorkerTask task) + public String run(String taskId, MSQWorkerTask task) { if (controller == null) { throw new ISE("Controller needs to be set using the register method"); @@ -161,8 +161,8 @@ public Map statuses(Set taskIds) taskStatus.getId(), taskStatus.getStatusCode(), taskStatus.getDuration(), - null, - null + taskStatus.getErrorMsg(), + taskStatus.getLocation() ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 65f75ee513b3..63a6f781b2b4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -178,7 +178,7 @@ default int getPriority() boolean isReady(TaskActionClient taskActionClient) throws Exception; /** - * Returns whether or not this task can restore its progress from its on-disk working directory. Restorable tasks + * Returns wheather or not this task can restore its progress from its on-disk working directory. Restorable tasks * may be started with a non-empty working directory. Tasks that exit uncleanly may still have a chance to attempt * restores, meaning that restorable tasks should be able to deal with potentially partially written on-disk state. */ From 1930b3bffedd5af81dd14856d18348375bc28296 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 7 Nov 2022 20:58:50 +0530 Subject: [PATCH 02/27] Fixing error message in retry exceeded exception --- .../java/org/apache/druid/msq/exec/WorkerImpl.java | 8 ++++---- .../druid/msq/indexing/MSQWorkerTaskLauncher.java | 4 ++-- .../java/org/apache/druid/msq/test/MSQTestBase.java | 11 +++++++++-- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 7f7441dd2821..023c7f057dbc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -260,7 +260,7 @@ public Optional runTask(final Closer closer) throws Exception // Delete all the stage outputs closer.register(() -> { for (final StageId stageId : stageOutputs.keySet()) { - cleanStageOutput(stageId); + cleanStageOutput(stageId, false); } }); @@ -559,7 +559,7 @@ public void postCleanupStage(final StageId stageId) log.info("Cleanup order for stage: [%s] received", stageId); kernelManipulationQueue.add( holder -> { - cleanStageOutput(stageId); + cleanStageOutput(stageId, true); // Mark the stage as FINISHED holder.getStageKernelMap().get(stageId).setStageFinished(); } @@ -705,7 +705,7 @@ private void postCountersToController() throws IOException * the readable channels corresponding to all the partitions for that stage, and removes it from the {@code stageOutputs} * map */ - private void cleanStageOutput(final StageId stageId) + private void cleanStageOutput(final StageId stageId, boolean removeDurableStorageFiles) { // This code is thread-safe because remove() on ConcurrentHashMap will remove and return the removed channel only for // one thread. For the other threads it will return null, therefore we will call doneReading for a channel only once @@ -725,7 +725,7 @@ private void cleanStageOutput(final StageId stageId) // temp directories where intermediate results were stored, it won't be the case for the external storage. // Therefore, the logic for cleaning the stage output in case of a worker/machine crash has to be external. // We currently take care of this in the controller. - if (durableStageStorageEnabled) { + if (durableStageStorageEnabled && removeDurableStorageFiles) { final String folderName = DurableStorageUtils.getTaskIdOutputsFolderName( task.getControllerTaskId(), stageId.getStageNumber(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 7e00d849ab6e..b89a4a081962 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -533,7 +533,7 @@ private void relaunchTasks() Limits.WORKER_RETRY_LIMIT, toRetry.getId(), toRetry.getWorkerNumber(), - "123" + tracker.status.getErrorMsg() )); } if (currentRetryCount.get() > Limits.TOTAL_RETRY_LIMIT) { @@ -541,7 +541,7 @@ private void relaunchTasks() Limits.TOTAL_RETRY_LIMIT, currentRetryCount.get(), toRetry.getId(), - "null" + tracker.status.getErrorMsg() )); } taskTrackers.remove(latestTaskId); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index bbe2d1168957..bcbf3889556d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -78,6 +78,7 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.MSQFaultUtils; +import org.apache.druid.msq.indexing.error.TooManyWorkerRetriesFault; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; @@ -893,9 +894,12 @@ public void verifyResults() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { + String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriesFault + ? ((TooManyWorkerRetriesFault) msqErrorReport.getFault()).getRootErrorMessage() + : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), - MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()) + errorMessage ); } if (expectedMSQFaultClass != null) { @@ -1061,9 +1065,12 @@ public Pair>> runQueryWithResult() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { + String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriesFault + ? ((TooManyWorkerRetriesFault) msqErrorReport.getFault()).getRootErrorMessage() + : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), - MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()) + errorMessage ); } if (expectedMSQFaultClass != null) { From 1ed9346a6d8d30e4e25864a451cf85d75b86ce8c Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 9 Nov 2022 19:31:32 +0530 Subject: [PATCH 03/27] Cleaning up some code --- .../apache/druid/msq/exec/ControllerImpl.java | 227 ++++++++---------- .../controller/ControllerQueryKernel.java | 2 +- .../apache/druid/msq/exec/MSQInsertTest.java | 2 +- .../ControllerQueryKernelTests.java | 2 +- .../apache/druid/msq/test/MSQTestBase.java | 8 + 5 files changed, 113 insertions(+), 128 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 05f151f7ede9..f16be072402a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1024,10 +1024,25 @@ private Int2ObjectMap> makeSegmentGeneratorWorkerFa return retVal; } - private void contactWorkersForStage(final TaskContactFn contactFn, final IntSet workers) + /** + * A blocking function used to contact multiple workers. Checks if all the workers are running before contacting them. + * + * @param queryKernel + * @param contactFn + * @param workers set of workers to contact + * @param succescCallBack on successfull api call, custom callback + * @param retryOnFailure if set to true, adds this worker to retry queue. If false, cancel all the futures and propergate the exception to the caller. + */ + private void contactWorkersForStage( + final ControllerQueryKernel queryKernel, + final TaskContactFn contactFn, + final IntSet workers, + final OnSuccess succescCallBack, + final boolean retryOnFailure + ) { final List taskIds = getTaskIds(); - final List> taskFutures = new ArrayList<>(workers.size()); + final List> taskFutures = new ArrayList<>(workers.size()); try { workerTaskLauncher.waitUntilWorkersReady(workers); @@ -1036,12 +1051,49 @@ private void contactWorkersForStage(final TaskContactFn contactFn, final IntSet throw new RuntimeException(e); } + Set> failedCalls = ConcurrentHashMap.newKeySet(); + for (int workerNumber : workers) { final String taskId = taskIds.get(workerNumber); - taskFutures.add(contactFn.contactTask(netClient, taskId, workerNumber)); + SettableFuture settableFuture = SettableFuture.create(); + ListenableFuture apiFuture = contactFn.contactTask(netClient, taskId, workerNumber); + Futures.addCallback(apiFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Void result) + { + succescCallBack.onSuccess(taskId, workerNumber); + settableFuture.set(true); + } + + @Override + public void onFailure(Throwable t) + { + if (retryOnFailure) { + log.info( + t, + "Detected failure while contacting task[%s]. Iniitiating relaunch of worker[%d] if applicable", + taskId, + workerNumber + ); + failedCalls.add(new Pair<>(workerNumber, taskId)); + settableFuture.set(false); + } else { + settableFuture.setException(t); + } + } + }); + + taskFutures.add(settableFuture); } FutureUtils.getUnchecked(MSQFutureUtils.allAsList(taskFutures, true), true); + + if (retryOnFailure) { + for (Pair workerIdTask : failedCalls) { + addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); + } + } } @@ -1062,42 +1114,15 @@ private void startWorkForStage( final Int2ObjectMap workOrders = queryKernel.createWorkOrders(stageNumber, extraInfos); final StageId stageId = new StageId(queryDef.getQueryId(), stageNumber); - Set> workOrdersNotSent = ConcurrentHashMap.newKeySet(); - + queryKernel.startStage(stageId); contactWorkersForStage( - (netClient, taskId, workerNumber) -> { - queryKernel.workOrdersSentForWorker(stageId, workerNumber); - SettableFuture settableFuture = SettableFuture.create(); - ListenableFuture future = netClient.postWorkOrder(taskId, workOrders.get(workerNumber)); - Futures.addCallback(future, new FutureCallback() - { - @Override - public void onSuccess(@Nullable Void result) - { - settableFuture.set(true); - } - - @Override - public void onFailure(Throwable t) - { - if (isDurableStorageEnabled) { - settableFuture.setException(new MSQException(t, new WorkerRpcFailedFault(taskId))); - } else { - workOrdersNotSent.add(new Pair<>(workerNumber, taskId)); - settableFuture.set(false); - } - } - }); - - return settableFuture; - }, - workOrders.keySet() + queryKernel, + (netClient, taskId, workerNumber) -> ( + netClient.postWorkOrder(taskId, workOrders.get(workerNumber))), workOrders.keySet(), + (taskId, workerNumber) -> queryKernel.workOrdersSentForWorker(stageId, workerNumber), + isDurableStorageEnabled ); - - for (Pair workerIdTask : workOrdersNotSent) { - addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); - } } private void postResultPartitionBoundariesForStage( @@ -1110,44 +1135,18 @@ private void postResultPartitionBoundariesForStage( { final StageId stageId = new StageId(queryDef.getQueryId(), stageNumber); - Set> partitionBoundariesNotSent = ConcurrentHashMap.newKeySet(); contactWorkersForStage( - (netClient, taskId, workerNumber) -> { - queryKernel.partitionBoundariesSentForWorker(stageId, workerNumber); - SettableFuture settableFuture = SettableFuture.create(); - ListenableFuture future = netClient.postResultPartitionBoundaries( - taskId, - new StageId(queryDef.getQueryId(), stageNumber), - resultPartitionBoundaries - ); - Futures.addCallback(future, new FutureCallback() - { - @Override - public void onSuccess(@Nullable Void result) - { - settableFuture.set(true); - } - - @Override - public void onFailure(Throwable t) - { - if (isDurableStorageEnabled) { - settableFuture.setException(new MSQException(t, new WorkerRpcFailedFault(taskId))); - } else { - partitionBoundariesNotSent.add(new Pair<>(workerNumber, taskId)); - settableFuture.set(false); - } - - } - }); - return settableFuture; - }, - workers + queryKernel, + (netClient, taskId, workerNumber) -> netClient.postResultPartitionBoundaries( + taskId, + stageId, + resultPartitionBoundaries + ), + workers, + (taskId, workerNumber) -> queryKernel.partitionBoundariesSentForWorker(stageId, workerNumber), + isDurableStorageEnabled ); - for (Pair workerIdTask : partitionBoundariesNotSent) { - addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); - } } /** @@ -2114,63 +2113,30 @@ private void retryFailedTasks() throws InterruptedException for (Map.Entry> stageWorkOrders : stageWorkerOrders.entrySet()) { - - Set> workOrdersNotSent = ConcurrentHashMap.newKeySet(); contactWorkersForStage( - (netClient, taskId, workerNumber) -> { - SettableFuture settableFuture = SettableFuture.create(); + queryKernel, + (netClient, taskId, workerNumber) -> netClient.postWorkOrder( + taskId, + stageWorkOrders.getValue().get(workerNumber) + ), + new IntArraySet(stageWorkOrders.getValue().keySet()), + (taskId, workerNumber) -> { queryKernel.workOrdersSentForWorker(stageWorkOrders.getKey(), workerNumber); - ListenableFuture future = netClient.postWorkOrder( - taskId, - stageWorkOrders.getValue().get(workerNumber) - ); - Futures.addCallback(future, new FutureCallback() - { - @Override - public void onSuccess(@Nullable Void result) - { - settableFuture.set(true); + workOrdersToRetry.compute(workerNumber, (task, workOrderSet) -> { + if (workOrderSet == null || workOrderSet.size() == 0 || !workOrderSet.remove(stageWorkOrders.getValue() + .get( + workerNumber))) { + throw new ISE("Worker[%d] orders not found", workerNumber); } - - @Override - public void onFailure(Throwable t) - { - // durable storage flag will always be set if code reaches here. Skipping the check. - workOrdersNotSent.add(new Pair<>(workerNumber, taskId)); - settableFuture.set(false); + if (workOrderSet.size() == 0) { + return null; } + return workOrderSet; }); - - return settableFuture; }, - new IntArraySet(stageWorkOrders.getValue().keySet()) - ); - - -// for (int worker : workOrdersSent) { -// queryKernel.workOrdersSentForWorker(stageWorkOrders.getKey(), worker); -// } - - - // remove worker orders from retryQueue - for (Integer workerNumber : workersNeedToBeFullyStarted) { - workOrdersToRetry.compute(workerNumber, (task, workOrderSet) -> { - if (workOrderSet == null || workOrderSet.size() == 0 || !workOrderSet.remove(stageWorkOrders.getValue() - .get( - workerNumber))) { - throw new ISE("Worker[%d] orders not found", workerNumber); - } - if (workOrderSet.size() == 0) { - return null; - } - return workOrderSet; - }); - } - - for (Pair workerIdTask : workOrdersNotSent) { - addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); - } + isDurableStorageEnabled + ); } } @@ -2226,7 +2192,6 @@ private void startStages() throws IOException, InterruptedException ); for (final StageId stageId : newStageIds) { - queryKernel.startStage(stageId); // Allocate segments, if this is the final stage of an ingestion. if (MSQControllerTask.isIngestion(task.getQuerySpec()) @@ -2381,8 +2346,11 @@ private void cleanUpEffectivelyFinishedStages() for (final StageId stageId : queryKernel.getEffectivelyFinishedStageIds()) { log.info("Query [%s] issuing cleanup order for stage %d.", queryDef.getQueryId(), stageId.getStageNumber()); contactWorkersForStage( + queryKernel, (netClient, taskId, workerNumber) -> netClient.postCleanupStage(taskId, stageId), - queryKernel.getWorkerInputsForStage(stageId).workers() + queryKernel.getWorkerInputsForStage(stageId).workers(), + (ignore1, ignore2) -> {}, + false ); queryKernel.finishStage(stageId, true); } @@ -2411,8 +2379,17 @@ private void throwKernelExceptionIfNotUnknown() /** * Interface used by {@link #contactWorkersForStage}. */ - private interface TaskContactFn + private interface TaskContactFn { - ListenableFuture contactTask(WorkerClient client, String taskId, int workerNumber); + ListenableFuture contactTask(WorkerClient client, String taskId, int workerNumber); } + + + private interface OnSuccess + { + void onSuccess(String taskId, int workerNumber); + + } + + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 502c90f640f9..50b0feeacb9e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -602,7 +602,7 @@ public List getRetriableWorkOrders(int workerNumber, MSQFault msqFaul final String errorCode; if (msqFault instanceof WorkerFailedFault) { - errorCode = MSQFaultUtils.getErrorCodeFromMessage((msqFault.getErrorMessage())); + errorCode = MSQFaultUtils.getErrorCodeFromMessage((((WorkerFailedFault) msqFault).getErrorMsg())); } else { errorCode = msqFault.getErrorCode(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 037ddf6dd945..0a9f3ea24348 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -195,7 +195,7 @@ public void testInsertOnFoo1WithMultiValueToArrayGroupBy() public void testInsertOnFoo1WithMultiValueDimGroupByWithoutGroupByEnable() { Map context = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE) .put("groupByEnableMultiValueUnnesting", false) .build(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java index b1330e16aee7..48cc7af692bc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java @@ -268,7 +268,7 @@ public void testPrematureResultsComplete() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index bcbf3889556d..2613a45a8961 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -219,6 +219,14 @@ public class MSQTestBase extends BaseCalciteQueryTest .put(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false) .build(); + public static final Map MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE = + ImmutableMap.builder() + .put(QueryContexts.CTX_SQL_QUERY_ID, "test-query") + .put(QueryContexts.FINALIZE_KEY, true) + .put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, false) + .build(); + + public final boolean useDefault = NullHandling.replaceWithDefault(); protected File localFileStorageDir; From 14d8c60d6e77e29fb6975fb26a8d1ac2a4e940cd Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 10 Nov 2022 21:24:25 +0530 Subject: [PATCH 04/27] Adding some test cases. --- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../msq/indexing/MSQWorkerTaskLauncher.java | 4 +- .../apache/druid/msq/indexing/RetryTask.java | 9 +- .../controller/ControllerQueryKernel.java | 19 +- .../controller/ControllerStagePhase.java | 20 +- .../controller/ControllerStageTracker.java | 34 +- .../BaseControllerQueryKernelTest.java | 59 +- .../MockQueryDefinitionBuilder.java | 4 +- ...nShufflingWorkersWithRetryKernelTests.java | 242 ++++++++ .../ShufflingWorkersWithRetryKernelTests.java | 568 ++++++++++++++++++ 10 files changed, 912 insertions(+), 49 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index f16be072402a..6963daac1777 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -584,7 +584,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault fault) { - List retriableWorkOrders = kernel.getRetriableWorkOrders(worker, fault); + List retriableWorkOrders = kernel.getRetriableWorkOrdersAndChangeState(worker, fault); if (retriableWorkOrders.size() != 0) { log.info("Submitting worker[%s] for relaunch because of fault[%s]", worker, fault); workerTaskLauncher.submitForRelaunch(worker); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index b89a4a081962..c9077a2341de 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -473,7 +473,7 @@ private void checkForErroneousTasks() removeWorkerFromFullyStartedWorkers(tracker); final String errorMessage = StringUtils.format("Task [%s] status missing", taskId); log.info(errorMessage + ". Trying to relaunch the worker"); - retryTask.shouldRetry( + retryTask.retry( tracker.msqWorkerTask, UnknownFault.forMessage(errorMessage) ); @@ -484,7 +484,7 @@ private void checkForErroneousTasks() } else if (tracker.didFail() && !canceledWorkerTasks.contains(taskId)) { removeWorkerFromFullyStartedWorkers(tracker); log.info("Task[%s] failed because %s", taskId, tracker.status.getErrorMsg()); - retryTask.shouldRetry(tracker.msqWorkerTask, new WorkerFailedFault(taskId, tracker.status.getErrorMsg())); + retryTask.retry(tracker.msqWorkerTask, new WorkerFailedFault(taskId, tracker.status.getErrorMsg())); tracker.retry(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java index fd0602d72c42..39fb1e688ecf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/RetryTask.java @@ -23,6 +23,11 @@ public interface RetryTask { - - void shouldRetry(MSQWorkerTask workerTask, MSQFault msqFault); + /** + * Retry task when {@link MSQFault} is encountered. + * + * @param workerTask + * @param msqFault + */ + void retry(MSQWorkerTask workerTask, MSQFault msqFault); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 50b0feeacb9e..92a2df339831 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -246,7 +246,7 @@ public Int2ObjectMap createWorkOrders( @Nullable final Int2ObjectMap extraInfos ) { - final Int2ObjectMap retVal = new Int2ObjectAVLTreeMap<>(); + final Int2ObjectMap workerToWorkOrder = new Int2ObjectAVLTreeMap<>(); final ControllerStageTracker stageKernel = getStageKernelOrThrow(getStageId(stageNumber)); final WorkerInputs workerInputs = stageKernel.getWorkerInputs(); @@ -266,10 +266,10 @@ public Int2ObjectMap createWorkOrders( ); QueryValidator.validateWorkOrder(workOrder); - retVal.put(workerNumber, workOrder); + workerToWorkOrder.put(workerNumber, workOrder); } - stageWorkOrders.put(new StageId(queryDef.getQueryId(), stageNumber), retVal); - return retVal; + stageWorkOrders.put(new StageId(queryDef.getQueryId(), stageNumber), workerToWorkOrder); + return workerToWorkOrder; } private void createNewKernels( @@ -387,7 +387,7 @@ public Object getResultObjectForStage(final StageId stageId) /** * Checks if the stage can be started, delegates call to {@link ControllerStageTracker#start()} for internal phase - * transition and registers the transition in this queryKernel + * transition and registers the transition in this queryKernel. Work orders need to created via {@link ControllerQueryKernel#createWorkOrders(int, Int2ObjectMap)} before calling this method. */ public void startStage(final StageId stageId) { @@ -395,6 +395,9 @@ public void startStage(final StageId stageId) if (stageKernel.getPhase() != ControllerStagePhase.NEW) { throw new ISE("Cannot start the stage: [%s]", stageId); } + if (stageWorkOrders.get(stageId) == null) { + throw new ISE("Work orders not present for stage %s", stageId); + } stageKernel.start(); transitionStageKernel(stageId, ControllerStagePhase.READING_INPUT); } @@ -597,7 +600,7 @@ private static Map> computeStageOutflowMap(final QueryDefi return retVal; } - public List getRetriableWorkOrders(int workerNumber, MSQFault msqFault) + public List getRetriableWorkOrdersAndChangeState(int workerNumber, MSQFault msqFault) { final String errorCode; @@ -608,14 +611,14 @@ public List getRetriableWorkOrders(int workerNumber, MSQFault msqFaul } if (retriableErrorCodes.contains(errorCode)) { - return getRetriableWorkOrders(workerNumber); + return getRetriableWorkOrdersAndChangeState(workerNumber); } else { throw new MSQException(msqFault); } } - private List getRetriableWorkOrders(int worker) + private List getRetriableWorkOrdersAndChangeState(int worker) { List trackedSet = new ArrayList<>(getActiveStages()); // no need to retry effectively finished stages diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index f5dd20c8df59..e517fa50f071 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java @@ -39,15 +39,6 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) } }, - - RETRYING { - @Override - public boolean canTransitionFrom(final ControllerStagePhase priorPhase) - { - return priorPhase == READING_INPUT || priorPhase == POST_READING || priorPhase == RETRYING; - } - }, - // Reading and mapping inputs (using "stateless" operators like filters, transforms which operate on individual records). READING_INPUT { @Override @@ -64,7 +55,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return priorPhase == RETRYING || priorPhase == READING_INPUT; + return priorPhase == READING_INPUT; } }, @@ -94,6 +85,15 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { return true; } + }, + + // Stage currently under retry. Prior phases stages did not publish its final results yet. + RETRYING { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == READING_INPUT || priorPhase == POST_READING || priorPhase == RETRYING; + } }; public abstract boolean canTransitionFrom(ControllerStagePhase priorPhase); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index c592906a2773..7c7feca530fa 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; import org.apache.druid.msq.indexing.error.UnknownFault; @@ -57,12 +58,16 @@ */ class ControllerStageTracker { + private static final Logger log = new Logger(ControllerStageTracker.class); private final StageDefinition stageDef; + private final int workerCount; private final WorkerInputs workerInputs; private final Int2ObjectMap workerStagePhases = new Int2ObjectOpenHashMap<>(); + private final IntSet resultstatisticsRecieved = new IntAVLTreeSet(); + private ControllerStagePhase phase = ControllerStagePhase.NEW; @@ -341,15 +346,14 @@ ControllerStagePhase addResultKeyStatisticsForWorker( try { if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.canTransitionFrom(currentPhase)) { workerStagePhases.put(workerNumber, WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); - resultKeyStatisticsCollector.addAll(snapshot); + + // if stats already recieved for worker, donot update the sketch. + if (resultstatisticsRecieved.add(workerNumber)) { + resultKeyStatisticsCollector.addAll(snapshot); + } if (allPartitionStatisticsPresent()) { generateResultPartitionsAndBoundaries(); - -// for (int worker : workerStagePhases.keySet()) { -// workerStagePhases.compute(worker, (wk, state) -> WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT); -// } - // Phase can become FAILED after generateResultPartitionsAndBoundaries, if there were too many partitions. if (phase != ControllerStagePhase.FAILED) { transitionTo(ControllerStagePhase.POST_READING); @@ -415,7 +419,11 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO throw new ISE( "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", workerNumber, - (stageDef.getStageNumber()), WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT, currentPhase + (stageDef.getStageNumber()), + stageDef.mustGatherResultKeyStatistics() + ? WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT + : WorkerStagePhase.READING_INPUT, + currentPhase ); } @@ -464,7 +472,8 @@ void fail() private void generateResultPartitionsAndBoundaries() { if (resultPartitions != null) { - throw new ISE("Result partitions have already been generated"); + log.debug("Partition boundaries already generated for stage %d", stageDef.getStageNumber()); + return; } final int stageNumber = stageDef.getStageNumber(); @@ -513,7 +522,9 @@ private boolean allPartitionStatisticsPresent() { return workerStagePhases.values() .stream() - .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES)) + .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) + || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) + || stagePhase.equals(WorkerStagePhase.RESULTS_READY)) .count() == workerCount; } @@ -523,7 +534,10 @@ private boolean allWorkOrderSent() return workerStagePhases.values() .stream() .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) - || stagePhase.equals(WorkerStagePhase.READING_INPUT)) + || stagePhase.equals(WorkerStagePhase.READING_INPUT) + || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) + || stagePhase.equals(WorkerStagePhase.RESULTS_READY) + ) .count() == workerCount; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java index 519de6bd9ddb..c4c98157f7d8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -26,17 +26,19 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.input.InputSpecSlicerFactory; import org.apache.druid.msq.input.MapInputSpecSlicer; import org.apache.druid.msq.input.stage.StageInputSpec; import org.apache.druid.msq.input.stage.StageInputSpecSlicer; import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.testing.InitializedNullHandlingTest; +import javax.annotation.Nonnull; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -115,6 +117,7 @@ public ControllerQueryKernelTester setupStage( break; case READING_INPUT: + controllerQueryKernel.createWorkOrders(stageId.getStageNumber(), null); controllerQueryKernel.startStage(stageId); for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { controllerQueryKernel.workOrdersSentForWorker(stageId, i); @@ -125,13 +128,22 @@ public ControllerQueryKernelTester setupStage( setupStage(stageNumber, ControllerStagePhase.READING_INPUT, true); if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { + + final ClusterByStatisticsCollector keyStatsCollector = getMockCollector( + stageNumber); + for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( stageId, i, - ClusterByStatisticsSnapshot.empty() + keyStatsCollector.snapshot() ); } + + for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { + controllerQueryKernel.partitionBoundariesSentForWorker(stageId, i); + } + } else { throw new IAE("Stage %d doesn't gather key result statistics", stageNumber); } @@ -168,6 +180,7 @@ public ControllerQueryKernelTester setupStage( return this; } + public ControllerQueryKernelTester init() { @@ -228,7 +241,9 @@ public boolean isSuccess() public void startStage(int stageNumber) { Preconditions.checkArgument(initialized); + controllerQueryKernel.createWorkOrders(stageNumber, null); controllerQueryKernel.startStage(new StageId(queryDefinition.getQueryId(), stageNumber)); + } public void startWorkOrder(int stageNumber) @@ -257,16 +272,7 @@ public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerN Preconditions.checkArgument(initialized); // Simulate 1000 keys being encountered in the data, so the kernel can generate some partitions. - final ClusterByStatisticsCollector keyStatsCollector = - queryDefinition.getStageDefinition(stageNumber).createResultKeyStatisticsCollector(10_000_000); - for (int i = 0; i < 1000; i++) { - final RowKey key = KeyTestUtils.createKey( - MockQueryDefinitionBuilder.STAGE_SIGNATURE, - String.valueOf(i) - ); - - keyStatsCollector.add(key, 1); - } + final ClusterByStatisticsCollector keyStatsCollector = getMockCollector(stageNumber); controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), @@ -302,16 +308,17 @@ public void assertStagePhase(int stageNumber, ControllerStagePhase expectedContr if (controllerStageTracker.getPhase() != expectedControllerStagePhase) { throw new ISE( StringUtils.format( - "Stage kernel for stage number %d is in %s phase which is different from the expected phase", + "Stage kernel for stage number %d is in %s phase which is different from the expected phase %s", stageNumber, - controllerStageTracker.getPhase() + controllerStageTracker.getPhase(), + expectedControllerStagePhase ) ); } } /** - * Checks if the state of the BaseControllerQueryKernel is initialized properly. Currently this is just stubbed to + * Checks if the state of the BaseControllerQueryKernel is initialized properly. Currently, this is just stubbed to * return true irrespective of the actual state */ private boolean isValidInitState() @@ -339,5 +346,27 @@ public void sendPartitionBoundariesForStageAndWorker(int stageNumber, int worker final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); controllerQueryKernel.partitionBoundariesSentForWorker(stageId, worker); } + + public List getRetriableWorkOrdersAndChangeState(int workeNumber, MSQFault msqFault) + { + return controllerQueryKernel.getRetriableWorkOrdersAndChangeState(workeNumber, msqFault); + } + + @Nonnull + private ClusterByStatisticsCollector getMockCollector(int stageNumber) + { + final ClusterByStatisticsCollector keyStatsCollector = + queryDefinition.getStageDefinition(stageNumber).createResultKeyStatisticsCollector(10_000_000); + for (int i = 0; i < 1000; i++) { + final RowKey key = KeyTestUtils.createKey( + MockQueryDefinitionBuilder.STAGE_SIGNATURE, + String.valueOf(i) + ); + keyStatsCollector.add(key, 1); + } + return keyStatsCollector; + } } + + } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java index f6046c847705..bd8a473e1774 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/MockQueryDefinitionBuilder.java @@ -135,7 +135,9 @@ public MockQueryDefinitionBuilder defineStage( .map(StageInputSpec::new).collect(Collectors.toList()); if (inputSpecs.isEmpty()) { - inputSpecs.add(new ControllerTestInputSpec()); + for (int i = 0; i < maxWorkers; i++) { + inputSpecs.add(new ControllerTestInputSpec()); + } } queryDefinitionBuilder.add( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java new file mode 100644 index 000000000000..8bb2942efe20 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.kernel.controller; + +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.kernel.WorkOrder; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nonnull; +import java.util.List; + +public class NonShufflingWorkersWithRetryKernelTests extends BaseControllerQueryKernelTest +{ + private static final UnknownFault RETRIABLE_FAULT = UnknownFault.forMessage(""); + + + @Test + public void workerFailedAfterInitialization() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + controllerQueryKernelTester.init(); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + Assert.assertTrue(controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size() == 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.NEW); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + } + + @Test + public void workerFailedBeforeAnyWorkOrdersSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + + } + + @Test + public void workerFailedBeforeAllWorkOrdersSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + } + + + @Test + public void workerFailedBeforeAnyResultsRecieved() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + + // workorders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + // fail one worker + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + + } + + @Test + public void workerFailedBeforeAllResultsRecieved() + { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + + // workorders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + // fail one worker + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } + + @Test + public void workerFailedBeforeFinished() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.RESULTS_READY); + controllerQueryKernelTester.init(); + + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size()); + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT).size()); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + } + + + @Test + public void workerFailedAfterFinished() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.FINISHED); + controllerQueryKernelTester.init(); + + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size()); + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT).size()); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); + + } + + @Nonnull + private ControllerQueryKernelTester getSimpleQueryDefinition() + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(2); + // 0 -> 1 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(2) + .addVertex(0, 1) + .defineStage(0, false, 2) // Ingestion only on one worker + .defineStage(1, false, 2) + .getQueryDefinitionBuilder() + .build() + ); + return controllerQueryKernelTester; + } + + + public void multipleWorkerFailedAfterInitialization() + { + + } + + public void multipleWorkerFailedBeforeAnyWorkOrdersSent() {} + + public void multipleWorkerFailedAfterOneWorkOrdersSent() {} + + public void multipleWorkerFailedBeforeAnyResultsStatsArrive() {} + + public void multipleWorkerFailedBeforeAllResultsStatsArrive() {} + + public void multipleWorkerFailedBeforeAnyPartitionBoundariesAreSent() {} + + + public void multipleWorkerFailedBeforeAllPartitionBoundariesAreSent() {} + + public void multipleWorkerFailedBeforeAnyResultsRecieved() {} + + public void multipleWorkerFailedBeforeAllResultsRecieved() {} + + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java new file mode 100644 index 000000000000..f3b35bd9a8c3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java @@ -0,0 +1,568 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.kernel.controller; + +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.kernel.WorkOrder; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nonnull; +import java.util.List; + +public class ShufflingWorkersWithRetryKernelTests extends BaseControllerQueryKernelTest +{ + + private static final UnknownFault RETRIABLE_FAULT = UnknownFault.forMessage(""); + + + @Test + public void workerFailedAfterInitialization() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.init(); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + Assert.assertTrue(controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size() == 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.NEW); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + } + + + @Test + public void workerFailedBeforeAnyWorkOrdersSent() + { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + } + + + @Test + public void workerFailedBeforeAllWorkOrdersSent() + { + + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + + } + + + @Test + public void workerFailedBeforeAnyResultsStatsArrive() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + + // workorders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + // fail one worker + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + @Test + public void workerFailedBeforeAllResultsStatsArrive() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + + // workorders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + // fail one worker + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + + @Test + public void workerFailedBeforeAnyPartitionBoundariesAreSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + + // workorders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + // fail one worker + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + + controllerQueryKernelTester.assertStagePhase( + 0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } + + + @Test + public void workerFailedBeforeAllPartitionBoundariesAreSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + + // workorders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + // fail one worker + + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } + + + @Test + public void workerFailedBeforeAnyResultsRecieved() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.init(); + + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + + } + + + @Test + public void workerFailedBeforeAllResultsRecieved() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.init(); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + + List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( + 0, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + + } + + + @Test + public void workerFailedBeforeFinished() + { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.RESULTS_READY); + controllerQueryKernelTester.init(); + + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size()); + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT).size()); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + + } + + @Test + public void workerFailedAfterFinished() + { + + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.FINISHED); + controllerQueryKernelTester.init(); + + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size()); + Assert.assertEquals(0, controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT).size()); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); + + } + + + @Nonnull + private ControllerQueryKernelTester getSimpleQueryDefinition(int numWorkers) + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(numWorkers); + // 0 -> 1 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(2) + .addVertex(0, 1) + .defineStage(0, true, numWorkers) // Ingestion only on one worker + .defineStage(1, true, numWorkers) + .getQueryDefinitionBuilder() + .build() + ); + return controllerQueryKernelTester; + } + + + @Test + public void multipleWorkerFailedAfterInitialization() + { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + controllerQueryKernelTester.init(); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + Assert.assertTrue(controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size() == 0); + Assert.assertTrue(controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT).size() == 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.NEW); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + + } + + @Test + public void multipleWorkerFailedBeforeAnyWorkOrdersSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + + + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + } + + @Test + public void multipleWorkerFailedBeforeAllWorkOrdersSent() + { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 2); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + } + + public void multipleWorkerFailedBeforeAnyResultsStatsArrive() {} + + public void multipleWorkerFailedBeforeAllResultsStatsArrive() {} + + public void multipleWorkerFailedBeforeAnyPartitionBoundariesAreSent() {} + + + public void multipleWorkerFailedBeforeAllPartitionBoundariesAreSent() {} + + public void multipleWorkerFailedBeforeAnyResultsRecieved() {} + + public void multipleWorkerFailedBeforeAllResultsRecieved() {} + + +} From f8943d33c3d28c7e6ebc4dc8d6fb26ab30b8cb50 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 11 Nov 2022 11:57:04 +0530 Subject: [PATCH 05/27] Adding java docs. --- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../org/apache/druid/msq/exec/Limits.java | 2 +- .../msq/indexing/MSQWorkerTaskLauncher.java | 97 +++++++++------ .../msq/indexing/error/MSQFaultUtils.java | 7 ++ .../controller/ControllerQueryKernel.java | 59 +++++++-- .../controller/ControllerStagePhase.java | 2 +- .../controller/ControllerStageTracker.java | 113 +++++++++++------- .../BaseControllerQueryKernelTest.java | 2 +- .../druid/indexing/common/task/Task.java | 2 +- 9 files changed, 192 insertions(+), 94 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 6963daac1777..6e8279043f3a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -584,7 +584,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault fault) { - List retriableWorkOrders = kernel.getRetriableWorkOrdersAndChangeState(worker, fault); + List retriableWorkOrders = kernel.getWorkInCaseWorkerElgibileForRetryElseThrow(worker, fault); if (retriableWorkOrders.size() != 0) { log.info("Submitting worker[%s] for relaunch because of fault[%s]", worker, fault); workerTaskLauncher.submitForRelaunch(worker); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index db6fe8463ead..cacd04481250 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -65,7 +65,7 @@ public class Limits public static final int TOTAL_RETRY_LIMIT = 30; /** - * Maximum worker retries. Initial run is not a retry. The worker will be spawned 1 + retryNumber of times before erroring out; + * Maximum worker retries. Initial run is not a retry. The worker will be spawned 1 + retryNumber of times before erroring out. */ public static int WORKER_RETRY_LIMIT = 2; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index c9077a2341de..17cde35c8083 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -60,7 +60,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.IntStream; @@ -74,7 +73,7 @@ public class MSQWorkerTaskLauncher private static final long LOW_FREQUENCY_CHECK_MILLIS = 2000; private static final long SWITCH_TO_LOW_FREQUENCY_CHECK_AFTER_MILLIS = 10000; private static final long SHUTDOWN_TIMEOUT_MS = Duration.ofMinutes(1).toMillis(); - private AtomicInteger currentRetryCount = new AtomicInteger(); + private int currentRetryCount = 0; // States for "state" variable. private enum State @@ -125,9 +124,6 @@ private enum State // retry worker set private final Set retryWorkerSet = ConcurrentHashMap.newKeySet(); - private final Set retryingWorkerSet = ConcurrentHashMap.newKeySet(); - - private final Map> workerToTaskIds = new ConcurrentHashMap<>(); private final RetryTask retryTask; @@ -244,12 +240,22 @@ public void launchTasksIfNeeded(final int taskCount) throws InterruptedException } } - + /** + * Queues worker for relaunch. A noop if the worker is already in the queue. + * + * @param workerNumber + */ public void submitForRelaunch(int workerNumber) { retryWorkerSet.add(workerNumber); } + /** + * Blocks the call untill the worker tasks are ready to be contacted for work. + * + * @param workerSet + * @throws InterruptedException + */ public void waitUntilWorkersReady(Set workerSet) throws InterruptedException { synchronized (taskIds) { @@ -289,7 +295,7 @@ private void mainLoop() updateTaskTrackersAndTaskIds(); checkForErroneousTasks(); relaunchTasks(); - cleanFailedTasks(); + cleanFailedTasksWhichAreRelaunched(); } catch (Throwable e) { state.set(State.STOPPED); @@ -465,7 +471,7 @@ private void checkForErroneousTasks() final Map.Entry taskEntry = taskTrackerIterator.next(); final String taskId = taskEntry.getKey(); final TaskTracker tracker = taskEntry.getValue(); - if (tracker.toRetry()) { + if (tracker.isRetrying()) { continue; } @@ -473,19 +479,20 @@ private void checkForErroneousTasks() removeWorkerFromFullyStartedWorkers(tracker); final String errorMessage = StringUtils.format("Task [%s] status missing", taskId); log.info(errorMessage + ". Trying to relaunch the worker"); + tracker.enableRetrying(); retryTask.retry( tracker.msqWorkerTask, UnknownFault.forMessage(errorMessage) ); - tracker.retry(); + } else if (tracker.didRunTimeOut(maxTaskStartDelayMillis) && !canceledWorkerTasks.contains(taskId)) { removeWorkerFromFullyStartedWorkers(tracker); throw new MSQException(new TaskStartTimeoutFault(numTasks + 1)); } else if (tracker.didFail() && !canceledWorkerTasks.contains(taskId)) { removeWorkerFromFullyStartedWorkers(tracker); - log.info("Task[%s] failed because %s", taskId, tracker.status.getErrorMsg()); + log.info("Task[%s] failed because %s. Trying to relaunch the worker", taskId, tracker.status.getErrorMsg()); + tracker.enableRetrying(); retryTask.retry(tracker.msqWorkerTask, new WorkerFailedFault(taskId, tracker.status.getErrorMsg())); - tracker.retry(); } } } @@ -506,10 +513,6 @@ private void relaunchTasks() int worker = iterator.next(); workerToTaskIds.compute(worker, (workerId, taskHistory) -> { - if (retryingWorkerSet.contains(worker)) { - return taskHistory; - } - if (taskHistory == null || taskHistory.isEmpty()) { throw new ISE("TaskHistory cannot by null for worker %d", workerId); } @@ -519,33 +522,22 @@ private void relaunchTasks() if (tracker == null) { throw new ISE("Did not find taskTracker for latest taskId[%s]", latestTaskId); } + // if task is not failed donot retry if (!tracker.isComplete()) { - // if task is not failed donot retry return taskHistory; } MSQWorkerTask toRetry = tracker.msqWorkerTask; + MSQWorkerTask retryTask = toRetry.getRetryTask(); - if (toRetry.getRetryCount() > Limits.WORKER_RETRY_LIMIT) { - throw new MSQException(new TooManyWorkerRetriesFault( - Limits.WORKER_RETRY_LIMIT, - toRetry.getId(), - toRetry.getWorkerNumber(), - tracker.status.getErrorMsg() - )); - } - if (currentRetryCount.get() > Limits.TOTAL_RETRY_LIMIT) { - throw new MSQException(new TotalRetryLimitExceededFault( - Limits.TOTAL_RETRY_LIMIT, - currentRetryCount.get(), - toRetry.getId(), - tracker.status.getErrorMsg() - )); - } + // check retry limits + checkRetryLimits(tracker, toRetry); + // clean up trackers and tasks + tasksToCleanup.add(latestTaskId); taskTrackers.remove(latestTaskId); - currentRetryCount.addAndGet(1); + currentRetryCount += 1; taskTrackers.put(retryTask.getId(), new TaskTracker(retryTask.getWorkerNumber(), retryTask)); context.workerManager().run(retryTask.getId(), retryTask); taskHistory.add(retryTask.getId()); @@ -562,10 +554,30 @@ private void relaunchTasks() } } + private void checkRetryLimits(TaskTracker tracker, MSQWorkerTask toRetry) + { + if (toRetry.getRetryCount() > Limits.WORKER_RETRY_LIMIT) { + throw new MSQException(new TooManyWorkerRetriesFault( + Limits.WORKER_RETRY_LIMIT, + toRetry.getId(), + toRetry.getWorkerNumber(), + tracker.status.getErrorMsg() + )); + } + if (currentRetryCount > Limits.TOTAL_RETRY_LIMIT) { + throw new MSQException(new TotalRetryLimitExceededFault( + Limits.TOTAL_RETRY_LIMIT, + currentRetryCount, + toRetry.getId(), + tracker.status.getErrorMsg() + )); + } + } + private void shutDownTasks() { - cleanFailedTasks(); + cleanFailedTasksWhichAreRelaunched(); for (final Map.Entry taskEntry : taskTrackers.entrySet()) { final String taskId = taskEntry.getKey(); final TaskTracker tracker = taskEntry.getValue(); @@ -578,7 +590,10 @@ private void shutDownTasks() } - private void cleanFailedTasks() + /** + * Cleans the task indentified in {@link MSQWorkerTaskLauncher#relaunchTasks()} for realunch. Asks the overlord to cancel the task. + */ + private void cleanFailedTasksWhichAreRelaunched() { Iterator tasksToCancel = tasksToCleanup.iterator(); while (tasksToCancel.hasNext()) { @@ -676,12 +691,20 @@ && unknownLocation() && System.currentTimeMillis() - startTimeMs > maxTaskStartDelayMillis; } - public void retry() + /** + * Enables retrying for the task + */ + public void enableRetrying() { isRetrying = true; } - public boolean toRetry() + /** + * Checks is the task is retrying, + * + * @return + */ + public boolean isRetrying() { return isRetrying; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java index f6cb04629a08..35a97b0fc002 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java @@ -24,6 +24,9 @@ public class MSQFaultUtils public static final String ERROR_CODE_DELIMITER = ": "; + /** + * Generate string message with error code delimited by {@link MSQFaultUtils#ERROR_CODE_DELIMITER} + */ public static String generateMessageWithErrorCode(MSQFault msqFault) { final String message = msqFault.getErrorMessage(); @@ -35,6 +38,10 @@ public static String generateMessageWithErrorCode(MSQFault msqFault) } } + /** + * Gets the error code from the message. If the messay is empty or null, {@link UnknownFault#CODE} is returned. This method + * does not gurantee that the error code we get out of the message is a valid error code. + */ public static String getErrorCodeFromMessage(String message) { if (message == null || message.isEmpty()) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 92a2df339831..4c58f4abc4b0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -118,13 +118,19 @@ public class ControllerQueryKernel private final Set effectivelyFinishedStages = new HashSet<>(); + /** + * Store the work orders for the stage so that we can retrieve that in case of worker retry + */ private final Map> stageWorkOrders; + /** + * {@link MSQFault#getErrorCode()} which are retried. + */ private final Set retriableErrorCodes = ImmutableSet.of(CanceledFault.CODE, UnknownFault.CODE, WorkerRpcFailedFault.CODE ); - public ControllerQueryKernel(final QueryDefinition queryDef,final int partitionStatisticsMaxRetainedBytes) + public ControllerQueryKernel(final QueryDefinition queryDef, final int partitionStatisticsMaxRetainedBytes) { this.queryDef = queryDef; this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes; @@ -345,20 +351,28 @@ public ReadablePartitions getResultPartitionsForStage(final StageId stageId) return getStageKernelOrThrow(stageId).getResultPartitions(); } + /** + * Delegates call to {@link ControllerStageTracker#getWorkersToSendParitionBoundaries()} + */ public IntSet getWorkersToSendPartitionBoundaries(final StageId stageId) { - return getStageKernelOrThrow(stageId).getWorkersForPartitionBoundaries(); + return getStageKernelOrThrow(stageId).getWorkersToSendParitionBoundaries(); } - - public void partitionBoundariesSentForWorker(final StageId stageId, int worker) + /** + * Delegates call to {@link ControllerQueryKernel#workOrdersSentForWorker(StageId, int)} + */ + public void workOrdersSentForWorker(final StageId stageId, int worker) { - getStageKernelOrThrow(stageId).partitionBoundariesSentForWorker(worker); + getStageKernelOrThrow(stageId).workOrderSentForWorker(worker); } - public void workOrdersSentForWorker(final StageId stageId, int worker) + /** + * Delegates call to {@link ControllerStageTracker#partitionBoundariesSentForWorker(int)} ()} + */ + public void partitionBoundariesSentForWorker(final StageId stageId, int worker) { - getStageKernelOrThrow(stageId).workOrderSentForWorker(worker); + getStageKernelOrThrow(stageId).partitionBoundariesSentForWorker(worker); } /** @@ -537,6 +551,7 @@ private void transitionStageKernel(StageId stageId, ControllerStagePhase newPhas } } + // might need to change this if (ControllerStagePhase.isPostReadingPhase(newPhase)) { // Once the stage has consumed all the data/input from its dependent stages, we remove it from all the stages // whose input it was dependent on @@ -600,7 +615,18 @@ private static Map> computeStageOutflowMap(final QueryDefi return retVal; } - public List getRetriableWorkOrdersAndChangeState(int workerNumber, MSQFault msqFault) + /** + * Checks the {@link MSQFault#getErrorCode()} is eligible for retry. + *
+ * If yes, transitions the stage to{@link ControllerStagePhase#RETRYING} and returns all the {@link WorkOrder} + *
+ * else throw {@link MSQException} + * + * @param workerNumber + * @param msqFault + * @return List of {@link WorkOrder} that needs to be retried. + */ + public List getWorkInCaseWorkerElgibileForRetryElseThrow(int workerNumber, MSQFault msqFault) { final String errorCode; @@ -611,14 +637,24 @@ public List getRetriableWorkOrdersAndChangeState(int workerNumber, MS } if (retriableErrorCodes.contains(errorCode)) { - return getRetriableWorkOrdersAndChangeState(workerNumber); + return getWorkInCaseWorkerElgibileForRetryElseThrow(workerNumber); } else { throw new MSQException(msqFault); } } - private List getRetriableWorkOrdersAndChangeState(int worker) + /** + * Gets all the stages currently being tracked and filtres out all effectively finished stages. + *
+ * From the remaining stages, checks if (stage,worker) needs to be retried. + *
+ * If yes adds the workOrder for that stage to the return list and transitions the stage kernel to {@link ControllerStagePhase#RETRYING} + * + * @param worker + * @return List of {@link WorkOrder} that needs to be retried. + */ + private List getWorkInCaseWorkerElgibileForRetryElseThrow(int worker) { List trackedSet = new ArrayList<>(getActiveStages()); // no need to retry effectively finished stages @@ -626,8 +662,8 @@ private List getRetriableWorkOrdersAndChangeState(int worker) trackedSet.removeAll(getEffictivelyFinishedStages); List workOrders = new ArrayList<>(); - for (StageId stageId : trackedSet) { + for (StageId stageId : trackedSet) { ControllerStageTracker controllerStageTracker = getStageKernelOrThrow(stageId); if (ControllerStagePhase.RETRYING.canTransitionFrom(controllerStageTracker.getPhase()) && controllerStageTracker.retryIfNeeded(worker)) { @@ -635,7 +671,6 @@ private List getRetriableWorkOrdersAndChangeState(int worker) // should be a no-op. Calling for code patterns. transitionStageKernel(stageId, ControllerStagePhase.RETRYING); } - } return workOrders; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index e517fa50f071..151df2a93065 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java @@ -87,7 +87,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) } }, - // Stage currently under retry. Prior phases stages did not publish its final results yet. + // Stage currently under retry. priorPhase did not publish its final results yet. RETRYING { @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index 7c7feca530fa..707a7905c445 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -64,9 +64,11 @@ class ControllerStageTracker private final int workerCount; private final WorkerInputs workerInputs; - private final Int2ObjectMap workerStagePhases = new Int2ObjectOpenHashMap<>(); - private final IntSet resultstatisticsRecieved = new IntAVLTreeSet(); + // worker-> workerStagePhase + private final Int2ObjectMap workerToPhase = new Int2ObjectOpenHashMap<>(); + + private final IntSet workersWithResultKeyStatistics = new IntAVLTreeSet(); private ControllerStagePhase phase = ControllerStagePhase.NEW; @@ -109,9 +111,14 @@ private ControllerStageTracker( } } + /** + * Initalized stage for each worker to {@link WorkerStagePhase#NEW} + * + * @param workerCount + */ private void initializeWorkerState(int workerCount) { - IntStream.range(0, workerCount).forEach(wokerNumber -> workerStagePhases.put(wokerNumber, WorkerStagePhase.NEW)); + IntStream.range(0, workerCount).forEach(wokerNumber -> workerToPhase.put(wokerNumber, WorkerStagePhase.NEW)); } /** @@ -182,24 +189,35 @@ ClusterByPartitions getResultPartitionBoundaries() } - IntSet getWorkersForPartitionBoundaries() + /** + * Get workers which need to be sent partition boundaries + * + * @return + */ + IntSet getWorkersToSendParitionBoundaries() { if (!getStageDefinition().doesShuffle()) { throw new ISE("Result partition information is not relevant to this stage because it does not shuffle"); } IntAVLTreeSet workers = new IntAVLTreeSet(); - for (Integer worker : workerStagePhases.keySet()) { - if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.equals(workerStagePhases.get(worker))) { + for (Integer worker : workerToPhase.keySet()) { + if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.equals(workerToPhase.get(worker))) { workers.add(worker); } } return workers; } + /** + * Indicates that the work order for worker has been sent. Transistions the state to {@link WorkerStagePhase#READING_INPUT} + * if no more work orders need to be sent. + * + * @param worker + */ void workOrderSentForWorker(int worker) { - workerStagePhases.compute(worker, (wk, state) -> { + workerToPhase.compute(worker, (wk, state) -> { if (state == null) { throw new ISE("Worker[%d] not found for stage[%s]", wk, stageDef.getStageNumber()); } @@ -214,19 +232,23 @@ void workOrderSentForWorker(int worker) return WorkerStagePhase.READING_INPUT; }); if (phase != ControllerStagePhase.READING_INPUT) { - if (allWorkOrderSent()) { - // if all the work orders are sent, change state to reading input from retrying + if (workOrdersNeedToBeSent()) { + // if no more work orders need to be sent, change state to reading input from retrying. transitionTo(ControllerStagePhase.READING_INPUT); } } } - + /** + * Indicates that the partition boundaries for worker has been sent. + * + * @param worker + */ void partitionBoundariesSentForWorker(int worker) { - workerStagePhases.compute(worker, (wk, state) -> { + workerToPhase.compute(worker, (wk, state) -> { if (state == null) { throw new ISE("Worker[%d] not found for stage[%s]", wk, stageDef.getStageNumber()); } @@ -337,7 +359,7 @@ ControllerStagePhase addResultKeyStatisticsForWorker( throw new ISE("Cannot add result key statistics from stage [%s]", phase); } - WorkerStagePhase currentPhase = workerStagePhases.get(workerNumber); + WorkerStagePhase currentPhase = workerToPhase.get(workerNumber); if (currentPhase == null) { throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); @@ -345,10 +367,10 @@ ControllerStagePhase addResultKeyStatisticsForWorker( try { if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.canTransitionFrom(currentPhase)) { - workerStagePhases.put(workerNumber, WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); + workerToPhase.put(workerNumber, WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); // if stats already recieved for worker, donot update the sketch. - if (resultstatisticsRecieved.add(workerNumber)) { + if (workersWithResultKeyStatistics.add(workerNumber)) { resultKeyStatisticsCollector.addAll(snapshot); } @@ -394,7 +416,7 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO throw new NullPointerException("resultObject must not be null"); } - WorkerStagePhase currentPhase = workerStagePhases.get(workerNumber); + WorkerStagePhase currentPhase = workerToPhase.get(workerNumber); if (currentPhase == null) { throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); } @@ -407,7 +429,7 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO if (WorkerStagePhase.RESULTS_READY.canTransitionFrom(currentPhase)) { - workerStagePhases.put(workerNumber, WorkerStagePhase.RESULTS_READY); + workerToPhase.put(workerNumber, WorkerStagePhase.RESULTS_READY); if (this.resultObject == null) { this.resultObject = resultObject; } else { @@ -437,10 +459,10 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO private boolean allResultsPresent() { - return workerStagePhases.values() - .stream() - .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.RESULTS_READY)) - .count() == workerCount; + return workerToPhase.values() + .stream() + .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.RESULTS_READY)) + .count() == workerCount; } /** @@ -518,27 +540,33 @@ private void generateResultPartitionsAndBoundaries() } } + /** + * True if all partitions stats are present, else false. + */ private boolean allPartitionStatisticsPresent() { - return workerStagePhases.values() - .stream() - .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) - || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) - || stagePhase.equals(WorkerStagePhase.RESULTS_READY)) - .count() + return workerToPhase.values() + .stream() + .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) + || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) + || stagePhase.equals(WorkerStagePhase.RESULTS_READY)) + .count() == workerCount; } - private boolean allWorkOrderSent() + /** + * True if work orders needs to be sent else false. + */ + private boolean workOrdersNeedToBeSent() { - return workerStagePhases.values() - .stream() - .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) - || stagePhase.equals(WorkerStagePhase.READING_INPUT) - || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) - || stagePhase.equals(WorkerStagePhase.RESULTS_READY) - ) - .count() + return workerToPhase.values() + .stream() + .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) + || stagePhase.equals(WorkerStagePhase.READING_INPUT) + || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) + || stagePhase.equals(WorkerStagePhase.RESULTS_READY) + ) + .count() == workerCount; } @@ -567,6 +595,11 @@ private void transitionTo(final ControllerStagePhase newPhase) } } + /** + * Retry true if the worker needs to be retried based on state else returns false. + * + * @param workerNumber + */ public boolean retryIfNeeded(int workerNumber) { if (phase.equals(ControllerStagePhase.FINISHED) || phase.equals(ControllerStagePhase.RESULTS_READY)) { @@ -578,20 +611,20 @@ public boolean retryIfNeeded(int workerNumber) return false; } - if (workerStagePhases.get(workerNumber).equals(WorkerStagePhase.RESULTS_READY) - || workerStagePhases.get(workerNumber).equals(WorkerStagePhase.FINISHED)) { + if (workerToPhase.get(workerNumber).equals(WorkerStagePhase.RESULTS_READY) + || workerToPhase.get(workerNumber).equals(WorkerStagePhase.FINISHED)) { // do nothing return false; } - workerStagePhases.put(workerNumber, WorkerStagePhase.NEW); + workerToPhase.put(workerNumber, WorkerStagePhase.NEW); transitionTo(ControllerStagePhase.RETRYING); return true; } - public boolean isTrackingWorker(int workerNumber) + private boolean isTrackingWorker(int workerNumber) { - return workerStagePhases.get(workerNumber) != null; + return workerToPhase.get(workerNumber) != null; } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java index c4c98157f7d8..a822066454dd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -349,7 +349,7 @@ public void sendPartitionBoundariesForStageAndWorker(int stageNumber, int worker public List getRetriableWorkOrdersAndChangeState(int workeNumber, MSQFault msqFault) { - return controllerQueryKernel.getRetriableWorkOrdersAndChangeState(workeNumber, msqFault); + return controllerQueryKernel.getWorkInCaseWorkerElgibileForRetryElseThrow(workeNumber, msqFault); } @Nonnull diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 63a6f781b2b4..9a9d49670d65 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -178,7 +178,7 @@ default int getPriority() boolean isReady(TaskActionClient taskActionClient) throws Exception; /** - * Returns wheather or not this task can restore its progress from its on-disk working directory. Restorable tasks + * Returns whether this task can restore its progress from its on-disk working directory. Restorable tasks * may be started with a non-empty working directory. Tasks that exit uncleanly may still have a chance to attempt * restores, meaning that restorable tasks should be able to deal with potentially partially written on-disk state. */ From c92b21749c18b665b8d85448645662413264ac5a Mon Sep 17 00:00:00 2001 From: cryptoe Date: Sun, 13 Nov 2022 23:10:47 +0530 Subject: [PATCH 06/27] Finishing up state test cases. --- .../controller/ControllerStageTracker.java | 17 +- .../BaseControllerQueryKernelTest.java | 65 ++- .../ControllerQueryKernelTests.java | 81 +-- ...nShufflingWorkersWithRetryKernelTests.java | 240 ++++++--- .../ShufflingWorkersWithRetryKernelTests.java | 510 +++++++++++------- 5 files changed, 591 insertions(+), 322 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index 707a7905c445..cfce3fdfd84b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -421,14 +421,17 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); } - // This is unidirectional flow of data. While this works in the current state of MSQ where partial fault tolerance - // is implemented and a query flows in one direction only, rolling back of workers' state and query kernel's - // phase should be allowed to fully support fault tolerance in cases such as: - // 1. Rolling back worker's state in case it fails (and then retries) - // 2. Rolling back query kernel's phase in case the results are lost (and needs workers to retry the computation) - - if (WorkerStagePhase.RESULTS_READY.canTransitionFrom(currentPhase)) { + + if (stageDef.mustGatherResultKeyStatistics() && currentPhase == WorkerStagePhase.READING_INPUT) { + throw new ISE( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + workerNumber, + (stageDef.getStageNumber()), + WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT, + currentPhase + ); + } workerToPhase.put(workerNumber, WorkerStagePhase.RESULTS_READY); if (this.resultObject == null) { this.resultObject = resultObject; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java index a822066454dd..212a3a1d49aa 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.input.InputSpecSlicerFactory; import org.apache.druid.msq.input.MapInputSpecSlicer; import org.apache.druid.msq.input.stage.StageInputSpec; @@ -37,6 +38,7 @@ import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; import javax.annotation.Nonnull; import java.util.HashSet; @@ -47,6 +49,7 @@ public class BaseControllerQueryKernelTest extends InitializedNullHandlingTest { + public static final UnknownFault RETRIABLE_FAULT = UnknownFault.forMessage(""); public ControllerQueryKernelTester testControllerQueryKernel(int numWorkers) { @@ -267,29 +270,34 @@ public void finishStage(int stageNumber, boolean strict) controllerQueryKernel.finishStage(new StageId(queryDefinition.getQueryId(), stageNumber), strict); } - public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerNumber) + + public void addResultKeyStatisticsForStageAndWorkers(int stageNumber, int... workers) { Preconditions.checkArgument(initialized); - // Simulate 1000 keys being encountered in the data, so the kernel can generate some partitions. - final ClusterByStatisticsCollector keyStatsCollector = getMockCollector(stageNumber); + for (int worker : workers) { + // Simulate 1000 keys being encountered in the data, so the kernel can generate some partitions. + final ClusterByStatisticsCollector keyStatsCollector = getMockCollector(stageNumber); - controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( - new StageId(queryDefinition.getQueryId(), stageNumber), - workerNumber, - keyStatsCollector.snapshot() - ); + controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( + new StageId(queryDefinition.getQueryId(), stageNumber), + worker, + keyStatsCollector.snapshot() + ); + } } - public void setResultsCompleteForStageAndWorker(int stageNumber, int workerNumber) + public void setResultsCompleteForStageAndWorkerAndWorkers(int stageNumber, int... workers) { Preconditions.checkArgument(initialized); final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); - controllerQueryKernel.setResultsCompleteForStageAndWorker( - stageId, - workerNumber, - new Object() - ); + for (int worker : workers) { + controllerQueryKernel.setResultsCompleteForStageAndWorker( + stageId, + worker, + new Object() + ); + } } public void failStage(int stageNumber) @@ -333,18 +341,22 @@ private Set mapStageIdsToStageNumbers(List stageIds) .collect(Collectors.toSet()); } - public void sendWorkOrdersForWorker(int stageNumber, int worker) + public void sendWorkOrdersForWorkers(int stageNumber, int... workers) { Preconditions.checkArgument(initialized); final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); - controllerQueryKernel.workOrdersSentForWorker(stageId, worker); + for (int worker : workers) { + controllerQueryKernel.workOrdersSentForWorker(stageId, worker); + } } - public void sendPartitionBoundariesForStageAndWorker(int stageNumber, int worker) + public void sendPartitionBoundariesForStageAndWorkers(int stageNumber, int... workers) { Preconditions.checkArgument(initialized); final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); - controllerQueryKernel.partitionBoundariesSentForWorker(stageId, worker); + for (int worker : workers) { + controllerQueryKernel.partitionBoundariesSentForWorker(stageId, worker); + } } public List getRetriableWorkOrdersAndChangeState(int workeNumber, MSQFault msqFault) @@ -352,6 +364,23 @@ public List getRetriableWorkOrdersAndChangeState(int workeNumber, MSQ return controllerQueryKernel.getWorkInCaseWorkerElgibileForRetryElseThrow(workeNumber, msqFault); } + public void failWorkerAndAssertWorkOrderes(int workeNumber, int retriedStage) + { + // fail one worker + List workOrderList = getRetriableWorkOrdersAndChangeState( + workeNumber, + RETRIABLE_FAULT + ); + + // does not enable the current stage to enable running from start + Assert.assertTrue(createAndGetNewStageNumbers().size() == 0); + // only work order of failed worker should be there + Assert.assertTrue(workOrderList.size() == 1); + Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == workeNumber); + Assert.assertTrue(workOrderList.get(0).getStageNumber() == retriedStage); + + } + @Nonnull private ClusterByStatisticsCollector getMockCollector(int stageNumber) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java index 48cc7af692bc..835c7ea0d4c6 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java @@ -149,11 +149,11 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(0), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); @@ -161,17 +161,17 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(1), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(1); - controllerQueryKernelTester.sendWorkOrdersForWorker(1, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(1, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.sendWorkOrdersForWorker(1, 1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(1, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(1, 1); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(1, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(1, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(1, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(1, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(1, 1); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.RESULTS_READY); newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); @@ -180,8 +180,8 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(0), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(2); controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.sendWorkOrdersForWorker(2, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(2, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(2, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(2, 0); controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); @@ -193,16 +193,16 @@ public void testCompleteDAGExecutionForMultipleWorkers() controllerQueryKernelTester.startStage(3); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.startWorkOrder(3); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(3, 1); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(3, 0); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(3, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(3, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(3, 0); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(3, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(3, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(3, 1); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(1); @@ -231,20 +231,20 @@ public void testTransitionsInShufflingStagesAndMultipleWorkers() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(0, false); @@ -268,19 +268,22 @@ public void testPrematureResultsComplete() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - Assert.assertThrows(StringUtils.format( - "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", - 1, - 0, - WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES, - WorkerStagePhase.NEW - ) - , ISE.class, () -> controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0)); + Assert.assertThrows( + StringUtils.format( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + 1, + 0, + WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES, + WorkerStagePhase.NEW + ), + ISE.class, + () -> controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0) + ); } @@ -403,7 +406,7 @@ private static void transitionNewToResultsComplete(ControllerQueryKernelTester q { queryKernelTester.startStage(stageNumber); queryKernelTester.startWorkOrder(stageNumber); - queryKernelTester.setResultsCompleteForStageAndWorker(stageNumber, 0); + queryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(stageNumber, 0); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java index 8bb2942efe20..7137aee7f107 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java @@ -19,23 +19,20 @@ package org.apache.druid.msq.kernel.controller; -import org.apache.druid.msq.indexing.error.UnknownFault; -import org.apache.druid.msq.kernel.WorkOrder; + import org.junit.Assert; import org.junit.Test; import javax.annotation.Nonnull; -import java.util.List; public class NonShufflingWorkersWithRetryKernelTests extends BaseControllerQueryKernelTest { - private static final UnknownFault RETRIABLE_FAULT = UnknownFault.forMessage(""); @Test - public void workerFailedAfterInitialization() + public void testWorkerFailedAfterInitialization() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.init(); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 @@ -47,9 +44,9 @@ public void workerFailedAfterInitialization() } @Test - public void workerFailedBeforeAnyWorkOrdersSent() + public void testWorkerFailedBeforeAnyWorkOrdersSent() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.init(); controllerQueryKernelTester.createAndGetNewStageNumbers(); @@ -57,47 +54,47 @@ public void workerFailedBeforeAnyWorkOrdersSent() controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @Test - public void workerFailedBeforeAllWorkOrdersSent() + public void testWorkerFailedBeforeAllWorkOrdersSent() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.init(); controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @Test - public void workerFailedBeforeAnyResultsRecieved() + public void testWorkerFailedBeforeAnyResultsRecieved() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); // workorders sent for both stage controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); @@ -106,27 +103,18 @@ public void workerFailedBeforeAnyResultsRecieved() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); // fail one worker - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -135,46 +123,37 @@ public void workerFailedBeforeAnyResultsRecieved() } @Test - public void workerFailedBeforeAllResultsRecieved() + public void testWorkerFailedBeforeAllResultsRecieved() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); // workorders sent for both stage controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); // fail one worker - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); } @Test - public void workerFailedBeforeFinished() + public void testWorkerFailedBeforeFinished() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.setupStage(0, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.init(); @@ -186,9 +165,9 @@ public void workerFailedBeforeFinished() @Test - public void workerFailedAfterFinished() + public void testWorkerFailedAfterFinished() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.setupStage(0, ControllerStagePhase.FINISHED); controllerQueryKernelTester.init(); @@ -199,44 +178,157 @@ public void workerFailedAfterFinished() } - @Nonnull - private ControllerQueryKernelTester getSimpleQueryDefinition() + + @Test + public void testMultipleWorkersFailedAfterInitialization() { - ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(2); - // 0 -> 1 - controllerQueryKernelTester.queryDefinition( - new MockQueryDefinitionBuilder(2) - .addVertex(0, 1) - .defineStage(0, false, 2) // Ingestion only on one worker - .defineStage(1, false, 2) - .getQueryDefinitionBuilder() - .build() - ); - return controllerQueryKernelTester; + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + controllerQueryKernelTester.init(); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + Assert.assertTrue(controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT).size() == 0); + Assert.assertTrue(controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT).size() == 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.NEW); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1 + && controllerQueryKernelTester.createAndGetNewStageNumbers().contains(0)); + } + + @Test + public void testMultipleWorkersFailedBeforeAnyWorkOrdersSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT); + + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + } + + @Test + public void testMulttipleWorkerFailedBeforeAllWorkOrdersSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(2, RETRIABLE_FAULT); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + } - public void multipleWorkerFailedAfterInitialization() + @Test + public void testMultipleWorkersFailedBeforeAnyResultsRecieved() { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } - public void multipleWorkerFailedBeforeAnyWorkOrdersSent() {} + @Test + public void testMultipleWorkersFailedBeforeAllResultsRecieved() + { - public void multipleWorkerFailedAfterOneWorkOrdersSent() {} + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); - public void multipleWorkerFailedBeforeAnyResultsStatsArrive() {} + // workorders sent for all stages + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); - public void multipleWorkerFailedBeforeAllResultsStatsArrive() {} + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - public void multipleWorkerFailedBeforeAnyPartitionBoundariesAreSent() {} + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(2, 0); + // should be no op + Assert.assertTrue(controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT).size() == 0); - public void multipleWorkerFailedBeforeAllPartitionBoundariesAreSent() {} + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - public void multipleWorkerFailedBeforeAnyResultsRecieved() {} + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - public void multipleWorkerFailedBeforeAllResultsRecieved() {} + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } + + + @Nonnull + private ControllerQueryKernelTester getSimpleQueryDefinition(int numWorkers) + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(numWorkers); + // 0 -> 1 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(2) + .addVertex(0, 1) + .defineStage(0, false, numWorkers) + .defineStage(1, false, numWorkers) + .getQueryDefinitionBuilder() + .build() + ); + return controllerQueryKernelTester; + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java index f3b35bd9a8c3..143d4467bf37 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java @@ -19,22 +19,16 @@ package org.apache.druid.msq.kernel.controller; -import org.apache.druid.msq.indexing.error.UnknownFault; -import org.apache.druid.msq.kernel.WorkOrder; import org.junit.Assert; import org.junit.Test; import javax.annotation.Nonnull; -import java.util.List; public class ShufflingWorkersWithRetryKernelTests extends BaseControllerQueryKernelTest { - private static final UnknownFault RETRIABLE_FAULT = UnknownFault.forMessage(""); - - @Test - public void workerFailedAfterInitialization() + public void testWorkerFailedAfterInitialization() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.init(); @@ -49,7 +43,7 @@ public void workerFailedAfterInitialization() @Test - public void workerFailedBeforeAnyWorkOrdersSent() + public void testWorkerFailedBeforeAnyWorkOrdersSent() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); @@ -60,29 +54,28 @@ public void workerFailedBeforeAnyWorkOrdersSent() controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @Test - public void workerFailedBeforeAllWorkOrdersSent() + public void testWorkerFailedBeforeAllWorkOrdersSent() { @@ -92,32 +85,30 @@ public void workerFailedBeforeAllWorkOrdersSent() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); - } @Test - public void workerFailedBeforeAnyResultsStatsArrive() + public void testWorkerFailedBeforeAnyResultsStatsArrive() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); @@ -127,44 +118,33 @@ public void workerFailedBeforeAnyResultsStatsArrive() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - // fail one worker - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); } + @Test - public void workerFailedBeforeAllResultsStatsArrive() + public void testWorkerFailedBeforeAllResultsStatsArrive() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); @@ -172,39 +152,29 @@ public void workerFailedBeforeAllResultsStatsArrive() controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); // fail one worker - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -213,7 +183,7 @@ public void workerFailedBeforeAllResultsStatsArrive() @Test - public void workerFailedBeforeAnyPartitionBoundariesAreSent() + public void testWorkerFailedBeforeAnyPartitionBoundariesAreSent() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); @@ -221,43 +191,32 @@ public void workerFailedBeforeAnyPartitionBoundariesAreSent() controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); // fail one worker - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); - controllerQueryKernelTester.assertStagePhase( - 0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -265,7 +224,7 @@ public void workerFailedBeforeAnyPartitionBoundariesAreSent() @Test - public void workerFailedBeforeAllPartitionBoundariesAreSent() + public void testWorkerFailedBeforeAllPartitionBoundariesAreSent() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); @@ -273,44 +232,35 @@ public void workerFailedBeforeAllPartitionBoundariesAreSent() controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); // fail one worker - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -318,34 +268,25 @@ public void workerFailedBeforeAllPartitionBoundariesAreSent() @Test - public void workerFailedBeforeAnyResultsRecieved() + public void testWorkerFailedBeforeAnyResultsRecieved() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.init(); - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); @@ -355,7 +296,7 @@ public void workerFailedBeforeAnyResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -365,35 +306,26 @@ public void workerFailedBeforeAnyResultsRecieved() @Test - public void workerFailedBeforeAllResultsRecieved() + public void testWorkerFailedBeforeAllResultsRecieved() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.init(); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); - List workOrderList = controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState( - 0, - RETRIABLE_FAULT - ); - - // does not enable the current stage to enable running from start - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - // only work order of failed worker should be there - Assert.assertTrue(workOrderList.size() == 1); - Assert.assertTrue(workOrderList.get(0).getWorkerNumber() == 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); @@ -403,7 +335,7 @@ public void workerFailedBeforeAllResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -413,7 +345,7 @@ public void workerFailedBeforeAllResultsRecieved() @Test - public void workerFailedBeforeFinished() + public void testWorkerFailedBeforeFinished() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); @@ -429,7 +361,7 @@ public void workerFailedBeforeFinished() } @Test - public void workerFailedAfterFinished() + public void testWorkerFailedAfterFinished() { @@ -444,26 +376,8 @@ public void workerFailedAfterFinished() } - - @Nonnull - private ControllerQueryKernelTester getSimpleQueryDefinition(int numWorkers) - { - ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(numWorkers); - // 0 -> 1 - controllerQueryKernelTester.queryDefinition( - new MockQueryDefinitionBuilder(2) - .addVertex(0, 1) - .defineStage(0, true, numWorkers) // Ingestion only on one worker - .defineStage(1, true, numWorkers) - .getQueryDefinitionBuilder() - .build() - ); - return controllerQueryKernelTester; - } - - @Test - public void multipleWorkerFailedAfterInitialization() + public void testMultipleWorkersFailedAfterInitialization() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); @@ -480,7 +394,7 @@ public void multipleWorkerFailedAfterInitialization() } @Test - public void multipleWorkerFailedBeforeAnyWorkOrdersSent() + public void testMultipleWorkersFailedBeforeAnyWorkOrdersSent() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); controllerQueryKernelTester.init(); @@ -494,23 +408,26 @@ public void multipleWorkerFailedBeforeAnyWorkOrdersSent() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 2); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 2); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @Test - public void multipleWorkerFailedBeforeAllWorkOrdersSent() + public void testMultipleWorkersFailedBeforeAllWorkOrdersSent() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); @@ -519,50 +436,275 @@ public void multipleWorkerFailedBeforeAllWorkOrdersSent() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 2); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(1, RETRIABLE_FAULT); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); - controllerQueryKernelTester.sendWorkOrdersForWorker(0, 1); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + } + + @Test + public void testMultipleWorkersFailedBeforeAnyResultsStatsArrive() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 2); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0, 2); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 1); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorker(0, 2); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + @Test + public void testMultipleWorkersFailedBeforeAllResultsStatsArrive() + { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 2); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } - public void multipleWorkerFailedBeforeAnyResultsStatsArrive() {} - public void multipleWorkerFailedBeforeAllResultsStatsArrive() {} + @Test + public void testMultipleWorkersFailedBeforeAnyPartitionBoundariesAreSent() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); - public void multipleWorkerFailedBeforeAnyPartitionBoundariesAreSent() {} + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); - public void multipleWorkerFailedBeforeAllPartitionBoundariesAreSent() {} + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - public void multipleWorkerFailedBeforeAnyResultsRecieved() {} - public void multipleWorkerFailedBeforeAllResultsRecieved() {} + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } + + + @Test + public void testMultipleWorkersFailedBeforeAllPartitionBoundariesAreSent() + { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 2); + + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } + + @Test + public void testMultipleWorkersFailedBeforeAnyResultsRecieved() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + } + + @Test + public void testMultipleWorkersFailedBeforeAllResultsRecieved() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + @Nonnull + private ControllerQueryKernelTester getSimpleQueryDefinition(int numWorkers) + { + ControllerQueryKernelTester controllerQueryKernelTester = testControllerQueryKernel(numWorkers); + // 0 -> 1 + controllerQueryKernelTester.queryDefinition( + new MockQueryDefinitionBuilder(2) + .addVertex(0, 1) + .defineStage(0, true, numWorkers) + .defineStage(1, true, numWorkers) + .getQueryDefinitionBuilder() + .build() + ); + return controllerQueryKernelTester; + } } From 58da3cf8f7fcf2b12f379b169bc05a16ab790acb Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 14 Nov 2022 01:23:55 +0530 Subject: [PATCH 07/27] Adding some more java docs and fixing spot bugs, intellij inspections --- .../apache/druid/msq/exec/ControllerImpl.java | 46 +++++++++++++------ .../org/apache/druid/msq/exec/Limits.java | 2 +- .../druid/msq/indexing/MSQWorkerTask.java | 4 ++ .../msq/indexing/MSQWorkerTaskLauncher.java | 5 +- .../controller/ControllerQueryKernel.java | 2 +- 5 files changed, 41 insertions(+), 18 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 0e18037227b9..d75ebd889ac2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -287,7 +287,9 @@ public ControllerImpl( { this.task = task; this.context = context; - this.isDurableStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context()); + this.isDurableStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec() + .getQuery() + .context()); } @@ -583,6 +585,10 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) return queryDef; } + /** + * Adds the workorders for worker to {@link ControllerImpl#workOrdersToRetry} if the {@link ControllerQueryKernel} determines that there + * are work orders which needs reprocessing. + */ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault fault) { List retriableWorkOrders = kernel.getWorkInCaseWorkerElgibileForRetryElseThrow(worker, fault); @@ -599,6 +605,11 @@ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault return workOrders; } }); + } else { + log.info( + "Worker[%d] has no active workOrders that need relaunch therefore not relaunching", + worker + ); } } @@ -1031,14 +1042,14 @@ private Int2ObjectMap> makeSegmentGeneratorWorkerFa * @param queryKernel * @param contactFn * @param workers set of workers to contact - * @param succescCallBack on successfull api call, custom callback + * @param successCallBack on successfull api call, custom callback * @param retryOnFailure if set to true, adds this worker to retry queue. If false, cancel all the futures and propergate the exception to the caller. */ private void contactWorkersForStage( final ControllerQueryKernel queryKernel, - final TaskContactFn contactFn, + final TaskContactFn contactFn, final IntSet workers, - final OnSuccess succescCallBack, + final TaskContactSuccesss successCallBack, final boolean retryOnFailure ) { @@ -1063,7 +1074,7 @@ private void contactWorkersForStage( @Override public void onSuccess(@Nullable Void result) { - succescCallBack.onSuccess(taskId, workerNumber); + successCallBack.onSuccess(taskId, workerNumber); settableFuture.set(true); } @@ -1964,7 +1975,7 @@ private static Map copyOfStageRuntimesEndingAtCurrentTime( /** * Performs a particular {@link SegmentTransactionalInsertAction}, publishing segments. - * + *

* Throws {@link MSQException} with {@link InsertLockPreemptedFault} if the action fails due to lock preemption. */ static void performSegmentPublish( @@ -2101,13 +2112,17 @@ private Pair> run() throws IOExceptio return Pair.of(queryKernel, workerTaskLauncherFuture); } + private void retryFailedTasks() throws InterruptedException { + // if no work orders to rety skip if (workOrdersToRetry.size() == 0) { return; } Set workersNeedToBeFullyStarted = new HashSet<>(); + // transform work orders from map to Map> + // since we would want workOrders of processed per stage Map> stageWorkerOrders = new HashMap<>(); for (Map.Entry> workerStages : workOrdersToRetry.entrySet()) { @@ -2126,6 +2141,7 @@ private void retryFailedTasks() throws InterruptedException } } + // wait till the workers identified above are fully ready workerTaskLauncher.waitUntilWorkersReady(workersNeedToBeFullyStarted); for (Map.Entry> stageWorkOrders : stageWorkerOrders.entrySet()) { @@ -2139,6 +2155,8 @@ private void retryFailedTasks() throws InterruptedException new IntArraySet(stageWorkOrders.getValue().keySet()), (taskId, workerNumber) -> { queryKernel.workOrdersSentForWorker(stageWorkOrders.getKey(), workerNumber); + + // remove sucessfully contacted workOrders from workOrdersToRetry workOrdersToRetry.compute(workerNumber, (task, workOrderSet) -> { if (workOrderSet == null || workOrderSet.size() == 0 || !workOrderSet.remove(stageWorkOrders.getValue() .get( @@ -2280,10 +2298,12 @@ private void sendPartitionBoundaries() && queryKernel.doesStageHaveResultPartitions(stageId)) { IntSet workersToSendPartitionBoundaries = queryKernel.getWorkersToSendPartitionBoundaries(stageId); if (workersToSendPartitionBoundaries.isEmpty()) { + log.debug("No workers for stage[%s] ready to recieve partition boundaries", stageId); return; } + final ClusterByPartitions partitions = queryKernel.getResultPartitionBoundariesForStage(stageId); + if (log.isDebugEnabled()) { - final ClusterByPartitions partitions = queryKernel.getResultPartitionBoundariesForStage(stageId); log.debug( "Query [%s] sending out partition boundaries for stage %d: %s for workers %s", stageId.getQueryId(), @@ -2306,7 +2326,7 @@ private void sendPartitionBoundaries() queryKernel, queryDef, stageId.getStageNumber(), - queryKernel.getResultPartitionBoundariesForStage(stageId), + partitions, workersToSendPartitionBoundaries ); } @@ -2396,17 +2416,17 @@ private void throwKernelExceptionIfNotUnknown() /** * Interface used by {@link #contactWorkersForStage}. */ - private interface TaskContactFn + private interface TaskContactFn { ListenableFuture contactTask(WorkerClient client, String taskId, int workerNumber); } - - private interface OnSuccess + /** + * Interface used when {@link TaskContactFn#contactTask(WorkerClient, String, int)} return future is successfull. + */ + private interface TaskContactSuccesss { void onSuccess(String taskId, int workerNumber); } - - } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index cacd04481250..83f3f7a0d65e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -67,5 +67,5 @@ public class Limits /** * Maximum worker retries. Initial run is not a retry. The worker will be spawned 1 + retryNumber of times before erroring out. */ - public static int WORKER_RETRY_LIMIT = 2; + public static final int WORKER_RETRY_LIMIT = 2; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java index 60623b19910c..22477d4860af 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java @@ -96,6 +96,10 @@ public int getRetryCount() return retryCount; } + /** + * Creates a new retry {@link MSQWorkerTask} with the same context as the current task, but with the retry count + * incremented by 1 + */ public MSQWorkerTask getRetryTask() { return new MSQWorkerTask(controllerTaskId, getDataSource(), workerNumber, getContext(), retryCount + 1); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 17cde35c8083..fd737fae12ac 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -124,7 +124,7 @@ private enum State // retry worker set private final Set retryWorkerSet = ConcurrentHashMap.newKeySet(); - private final Map> workerToTaskIds = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> workerToTaskIds = new ConcurrentHashMap<>(); private final RetryTask retryTask; public MSQWorkerTaskLauncher( @@ -528,8 +528,6 @@ private void relaunchTasks() } MSQWorkerTask toRetry = tracker.msqWorkerTask; - - MSQWorkerTask retryTask = toRetry.getRetryTask(); // check retry limits @@ -541,6 +539,7 @@ private void relaunchTasks() taskTrackers.put(retryTask.getId(), new TaskTracker(retryTask.getWorkerNumber(), retryTask)); context.workerManager().run(retryTask.getId(), retryTask); taskHistory.add(retryTask.getId()); + synchronized (taskIds) { // replace taskId with the retry taskID for the same worker number taskIds.set(toRetry.getWorkerNumber(), retryTask.getId()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 4c58f4abc4b0..8e58e13ce7a5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -645,7 +645,7 @@ public List getWorkInCaseWorkerElgibileForRetryElseThrow(int workerNu } /** - * Gets all the stages currently being tracked and filtres out all effectively finished stages. + * Gets all the stages currently being tracked and filters out all effectively finished stages. *
* From the remaining stages, checks if (stage,worker) needs to be retried. *
From 7b90edcc62f16bbe335645451bcdd000ff6691d7 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 14 Nov 2022 07:51:21 +0530 Subject: [PATCH 08/27] Fixing intellij inspections and added tests --- .../apache/druid/msq/exec/ControllerImpl.java | 5 - .../msq/indexing/MSQWorkerTaskLauncher.java | 4 +- ...lt.java => TooManyWorkerRetriedFault.java} | 8 +- .../druid/msq/indexing/MSQWorkerTaskTest.java | 107 ++++++++++++++++++ .../msq/indexing/error/MSQFaultSerdeTest.java | 2 +- ...ts.java => ControllerQueryKernelTest.java} | 2 +- ...nShufflingWorkersWithRetryKernelTest.java} | 2 +- ... ShufflingWorkersWithRetryKernelTest.java} | 2 +- .../apache/druid/msq/test/MSQTestBase.java | 10 +- 9 files changed, 122 insertions(+), 20 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/{TooManyWorkerRetriesFault.java => TooManyWorkerRetriedFault.java} (92%) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java rename extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/{ControllerQueryKernelTests.java => ControllerQueryKernelTest.java} (99%) rename extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/{NonShufflingWorkersWithRetryKernelTests.java => NonShufflingWorkersWithRetryKernelTest.java} (99%) rename extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/{ShufflingWorkersWithRetryKernelTests.java => ShufflingWorkersWithRetryKernelTest.java} (99%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index d75ebd889ac2..4a6ff31bb0d1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -2055,11 +2055,6 @@ private class RunQueryUntilDone private final Closer closer; private final ControllerQueryKernel queryKernel; - /** - * Set of stages that have got their partition boundaries sent out. - */ - private final Set stageResultPartitionBoundariesSent = new HashSet<>(); - /** * Return value of {@link MSQWorkerTaskLauncher#start()}. Set by {@link #startTaskLauncher()}. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index fd737fae12ac..ff8056536f23 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -40,7 +40,7 @@ import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; -import org.apache.druid.msq.indexing.error.TooManyWorkerRetriesFault; +import org.apache.druid.msq.indexing.error.TooManyWorkerRetriedFault; import org.apache.druid.msq.indexing.error.TotalRetryLimitExceededFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; @@ -556,7 +556,7 @@ private void relaunchTasks() private void checkRetryLimits(TaskTracker tracker, MSQWorkerTask toRetry) { if (toRetry.getRetryCount() > Limits.WORKER_RETRY_LIMIT) { - throw new MSQException(new TooManyWorkerRetriesFault( + throw new MSQException(new TooManyWorkerRetriedFault( Limits.WORKER_RETRY_LIMIT, toRetry.getId(), toRetry.getWorkerNumber(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriesFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriedFault.java similarity index 92% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriesFault.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriedFault.java index e1a301ed0efc..d834fc1be6cf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriesFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriedFault.java @@ -25,8 +25,8 @@ import java.util.Objects; -@JsonTypeName(TotalRetryLimitExceededFault.CODE) -public class TooManyWorkerRetriesFault extends BaseMSQFault +@JsonTypeName(TooManyWorkerRetriedFault.CODE) +public class TooManyWorkerRetriedFault extends BaseMSQFault { static final String CODE = "TooManyWorkerRetries"; @@ -41,7 +41,7 @@ public class TooManyWorkerRetriesFault extends BaseMSQFault private final String rootErrorMessage; @JsonCreator - public TooManyWorkerRetriesFault( + public TooManyWorkerRetriedFault( @JsonProperty("maxRetryCount") int maxRetryCount, @JsonProperty("taskId") String taskId, @JsonProperty("workerNumber") int workerNumber, @@ -98,7 +98,7 @@ public boolean equals(Object o) if (!super.equals(o)) { return false; } - TooManyWorkerRetriesFault that = (TooManyWorkerRetriesFault) o; + TooManyWorkerRetriedFault that = (TooManyWorkerRetriedFault) o; return maxRetryCount == that.maxRetryCount && workerNumber == that.workerNumber && Objects.equals( taskId, that.taskId diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java new file mode 100644 index 000000000000..81cbcdc843c4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing; + +import com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashSet; +import java.util.Set; + +public class MSQWorkerTaskTest +{ + + private final String controllerTaskId = "ctr"; + private final String dataSource = "ds"; + private final int workerNumber = 1; + private final ImmutableMap context = ImmutableMap.of("key", "val"); + private final int retryCount = 0; + + private final MSQWorkerTask msqWorkerTask = new MSQWorkerTask( + controllerTaskId, + dataSource, + workerNumber, + context, + retryCount + ); + + + @Test + public void testEquals() + { + Assert.assertNotEquals(msqWorkerTask, 0); + Assert.assertEquals(msqWorkerTask, msqWorkerTask); + Assert.assertEquals( + msqWorkerTask, + new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount) + ); + Assert.assertEquals( + msqWorkerTask.getRetryTask(), + new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount + 1) + ); + Assert.assertNotEquals(msqWorkerTask, msqWorkerTask.getRetryTask()); + } + + @Test + public void testHashCode() + { + Set msqWorkerTaskSet = new HashSet<>(); + + msqWorkerTaskSet.add(msqWorkerTask); + msqWorkerTaskSet.add(new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount)); + Assert.assertTrue(msqWorkerTaskSet.size() == 1); + + msqWorkerTaskSet.add(msqWorkerTask.getRetryTask()); + Assert.assertTrue(msqWorkerTaskSet.size() == 2); + + msqWorkerTaskSet.add(new MSQWorkerTask(controllerTaskId + 1, dataSource, workerNumber, context, retryCount)); + Assert.assertTrue(msqWorkerTaskSet.size() == 3); + + msqWorkerTaskSet.add(new MSQWorkerTask(controllerTaskId, dataSource + 1, workerNumber, context, retryCount)); + Assert.assertTrue(msqWorkerTaskSet.size() == 4); + + msqWorkerTaskSet.add(new MSQWorkerTask(controllerTaskId, dataSource, workerNumber + 1, context, retryCount)); + Assert.assertTrue(msqWorkerTaskSet.size() == 5); + + msqWorkerTaskSet.add(new MSQWorkerTask( + controllerTaskId, + dataSource, + workerNumber, + ImmutableMap.of("key1", "v1"), + retryCount + )); + Assert.assertTrue(msqWorkerTaskSet.size() == 6); + + msqWorkerTaskSet.add(new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount + 1)); + Assert.assertTrue(msqWorkerTaskSet.size() == 6); + } + + @Test + public void testGetter() + { + Assert.assertEquals(controllerTaskId, msqWorkerTask.getControllerTaskId()); + Assert.assertEquals(dataSource, msqWorkerTask.getDataSource()); + Assert.assertEquals(workerNumber, msqWorkerTask.getWorkerNumber()); + Assert.assertEquals(retryCount, msqWorkerTask.getRetryCount()); + } + + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index acf3045143ca..88d66ac0d9bd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -74,7 +74,7 @@ public void testFaultSerde() throws IOException assertFaultSerde(new TooManyPartitionsFault(10)); assertFaultSerde(new TooManyWarningsFault(10, "the error")); assertFaultSerde(new TooManyWorkersFault(10, 5)); - assertFaultSerde(new TooManyWorkerRetriesFault(2, "taskId", 1, "rootError")); + assertFaultSerde(new TooManyWorkerRetriedFault(2, "taskId", 1, "rootError")); assertFaultSerde(new TotalRetryLimitExceededFault(2, 2, "taskId", "rootError")); assertFaultSerde(UnknownFault.forMessage(null)); assertFaultSerde(UnknownFault.forMessage("the message")); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTest.java similarity index 99% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTest.java index 835c7ea0d4c6..2d5a00a052cd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTest.java @@ -28,7 +28,7 @@ import java.util.Set; -public class ControllerQueryKernelTests extends BaseControllerQueryKernelTest +public class ControllerQueryKernelTest extends BaseControllerQueryKernelTest { @Test diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTest.java similarity index 99% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTest.java index 7137aee7f107..d2f2180e7a4f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTest.java @@ -25,7 +25,7 @@ import javax.annotation.Nonnull; -public class NonShufflingWorkersWithRetryKernelTests extends BaseControllerQueryKernelTest +public class NonShufflingWorkersWithRetryKernelTest extends BaseControllerQueryKernelTest { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java similarity index 99% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java index 143d4467bf37..008fd81c1727 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java @@ -24,7 +24,7 @@ import javax.annotation.Nonnull; -public class ShufflingWorkersWithRetryKernelTests extends BaseControllerQueryKernelTest +public class ShufflingWorkersWithRetryKernelTest extends BaseControllerQueryKernelTest { @Test diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 2613a45a8961..1c439b8492fd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -78,7 +78,7 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.MSQFaultUtils; -import org.apache.druid.msq.indexing.error.TooManyWorkerRetriesFault; +import org.apache.druid.msq.indexing.error.TooManyWorkerRetriedFault; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; @@ -902,8 +902,8 @@ public void verifyResults() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { - String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriesFault - ? ((TooManyWorkerRetriesFault) msqErrorReport.getFault()).getRootErrorMessage() + String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriedFault + ? ((TooManyWorkerRetriedFault) msqErrorReport.getFault()).getRootErrorMessage() : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), @@ -1073,8 +1073,8 @@ public Pair>> runQueryWithResult() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { - String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriesFault - ? ((TooManyWorkerRetriesFault) msqErrorReport.getFault()).getRootErrorMessage() + String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriedFault + ? ((TooManyWorkerRetriedFault) msqErrorReport.getFault()).getRootErrorMessage() : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), From deafdf8be55bd95fa23b049fd0f6f2f2407c3143 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 15 Nov 2022 23:29:52 +0530 Subject: [PATCH 09/27] Documenting error codes --- docs/multi-stage-query/known-issues.md | 2 +- docs/multi-stage-query/reference.md | 5 ++ .../org/apache/druid/msq/exec/Limits.java | 10 ++-- .../druid/msq/guice/MSQIndexingModule.java | 9 ++- .../msq/indexing/MSQWorkerTaskLauncher.java | 58 +++++++++---------- ...a => TotalRelaunchLimitExceededFault.java} | 45 +++++++------- ...java => WorkerRelaunchedTooManyTimes.java} | 33 ++++++----- .../msq/indexing/error/MSQFaultSerdeTest.java | 4 +- .../apache/druid/msq/test/MSQTestBase.java | 10 ++-- 9 files changed, 93 insertions(+), 83 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/{TotalRetryLimitExceededFault.java => TotalRelaunchLimitExceededFault.java} (60%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/{TooManyWorkerRetriedFault.java => WorkerRelaunchedTooManyTimes.java} (67%) diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md index c76ab57aa7ac..6d5fab7bf88a 100644 --- a/docs/multi-stage-query/known-issues.md +++ b/docs/multi-stage-query/known-issues.md @@ -29,7 +29,7 @@ sidebar_label: Known issues ## Multi-stage query task runtime -- Fault tolerance is not implemented. If any task fails, the entire query fails. +- Fault tolerance is partially implemented. Workers get relaunched when they are killed unexpectedly. The controller does not get relaunched if its killed unexpectedly. - Worker task stage outputs are stored in the working directory given by `druid.indexer.task.baseDir`. Stages that generate a large amount of output data may exhaust all available disk space. In this case, the query fails with diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index a85a8e785b01..3c471514b2ec 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -234,6 +234,9 @@ The following table lists query limits: | Number of output columns for any one stage. | 2,000 | `TooManyColumns` | | Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | `TooManyWorkers` | | Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](concepts.md#memory-usage). | `BroadcastTablesTooLarge` | +| Maximum relaunches per worker. Initial run is not a relaunch. The worker will be spawned 1 + workerRelaunchLimit times before erroring out. | 2 | `WorkerRelaunchedTooManyTimes` | +| Maximum relaunches across all workers. | 30 | `TotalRelaunchLimitExceededFault` | + @@ -265,7 +268,9 @@ The following table describes error codes you may encounter in the `multiStageQu | `TooManyColumns` | Exceeded the number of columns for a stage. See the [Limits](#limits) table for the specific limit. | `numColumns`: The number of columns requested.

`maxColumns`: The limit on columns which was exceeded. | | `TooManyWarnings` | Exceeded the allowed number of warnings of a particular type. | `rootErrorCode`: The error code corresponding to the exception that exceeded the required limit.

`maxWarnings`: Maximum number of warnings that are allowed for the corresponding `rootErrorCode`. | | `TooManyWorkers` | Exceeded the supported number of workers running simultaneously. See the [Limits](#limits) table for the specific limit. | `workers`: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously.

`maxWorkers`: The hard or soft limit on workers that was exceeded. | +| `TotalRelaunchLimitExceededFault` | Total relaunch count across all workers exceeded max relaunch limit. See the [Limits](#limits) table for the specific limit. | `maxRelaunchCount`: Max number of relaunches across all the workers defined in the [Limits](#limits) section.

`currentRelaunchCount`: current relaunch counter for the job.

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| | `NotEnoughMemory` | Insufficient memory to launch a stage. | `serverMemory`: The amount of memory available to a single process.

`serverWorkers`: The number of workers running in a single process.

`serverThreads`: The number of threads in a single process. | | `WorkerFailed` | A worker task failed unexpectedly. | `errorMsg`

`workerTaskId`: The ID of the worker task. | +| `WorkerRelaunchedTooManyTimes` | Worker exceeded maximum relaunch count as definied in the [Limits](#limits) section. |`maxPerWorkerRelaunchCount`: Max number of relaunches allower per worker as defined in the [Limits](#limits) section.

`workerNumber`: the worker number for which the task failed

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| | `WorkerRpcFailed` | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task | | `UnknownError` | All other errors. | `message` | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index 83f3f7a0d65e..ed8bb6c4ec43 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -23,7 +23,7 @@ public class Limits { /** * Maximum number of columns that can appear in a frame signature. - * + *

* Somewhat less than {@link WorkerMemoryParameters#STANDARD_FRAME_SIZE} divided by typical minimum column size: * {@link org.apache.druid.frame.allocation.AppendableMemory#DEFAULT_INITIAL_ALLOCATION_SIZE}. */ @@ -60,12 +60,12 @@ public class Limits public static final int MAX_KERNEL_MANIPULATION_QUEUE_SIZE = 100_000; /** - * Maximum retries across all workers. + * Maximum relaunches across all workers. */ - public static final int TOTAL_RETRY_LIMIT = 30; + public static final int TOTAL_RELAUNCH_LIMIT = 30; /** - * Maximum worker retries. Initial run is not a retry. The worker will be spawned 1 + retryNumber of times before erroring out. + * Maximum relaunches per worker. Initial run is not a relaunch. The worker will be spawned 1 + workerRelaunchLimit times before erroring out. */ - public static final int WORKER_RETRY_LIMIT = 2; + public static final int PER_WORKER_RELAUNCH_LIMIT = 2; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index c94801ed3968..35af54d8be26 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -63,11 +63,11 @@ import org.apache.druid.msq.indexing.error.TooManyInputFilesFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; import org.apache.druid.msq.indexing.error.TooManyWarningsFault; -import org.apache.druid.msq.indexing.error.TooManyWorkerRetriedFault; import org.apache.druid.msq.indexing.error.TooManyWorkersFault; -import org.apache.druid.msq.indexing.error.TotalRetryLimitExceededFault; +import org.apache.druid.msq.indexing.error.TotalRelaunchLimitExceededFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; +import org.apache.druid.msq.indexing.error.WorkerRelaunchedTooManyTimes; import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.input.NilInputSlice; @@ -88,7 +88,6 @@ import org.apache.druid.query.DruidProcessingConfig; import javax.annotation.Nullable; - import java.util.Collections; import java.util.List; import java.util.Set; @@ -125,10 +124,10 @@ public class MSQIndexingModule implements DruidModule TooManyPartitionsFault.class, TooManyWarningsFault.class, TooManyWorkersFault.class, - TooManyWorkerRetriedFault.class, - TotalRetryLimitExceededFault.class, + TotalRelaunchLimitExceededFault.class, UnknownFault.class, WorkerFailedFault.class, + WorkerRelaunchedTooManyTimes.class, WorkerRpcFailedFault.class ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index ff8056536f23..6eddd3da8b3f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -40,10 +40,10 @@ import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; -import org.apache.druid.msq.indexing.error.TooManyWorkerRetriedFault; -import org.apache.druid.msq.indexing.error.TotalRetryLimitExceededFault; +import org.apache.druid.msq.indexing.error.TotalRelaunchLimitExceededFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; +import org.apache.druid.msq.indexing.error.WorkerRelaunchedTooManyTimes; import org.apache.druid.msq.util.MultiStageQueryContext; import javax.annotation.Nullable; @@ -73,7 +73,7 @@ public class MSQWorkerTaskLauncher private static final long LOW_FREQUENCY_CHECK_MILLIS = 2000; private static final long SWITCH_TO_LOW_FREQUENCY_CHECK_AFTER_MILLIS = 10000; private static final long SHUTDOWN_TIMEOUT_MS = Duration.ofMinutes(1).toMillis(); - private int currentRetryCount = 0; + private int currentRelaunchCount = 0; // States for "state" variable. private enum State @@ -121,8 +121,8 @@ private enum State // tasks to clean up due to retries private final Set tasksToCleanup = ConcurrentHashMap.newKeySet(); - // retry worker set - private final Set retryWorkerSet = ConcurrentHashMap.newKeySet(); + // workers to relaunch + private final Set workersToRelaunch = ConcurrentHashMap.newKeySet(); private final ConcurrentHashMap> workerToTaskIds = new ConcurrentHashMap<>(); private final RetryTask retryTask; @@ -247,7 +247,7 @@ public void launchTasksIfNeeded(final int taskCount) throws InterruptedException */ public void submitForRelaunch(int workerNumber) { - retryWorkerSet.add(workerNumber); + workersToRelaunch.add(workerNumber); } /** @@ -507,7 +507,7 @@ private void removeWorkerFromFullyStartedWorkers(TaskTracker tracker) private void relaunchTasks() { - Iterator iterator = retryWorkerSet.iterator(); + Iterator iterator = workersToRelaunch.iterator(); while (iterator.hasNext()) { int worker = iterator.next(); @@ -527,23 +527,23 @@ private void relaunchTasks() return taskHistory; } - MSQWorkerTask toRetry = tracker.msqWorkerTask; - MSQWorkerTask retryTask = toRetry.getRetryTask(); + MSQWorkerTask toRelaunch = tracker.msqWorkerTask; + MSQWorkerTask relaunchedTask = toRelaunch.getRetryTask(); - // check retry limits - checkRetryLimits(tracker, toRetry); + // check relaunch limits + checkRelaunchLimits(tracker, toRelaunch); // clean up trackers and tasks tasksToCleanup.add(latestTaskId); taskTrackers.remove(latestTaskId); - currentRetryCount += 1; - taskTrackers.put(retryTask.getId(), new TaskTracker(retryTask.getWorkerNumber(), retryTask)); - context.workerManager().run(retryTask.getId(), retryTask); - taskHistory.add(retryTask.getId()); + currentRelaunchCount += 1; + taskTrackers.put(relaunchedTask.getId(), new TaskTracker(relaunchedTask.getWorkerNumber(), relaunchedTask)); + context.workerManager().run(relaunchedTask.getId(), relaunchedTask); + taskHistory.add(relaunchedTask.getId()); synchronized (taskIds) { // replace taskId with the retry taskID for the same worker number - taskIds.set(toRetry.getWorkerNumber(), retryTask.getId()); - fullyStartedTasks.remove(retryTask.getWorkerNumber()); + taskIds.set(toRelaunch.getWorkerNumber(), relaunchedTask.getId()); + fullyStartedTasks.remove(relaunchedTask.getWorkerNumber()); taskIds.notifyAll(); } return taskHistory; @@ -553,21 +553,21 @@ private void relaunchTasks() } } - private void checkRetryLimits(TaskTracker tracker, MSQWorkerTask toRetry) + private void checkRelaunchLimits(TaskTracker tracker, MSQWorkerTask relaunchTask) { - if (toRetry.getRetryCount() > Limits.WORKER_RETRY_LIMIT) { - throw new MSQException(new TooManyWorkerRetriedFault( - Limits.WORKER_RETRY_LIMIT, - toRetry.getId(), - toRetry.getWorkerNumber(), + if (relaunchTask.getRetryCount() > Limits.PER_WORKER_RELAUNCH_LIMIT) { + throw new MSQException(new WorkerRelaunchedTooManyTimes( + Limits.PER_WORKER_RELAUNCH_LIMIT, + relaunchTask.getId(), + relaunchTask.getWorkerNumber(), tracker.status.getErrorMsg() )); } - if (currentRetryCount > Limits.TOTAL_RETRY_LIMIT) { - throw new MSQException(new TotalRetryLimitExceededFault( - Limits.TOTAL_RETRY_LIMIT, - currentRetryCount, - toRetry.getId(), + if (currentRelaunchCount > Limits.TOTAL_RELAUNCH_LIMIT) { + throw new MSQException(new TotalRelaunchLimitExceededFault( + Limits.TOTAL_RELAUNCH_LIMIT, + currentRelaunchCount, + relaunchTask.getId(), tracker.status.getErrorMsg() )); } @@ -590,7 +590,7 @@ private void shutDownTasks() } /** - * Cleans the task indentified in {@link MSQWorkerTaskLauncher#relaunchTasks()} for realunch. Asks the overlord to cancel the task. + * Cleans the task indentified in {@link MSQWorkerTaskLauncher#relaunchTasks()} for relaunch. Asks the overlord to cancel the task. */ private void cleanFailedTasksWhichAreRelaunched() { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRetryLimitExceededFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRelaunchLimitExceededFault.java similarity index 60% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRetryLimitExceededFault.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRelaunchLimitExceededFault.java index 003f0157b1dd..b26548c1db4c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRetryLimitExceededFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRelaunchLimitExceededFault.java @@ -25,48 +25,48 @@ import java.util.Objects; -@JsonTypeName(TotalRetryLimitExceededFault.CODE) -public class TotalRetryLimitExceededFault extends BaseMSQFault +@JsonTypeName(TotalRelaunchLimitExceededFault.CODE) +public class TotalRelaunchLimitExceededFault extends BaseMSQFault { - static final String CODE = "TotalRetryLimitExceededFault"; + static final String CODE = "TotalRelaunchLimitExceededFault"; - private final int maxRetryCount; + private final int maxRelaunchCount; private final String taskId; - private final int retryCount; + private final int currentRelaunchCount; private final String rootErrorMessage; @JsonCreator - public TotalRetryLimitExceededFault( - @JsonProperty("maxRetryCount") int maxRetryCount, - @JsonProperty("retryCount") int retryCount, + public TotalRelaunchLimitExceededFault( + @JsonProperty("maxRelaunchCount") int maxRelaunchCount, + @JsonProperty("currentRelaunchCount") int currentRelaunchCount, @JsonProperty("taskId") String taskId, @JsonProperty("rootErrorMessage") String rootErrorMessage ) { super( CODE, - "Retry count %d exceeded total retry limit %d .Latest task[%s] failure reason: %s", - retryCount, - maxRetryCount, + "Total relaunch count across all workers %d exceeded max relaunch limit %d . Latest task[%s] failure reason: %s", + currentRelaunchCount, + maxRelaunchCount, taskId, rootErrorMessage ); - this.maxRetryCount = maxRetryCount; - this.retryCount = retryCount; + this.maxRelaunchCount = maxRelaunchCount; + this.currentRelaunchCount = currentRelaunchCount; this.taskId = taskId; this.rootErrorMessage = rootErrorMessage; } @JsonProperty - public int getMaxRetryCount() + public int getMaxRelaunchCount() { - return maxRetryCount; + return maxRelaunchCount; } @JsonProperty @@ -76,9 +76,9 @@ public String getTaskId() } @JsonProperty - public int getRetryCount() + public int getCurrentRelaunchCount() { - return retryCount; + return currentRelaunchCount; } @JsonProperty @@ -99,16 +99,19 @@ public boolean equals(Object o) if (!super.equals(o)) { return false; } - TotalRetryLimitExceededFault that = (TotalRetryLimitExceededFault) o; - return maxRetryCount == that.maxRetryCount && retryCount == that.retryCount && Objects.equals( + TotalRelaunchLimitExceededFault that = (TotalRelaunchLimitExceededFault) o; + return maxRelaunchCount == that.maxRelaunchCount + && currentRelaunchCount == that.currentRelaunchCount + && Objects.equals( taskId, that.taskId - ) && Objects.equals(rootErrorMessage, that.rootErrorMessage); + ) + && Objects.equals(rootErrorMessage, that.rootErrorMessage); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), maxRetryCount, taskId, retryCount, rootErrorMessage); + return Objects.hash(super.hashCode(), maxRelaunchCount, taskId, currentRelaunchCount, rootErrorMessage); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriedFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRelaunchedTooManyTimes.java similarity index 67% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriedFault.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRelaunchedTooManyTimes.java index d834fc1be6cf..83d6412f97af 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyWorkerRetriedFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRelaunchedTooManyTimes.java @@ -25,13 +25,13 @@ import java.util.Objects; -@JsonTypeName(TooManyWorkerRetriedFault.CODE) -public class TooManyWorkerRetriedFault extends BaseMSQFault +@JsonTypeName(WorkerRelaunchedTooManyTimes.CODE) +public class WorkerRelaunchedTooManyTimes extends BaseMSQFault { - static final String CODE = "TooManyWorkerRetries"; + static final String CODE = "WorkerRelaunchedTooManyTimes"; - private final int maxRetryCount; + private final int maxPerWorkerRelaunchCount; private final String taskId; @@ -41,8 +41,8 @@ public class TooManyWorkerRetriedFault extends BaseMSQFault private final String rootErrorMessage; @JsonCreator - public TooManyWorkerRetriedFault( - @JsonProperty("maxRetryCount") int maxRetryCount, + public WorkerRelaunchedTooManyTimes( + @JsonProperty("maxPerWorkerRelaunchCount") int maxPerWorkerRelaunchCount, @JsonProperty("taskId") String taskId, @JsonProperty("workerNumber") int workerNumber, @JsonProperty("rootErrorMessage") String rootErrorMessage @@ -50,22 +50,22 @@ public TooManyWorkerRetriedFault( { super( CODE, - "Worker[%d] exceeded max retry count of %d for task[%s]. Latest failure reason: %s.", + "Worker[%d] exceeded max relaunch count of %d for task[%s]. Latest failure reason: %s.", workerNumber, - maxRetryCount, + maxPerWorkerRelaunchCount, taskId, rootErrorMessage ); - this.maxRetryCount = maxRetryCount; + this.maxPerWorkerRelaunchCount = maxPerWorkerRelaunchCount; this.taskId = taskId; this.workerNumber = workerNumber; this.rootErrorMessage = rootErrorMessage; } @JsonProperty - public int getMaxRetryCount() + public int getMaxPerWorkerRelaunchCount() { - return maxRetryCount; + return maxPerWorkerRelaunchCount; } @JsonProperty @@ -98,16 +98,19 @@ public boolean equals(Object o) if (!super.equals(o)) { return false; } - TooManyWorkerRetriedFault that = (TooManyWorkerRetriedFault) o; - return maxRetryCount == that.maxRetryCount && workerNumber == that.workerNumber && Objects.equals( + WorkerRelaunchedTooManyTimes that = (WorkerRelaunchedTooManyTimes) o; + return maxPerWorkerRelaunchCount == that.maxPerWorkerRelaunchCount + && workerNumber == that.workerNumber + && Objects.equals( taskId, that.taskId - ) && Objects.equals(rootErrorMessage, that.rootErrorMessage); + ) + && Objects.equals(rootErrorMessage, that.rootErrorMessage); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), maxRetryCount, taskId, workerNumber, rootErrorMessage); + return Objects.hash(super.hashCode(), maxPerWorkerRelaunchCount, taskId, workerNumber, rootErrorMessage); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index 88d66ac0d9bd..147dd2d07991 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -74,8 +74,8 @@ public void testFaultSerde() throws IOException assertFaultSerde(new TooManyPartitionsFault(10)); assertFaultSerde(new TooManyWarningsFault(10, "the error")); assertFaultSerde(new TooManyWorkersFault(10, 5)); - assertFaultSerde(new TooManyWorkerRetriedFault(2, "taskId", 1, "rootError")); - assertFaultSerde(new TotalRetryLimitExceededFault(2, 2, "taskId", "rootError")); + assertFaultSerde(new WorkerRelaunchedTooManyTimes(2, "taskId", 1, "rootError")); + assertFaultSerde(new TotalRelaunchLimitExceededFault(2, 2, "taskId", "rootError")); assertFaultSerde(UnknownFault.forMessage(null)); assertFaultSerde(UnknownFault.forMessage("the message")); assertFaultSerde(new WorkerFailedFault("the worker task", "the error msg")); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 1c439b8492fd..6903cb104acf 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -78,7 +78,7 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.MSQFaultUtils; -import org.apache.druid.msq.indexing.error.TooManyWorkerRetriedFault; +import org.apache.druid.msq.indexing.error.WorkerRelaunchedTooManyTimes; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; @@ -902,8 +902,8 @@ public void verifyResults() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { - String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriedFault - ? ((TooManyWorkerRetriedFault) msqErrorReport.getFault()).getRootErrorMessage() + String errorMessage = msqErrorReport.getFault() instanceof WorkerRelaunchedTooManyTimes + ? ((WorkerRelaunchedTooManyTimes) msqErrorReport.getFault()).getRootErrorMessage() : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), @@ -1073,8 +1073,8 @@ public Pair>> runQueryWithResult() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { - String errorMessage = msqErrorReport.getFault() instanceof TooManyWorkerRetriedFault - ? ((TooManyWorkerRetriedFault) msqErrorReport.getFault()).getRootErrorMessage() + String errorMessage = msqErrorReport.getFault() instanceof WorkerRelaunchedTooManyTimes + ? ((WorkerRelaunchedTooManyTimes) msqErrorReport.getFault()).getRootErrorMessage() : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), From edfd5b63d05e1fe5c4b966b5d3b1488f26a78ef2 Mon Sep 17 00:00:00 2001 From: abhagraw <99210446+abhagraw@users.noreply.github.com> Date: Mon, 21 Nov 2022 09:12:02 +0530 Subject: [PATCH 10/27] Migrate current integration batch tests to equivalent MSQ tests (#13374) * Migrate current integration batch tests to equivalent MSQ tests using new IT framework * Fix build issues * Trigger Build * Adding more tests and addressing comments * fixBuildIssues * fix dependency issues * Parameterized the test and addressed comments * Addressing comments * fixing checkstyle errors * Adressing comments --- .travis.yml | 16 +- distribution/pom.xml | 2 + integration-tests-ex/cases/pom.xml | 19 ++- .../msq/AbstractITSQLBasedIngestion.java | 121 ++++++++++++++ .../testsEx/msq/ITSQLBasedBatchIngestion.java | 71 +++++++++ .../batch-index/json_path_index_queries.json | 49 ++++++ .../batch-index/msq_inline.sql | 17 ++ .../batch-index/sparse_column_msq.json | 93 +++++++++++ .../batch-index/sparse_column_msq.sql | 21 +++ .../wikipedia_http_inputsource_msq.sql | 29 ++++ .../wikipedia_http_inputsource_queries.json | 47 ++++++ .../batch-index/wikipedia_index_msq.sql | 32 ++++ .../batch-index/wikipedia_index_queries.json | 150 ++++++++++++++++++ ...ikipedia_index_queries_with_transform.json | 62 ++++++++ .../wikipedia_index_task_with_transform.sql | 32 ++++ .../wikipedia_merge_index_task.sql | 33 ++++ .../testing/utils/MsqTestQueryHelper.java | 31 +++- 17 files changed, 808 insertions(+), 17 deletions(-) create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/AbstractITSQLBasedIngestion.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITSQLBasedBatchIngestion.java create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/json_path_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/msq_inline.sql create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.json create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.sql create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_msq.sql create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_msq.sql create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries_with_transform.json create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_task_with_transform.sql create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_merge_index_task.sql diff --git a/.travis.yml b/.travis.yml index 2fd32ffd7b9c..2d785486ac97 100644 --- a/.travis.yml +++ b/.travis.yml @@ -442,10 +442,6 @@ jobs: docker exec -it druid-$v sh -c 'dmesg | tail -3' ; done - - <<: *integration_batch_index - name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer" - env: TESTNG_GROUPS='-Dgroups=batch-index' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' - - &integration_input_format name: "(Compile=openjdk8, Run=openjdk8) input format integration test" stage: Tests - phase 2 @@ -689,11 +685,13 @@ jobs: env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' script: ./it.sh travis Catalog - # Disabling BatchIndex test as it is failing with due to timeout, fixing it will be taken in a separate PR. - #- <<: *integration_tests_ex - # name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer (new)" - # env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' - # script: ./it.sh travis BatchIndex + - &integration_tests_ex + name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer (new)" + stage: Tests - phase 2 + jdk: openjdk8 + services: *integration_test_services + env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' + script: ./it.sh travis BatchIndex # END - Integration tests for Compile with Java 8 and Run with Java 8 diff --git a/distribution/pom.xml b/distribution/pom.xml index b73e4215a64a..e2b7773b09d4 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -225,6 +225,8 @@ -c org.apache.druid.extensions:druid-multi-stage-query -c + org.apache.druid.extensions:druid-catalog + -c org.apache.druid.extensions:druid-protobuf-extensions -c org.apache.druid.extensions:mysql-metadata-storage diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 5456d4b81b01..cf781f6f8885 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -155,7 +155,7 @@ org.jdbi jdbi - + org.apache.druid.extensions mysql-metadata-storage ${project.parent.version} @@ -218,10 +218,15 @@ JUnitParams test - - javax.ws.rs - jsr311-api - + + javax.ws.rs + jsr311-api + + + org.apache.curator + curator-client + 5.4.0 + @@ -345,8 +350,8 @@ integration-test - - true + + False org.apache.druid.testsEx.categories.${it.category} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/AbstractITSQLBasedIngestion.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/AbstractITSQLBasedIngestion.java new file mode 100644 index 000000000000..4bb1cdc4783b --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/AbstractITSQLBasedIngestion.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.msq; + +import com.google.inject.Inject; +import org.apache.commons.io.IOUtils; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testing.utils.DataLoaderHelper; +import org.apache.druid.testing.utils.MsqTestQueryHelper; +import org.apache.druid.testing.utils.TestQueryHelper; +import org.apache.druid.testsEx.indexer.AbstractITBatchIndexTest; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +public class AbstractITSQLBasedIngestion +{ + public static final Logger LOG = new Logger(TestQueryHelper.class); + @Inject + private MsqTestQueryHelper msqHelper; + + @Inject + protected TestQueryHelper queryHelper; + + @Inject + private DataLoaderHelper dataLoaderHelper; + + /** + * Reads file as utf-8 string and replace %%DATASOURCE%% with the provide datasource value. + */ + protected String getStringFromFileAndReplaceDatasource(String filePath, String datasource) + { + String fileString; + try { + InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(filePath); + fileString = IOUtils.toString(is, StandardCharsets.UTF_8); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", filePath); + } + + fileString = StringUtils.replace( + fileString, + "%%DATASOURCE%%", + datasource + ); + + return fileString; + } + + /** + * Reads native queries from a file and runs against the provided datasource. + */ + protected void doTestQuery(String queryFilePath, String dataSource) + { + try { + String query = getStringFromFileAndReplaceDatasource(queryFilePath, dataSource); + queryHelper.testQueriesFromString(query); + } + catch (Exception e) { + LOG.error(e, "Error while running test query"); + throw new RuntimeException(e); + } + } + + /** + * Sumits a sqlTask, waits for task completion and then runs test queries on ingested datasource. + */ + protected void submitTaskAnddoTestQuery(String sqlTask, String queryFilePath, String datasource, + Map msqContext) throws Exception + { + LOG.info("SqlTask - \n %s", sqlTask); + + // Submit the tasks and wait for the datasource to get loaded + msqHelper.submitMsqTaskAndWaitForCompletion( + sqlTask, + msqContext + ); + + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + doTestQuery(queryFilePath, datasource); + } + + /** + * Runs a MSQ ingest sql test. + * + * @param sqlFilePath path of file containing the sql query. + * @param queryFilePath path of file containing the native test queries to be run on the ingested datasource. + * @param datasource name of the datasource. %%DATASOURCE%% in the sql and queries will be replaced with this value. + * @param msqContext context parameters to be passed with MSQ API call. + */ + protected void runMSQTaskandTestQueries(String sqlFilePath, String queryFilePath, String datasource, + Map msqContext) throws Exception + { + LOG.info("Starting MSQ test for [%s, %s]", sqlFilePath, queryFilePath); + + String sqlTask = getStringFromFileAndReplaceDatasource(sqlFilePath, datasource); + submitTaskAnddoTestQuery(sqlTask, queryFilePath, datasource, msqContext); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITSQLBasedBatchIngestion.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITSQLBasedBatchIngestion.java new file mode 100644 index 000000000000..dbc26d7d4085 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITSQLBasedBatchIngestion.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.msq; + +import junitparams.Parameters; +import org.apache.commons.io.FilenameUtils; +import org.apache.curator.shaded.com.google.common.collect.ImmutableMap; +import org.apache.druid.testsEx.categories.MultiStageQuery; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.Arrays; +import java.util.List; + +@RunWith(DruidTestRunner.class) +@Category(MultiStageQuery.class) +public class ITSQLBasedBatchIngestion extends AbstractITSQLBasedIngestion +{ + private static final String BATCH_INDEX_TASKS_DIR = "/multi-stage-query/batch-index/"; + + public static List> test_cases() + { + return Arrays.asList( + Arrays.asList("msq_inline.sql", "json_path_index_queries.json"), + Arrays.asList("sparse_column_msq.sql", "sparse_column_msq.json"), + Arrays.asList("wikipedia_http_inputsource_msq.sql", "wikipedia_http_inputsource_queries.json"), + Arrays.asList("wikipedia_index_msq.sql", "wikipedia_index_queries.json"), + Arrays.asList("wikipedia_merge_index_task.sql", "wikipedia_index_queries.json"), + Arrays.asList("wikipedia_index_task_with_transform.sql", "wikipedia_index_queries_with_transform.json") + ); + + } + + @Test + @Parameters(method = "test_cases") + public void testSQLBasedBatchIngestion(String sqlFileName, String queryFileName) + { + try { + runMSQTaskandTestQueries(BATCH_INDEX_TASKS_DIR + sqlFileName, + BATCH_INDEX_TASKS_DIR + queryFileName, + FilenameUtils.removeExtension(sqlFileName), + ImmutableMap.of("finalizeAggregations", false, + "maxNumTasks", 5, + "groupByEnableMultiValueUnnesting", false + )); + } + catch (Exception e) { + LOG.error(e, "Error while testing [%s, %s]", sqlFileName, queryFileName); + throw new RuntimeException(e); + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/json_path_index_queries.json b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/json_path_index_queries.json new file mode 100644 index 000000000000..845af00dd883 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/json_path_index_queries.json @@ -0,0 +1,49 @@ +[ + { + "description": "timeseries", + "query": { + "queryType": "timeseries", + "dataSource": "%%DATASOURCE%%", + "intervals": [ + "1000/3000" + ], + "aggregations": [ + { + "type": "longSum", + "name": "len", + "fieldName": "len" + }, + { + "type": "longSum", + "name": "max", + "fieldName": "max" + }, + { + "type": "longSum", + "name": "min", + "fieldName": "min" + }, + { + "type": "longSum", + "name": "sum", + "fieldName": "sum" + } + ], + "granularity": { + "type": "all" + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "sum": 10, + "min": 0, + "len": 5, + "max": 4 + } + } + ] + } +] + diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/msq_inline.sql b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/msq_inline.sql new file mode 100644 index 000000000000..a710691574ae --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/msq_inline.sql @@ -0,0 +1,17 @@ +REPLACE INTO "%%DATASOURCE%%" OVERWRITE ALL +WITH "source" AS (SELECT * FROM TABLE( + EXTERN( + '{"type":"inline","data":"{\"timestamp\": \"2013-08-31T01:02:33Z\", \"values\": [0,1,2,3,4] }"}', + '{"type":"json","flattenSpec":{"useFieldDiscovery":true,"fields":[{"type":"path","name":"len","expr":"$.values.length()"},{"type":"path","name":"min","expr":"$.values.min()"},{"type":"path","name":"max","expr":"$.values.max()"},{"type":"path","name":"sum","expr":"$.values.sum()"}]}}', + '[{"name":"timestamp","type":"string"},{"name":"len","type":"long"},{"name":"min","type":"long"},{"name":"max","type":"long"},{"name":"sum","type":"long"}]' + ) +)) +SELECT + TIME_PARSE("timestamp") AS __time, + "len", + "min", + "max", + "sum" +FROM "source" +GROUP BY 1, 2, 3, 4, 5 +PARTITIONED BY HOUR \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.json b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.json new file mode 100644 index 000000000000..4c2c5aa2950e --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.json @@ -0,0 +1,93 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2015-09-12T00:00:00.000Z", + "result" : { + "minTime" : "2015-09-12T00:00:00.000Z", + "maxTime" : "2015-09-12T00:00:00.000Z" + } + } + ] + }, + { + "description": "scan, all", + "query": { + "queryType": "scan", + "dataSource": "%%DATASOURCE%%", + "intervals": [ + "2013-01-01/2020-01-02" + ], + "resultFormat":"compactedList" + }, + "expectedResults": [{ + "segmentId":"dstsparse_column_msq.json_2015-09-12T00:00:00.000Z_2015-09-12T01:00:00.000Z_2022-11-17T12:32:11.247Z", + "columns":["__time","dimB","dimA","dimC","dimD","dimE","dimF","count","sum_metA"], + "events":[ + [1442016000000,"F","C",null,null,null,null,1,1], + [1442016000000,"J","C",null,null,null,null,1,1], + [1442016000000,"R","J",null,null,null,null,1,1], + [1442016000000,"S","Z",null,null,null,null,1,1], + [1442016000000,"T","H",null,null,null,null,1,1], + [1442016000000,"X",null,"A",null,null,null,1,1], + [1442016000000,"X","H",null,null,null,null,3,3], + [1442016000000,"Z","H",null,null,null,null,1,1] + ] + }], + "fieldsToTest": ["events"] + }, + { + "description": "roll up ratio", + "query": { + "queryType":"timeseries", + "dataSource":{ + "type":"table", + "name":"%%DATASOURCE%%" + }, + "intervals":{ + "type":"intervals", + "intervals":[ + "2013-01-01/2020-01-02" + ] + }, + "granularity":{ + "type":"all" + }, + "aggregations":[ + { + "type":"count", + "name":"a0" + }, + { + "type":"longSum", + "name":"a1", + "fieldName":"count", + "expression":null + } + ], + "postAggregations":[ + { + "type":"expression", + "name":"p0", + "expression":"((\"a0\" * 1.00) / \"a1\")", + "ordering":null + } + ] + }, + "expectedResults": [ + { + "timestamp" : "2015-09-12T00:00:00.000Z", + "result" : { + "a1" : 10, + "p0" : 0.8, + "a0" : 8 + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.sql b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.sql new file mode 100644 index 000000000000..f844f5996469 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/sparse_column_msq.sql @@ -0,0 +1,21 @@ +REPLACE INTO "%%DATASOURCE%%" OVERWRITE ALL +WITH "source" AS (SELECT * FROM TABLE( + EXTERN( + '{"type":"inline","data":"{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"F\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"J\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"Z\",\"dimB\":\"S\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"Z\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"J\",\"dimB\":\"R\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"T\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n"}', + '{"type":"json"}', + '[{"name":"time","type":"string"},{"name":"dimB","type":"string"},{"name":"dimA","type":"string"},{"name":"dimC","type":"string"},{"name":"dimD","type":"string"},{"name":"dimE","type":"string"},{"name":"dimF","type":"string"},{"name":"metA","type":"long"}]' + ) +)) +SELECT + TIME_FLOOR(TIME_PARSE("time"), 'PT1H') AS __time, + "dimB", + "dimA", + "dimC", + "dimD", + "dimE", + "dimF", + COUNT(*) AS "count", + SUM("metA") AS "sum_metA" +FROM "source" +GROUP BY 1, 2, 3, 4, 5, 6, 7 +PARTITIONED BY HOUR \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_msq.sql b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_msq.sql new file mode 100644 index 000000000000..f1af33bed43f --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_msq.sql @@ -0,0 +1,29 @@ +REPLACE INTO "%%DATASOURCE%%" OVERWRITE ALL +WITH "source" AS (SELECT * FROM TABLE( + EXTERN( + '{"type":"http","uris":["https://druid.apache.org/data/wikipedia.json.gz","https://druid.apache.org/data/wikipedia.json.gz"]}', + '{"type":"json"}', + '[{"name":"timestamp","type":"string"},{"name":"page","type":"string"},{"name":"language","type":"string"},{"name":"user","type":"string"},{"name":"unpatrolled","type":"string"},{"name":"newPage","type":"string"},{"name":"robot","type":"string"},{"name":"anonymous","type":"string"},{"name":"namespace","type":"string"},{"name":"continent","type":"string"},{"name":"country","type":"string"},{"name":"region","type":"string"},{"name":"city","type":"string"},{"name":"added","type":"double"},{"name":"deleted","type":"double"},{"name":"delta","type":"double"}]' + ) +)) +SELECT + TIME_FLOOR(CASE WHEN CAST("timestamp" AS BIGINT) > 0 THEN MILLIS_TO_TIMESTAMP(CAST("timestamp" AS BIGINT)) ELSE TIME_PARSE("timestamp") END, 'PT1S') AS __time, + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + COUNT(*) AS "count", + SUM("added") AS "added", + SUM("deleted") AS "deleted", + SUM("delta") AS "delta" +FROM "source" +GROUP BY 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 +PARTITIONED BY DAY \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_queries.json b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_queries.json new file mode 100644 index 000000000000..2d454d59d80a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_http_inputsource_queries.json @@ -0,0 +1,47 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2016-06-27T00:00:11.000Z", + "result" : { + "minTime" : "2016-06-27T00:00:11.000Z", + "maxTime" : "2016-06-27T21:31:02.000Z" + } + } + ] + }, + { + "description": "simple aggr", + "query":{ + "queryType" : "topN", + "dataSource" : "%%DATASOURCE%%", + "intervals" : ["2016-06-27/2016-06-28"], + "granularity" : "all", + "dimension" : "page", + "metric" : "count", + "threshold" : 3, + "aggregations" : [ + { + "type" : "count", + "name" : "count" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2016-06-27T00:00:11.000Z", + "result" : + [ + {"count":29,"page":"Copa América Centenario"}, + {"count":16,"page":"User:Cyde/List of candidates for speedy deletion/Subpage"}, + {"count":16,"page":"Wikipedia:Administrators' noticeboard/Incidents"} + ] + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_msq.sql b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_msq.sql new file mode 100644 index 000000000000..738e39fb8709 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_msq.sql @@ -0,0 +1,32 @@ +REPLACE INTO "%%DATASOURCE%%" OVERWRITE ALL +WITH "source" as (SELECT * FROM TABLE( + EXTERN( + '{"type":"local","files":["/resources/data/batch_index/json/wikipedia_index_data1.json","/resources/data/batch_index/json/wikipedia_index_data2.json","/resources/data/batch_index/json/wikipedia_index_data3.json"]}', + '{"type":"json"}', + '[{"name":"timestamp","type":"string"},{"name":"page","type":"string"},{"name":"language","type":"string"},{"name":"user","type":"string"},{"name":"unpatrolled","type":"string"},{"name":"newPage","type":"string"},{"name":"robot","type":"string"},{"name":"anonymous","type":"string"},{"name":"namespace","type":"string"},{"name":"continent","type":"string"},{"name":"country","type":"string"},{"name":"region","type":"string"},{"name":"city","type":"string"},{"name":"added","type":"double"},{"name":"deleted","type":"double"},{"name":"delta","type":"double"}]' + ) +)) +SELECT + TIME_FLOOR(CASE WHEN CAST("timestamp" AS BIGINT) > 0 THEN MILLIS_TO_TIMESTAMP(CAST("timestamp" AS BIGINT)) ELSE TIME_PARSE("timestamp") END, 'PT1S') AS __time, + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + COUNT(*) AS "count", + SUM("added") AS "added", + SUM("deleted") AS "deleted", + SUM("delta") AS "delta", + APPROX_COUNT_DISTINCT_DS_THETA("user") AS "thetaSketch", + DS_QUANTILES_SKETCH("delta") AS "quantilesDoublesSketch", + APPROX_COUNT_DISTINCT_DS_HLL("user") AS "HLLSketchBuild" +FROM "source" +GROUP BY 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 +PARTITIONED BY DAY \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries.json b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries.json new file mode 100644 index 000000000000..928effe65e97 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":5, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "first_user":"nuclear", + "last_user":"stringer" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries_with_transform.json b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries_with_transform.json new file mode 100644 index 000000000000..f0cfba677354 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_queries_with_transform.json @@ -0,0 +1,62 @@ +[ + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page", + "city" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"language-zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"triple-added", + "name":"added_count" + }, + { + "type":"longSum", + "fieldName":"delta", + "name":"delta_sum" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 27150.0, + "page" : "Crimson Typhoon", + "city" : "Taiyuan", + "added_count" : 2715, + "delta_sum" : 900, + "rows" : 1 + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_task_with_transform.sql b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_task_with_transform.sql new file mode 100644 index 000000000000..ebdeeda68937 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_index_task_with_transform.sql @@ -0,0 +1,32 @@ +REPLACE INTO "%%DATASOURCE%%" OVERWRITE ALL +WITH "source" AS (SELECT * FROM TABLE( + EXTERN( + '{"type":"local","baseDir":"/resources/data/batch_index/json","filter":"wikipedia_index_data*"}', + '{"type":"json"}', + '[{"name":"timestamp","type":"string"},{"name":"page","type":"string"},{"name":"language","type":"string"},{"name":"user","type":"string"},{"name":"unpatrolled","type":"string"},{"name":"robot","type":"string"},{"name":"anonymous","type":"string"},{"name":"namespace","type":"string"},{"name":"continent","type":"string"},{"name":"country","type":"string"},{"name":"region","type":"string"},{"name":"city","type":"string"},{"name":"added","type":"double"},{"name":"triple-added","type":"double"},{"name":"deleted","type":"double"},{"name":"delta","type":"double"}]' + ) +)) +SELECT + TIME_FLOOR(CASE WHEN CAST("timestamp" AS BIGINT) > 0 THEN MILLIS_TO_TIMESTAMP(CAST("timestamp" AS BIGINT)) ELSE TIME_PARSE("timestamp") END, 'PT1S') AS __time, + "page", + concat('language-', "language") AS "language", + "user", + "unpatrolled", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + COUNT(*) AS "count", + SUM("added") AS "added", + SUM("added")*3 AS "triple-added", + SUM("deleted") AS "deleted", + SUM("delta") AS "delta", + APPROX_COUNT_DISTINCT_DS_THETA("user") AS "thetaSketch", + DS_QUANTILES_SKETCH("delta") AS "quantilesDoublesSketch", + APPROX_COUNT_DISTINCT_DS_HLL("user") AS "HLLSketchBuild" +FROM "source" +GROUP BY 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 +PARTITIONED BY DAY \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_merge_index_task.sql b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_merge_index_task.sql new file mode 100644 index 000000000000..a8160aa9055b --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/batch-index/wikipedia_merge_index_task.sql @@ -0,0 +1,33 @@ +REPLACE INTO "%%DATASOURCE%%" OVERWRITE ALL +WITH "source" AS (SELECT * FROM TABLE( + EXTERN( + '{"type":"local","baseDir":"/resources/data/batch_index/json","filter":"wikipedia_index_data*"}', + '{"type":"json"}', + '[{"name":"timestamp","type":"string"},{"name":"page","type":"string"},{"name":"language","type":"string"},{"name":"user","type":"string"},{"name":"unpatrolled","type":"string"},{"name":"newPage","type":"string"},{"name":"robot","type":"string"},{"name":"anonymous","type":"string"},{"name":"namespace","type":"string"},{"name":"continent","type":"string"},{"name":"country","type":"string"},{"name":"region","type":"string"},{"name":"city","type":"string"},{"name":"added","type":"double"},{"name":"deleted","type":"double"},{"name":"delta","type":"double"}]' + ) +)) +SELECT + TIME_FLOOR(CASE WHEN CAST("timestamp" AS BIGINT) > 0 THEN MILLIS_TO_TIMESTAMP(CAST("timestamp" AS BIGINT)) ELSE TIME_PARSE("timestamp") END, 'PT1S') AS __time, + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "timestamp", + COUNT(*) AS "count", + SUM("added") AS "added", + SUM("deleted") AS "deleted", + SUM("delta") AS "delta", + APPROX_COUNT_DISTINCT_DS_THETA("user") AS "thetaSketch", + DS_QUANTILES_SKETCH("delta") AS "quantilesDoublesSketch", + APPROX_COUNT_DISTINCT_DS_HLL("user") AS "HLLSketchBuild" +FROM "source" +GROUP BY 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 +PARTITIONED BY DAY \ No newline at end of file diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java index 424d070529fe..37647fde87d0 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java @@ -43,6 +43,7 @@ import org.apache.druid.testing.clients.SqlResourceTestClient; import org.apache.druid.testing.clients.msq.MsqOverlordResourceTestClient; import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.testng.Assert; import java.util.ArrayList; import java.util.Collections; @@ -95,7 +96,15 @@ public String getQueryURL(String schemeAndHost) */ public SqlTaskStatus submitMsqTask(String sqlQueryString) throws ExecutionException, InterruptedException { - return submitMsqTask(new SqlQuery(sqlQueryString, null, false, false, false, ImmutableMap.of(), null)); + return submitMsqTask(sqlQueryString, ImmutableMap.of()); + } + + /** + * Submits a task to the MSQ API with the given query string, and default headers and custom context parameters + */ + public SqlTaskStatus submitMsqTask(String sqlQueryString, Map context) throws ExecutionException, InterruptedException + { + return submitMsqTask(new SqlQuery(sqlQueryString, null, false, false, false, context, null)); } // Run the task, wait for it to complete, fetch the reports, verify the results, @@ -154,6 +163,7 @@ public TaskState pollTaskIdForCompletion(String taskId) throws Exception throw new TaskStillRunningException(); }, (Throwable t) -> t instanceof TaskStillRunningException, + 99, 100 ); } @@ -250,6 +260,25 @@ public void testQueriesFromFile(String filePath, String fullDatasourcePath) thro } } + /** + * Submits a {@link SqlQuery} to the MSQ API for execution. This method waits for the created task to be completed. + */ + public void submitMsqTaskAndWaitForCompletion(String sqlQueryString, Map context) + throws Exception + { + SqlTaskStatus sqlTaskStatus = submitMsqTask(sqlQueryString, context); + + LOG.info("Sql Task submitted with task Id - %s", sqlTaskStatus.getTaskId()); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + } + private static class TaskStillRunningException extends Exception { From 4d1a3b169f785a6201ddfc72b2193656d6aa39b2 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 28 Nov 2022 18:57:42 +0530 Subject: [PATCH 11/27] Adding ITTest which kills the worker abruptly --- .../apache/druid/msq/exec/ControllerImpl.java | 5 +- .../org/apache/druid/msq/exec/MSQTasks.java | 22 +++ .../msq/indexing/MSQWorkerTaskLauncher.java | 13 +- .../msq/indexing/error/MSQFaultUtils.java | 2 +- .../controller/ControllerQueryKernel.java | 4 + .../apache/druid/msq/exec/MSQTasksTest.java | 18 ++ .../druid/msq/util/MSQFaultUtilsTest.java | 51 +++++ .../MultiStageQuery/docker-compose.yaml | 7 +- .../testsEx/msq/ITMultiStageQueryHA.java | 187 ++++++++++++++++++ .../wikipedia_msq_select_query_ha.json | 55 ++++++ 10 files changed, 359 insertions(+), 5 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQFaultUtilsTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java create mode 100644 integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_msq_select_query_ha.json diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 4a6ff31bb0d1..ef491964fdb2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -653,7 +653,10 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics public void workerError(MSQErrorReport errorReport) { // move inside kernel - if (!workerTaskLauncher.isTaskCanceledByController(errorReport.getTaskId())) { + if (workerTaskLauncher.isTaskCanceledByController(errorReport.getTaskId()) || + workerTaskLauncher.isTaskRetried(errorReport.getTaskId())) { + log.info("Ignoring task %s", errorReport.getTaskId()); + } else { workerErrorRef.compareAndSet(null, errorReport); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java index bceace299169..890f5d0c60b1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java @@ -23,6 +23,7 @@ import com.google.inject.Injector; import com.google.inject.Key; import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.guice.MultiStageQuery; import org.apache.druid.msq.indexing.error.CanceledFault; @@ -45,6 +46,8 @@ import javax.annotation.Nullable; import java.util.UUID; +import java.util.regex.Matcher; +import java.util.regex.Pattern; public class MSQTasks { @@ -55,6 +58,8 @@ public class MSQTasks private static final String TASK_ID_PREFIX = "query-"; + private static final Pattern WORKER_PATTERN = Pattern.compile(".*-worker([0-9]+)_[0-9]+"); + /** * Returns a controller task ID given a SQL query id. */ @@ -71,6 +76,23 @@ public static String workerTaskId(final String controllerTaskId, final int worke return StringUtils.format("%s-worker%d_%d", controllerTaskId, workerNumber, retryCount); } + /** + * Extract worker from taskId or throw exception if unable to parse out the worker. + */ + public static int workerFromTaskId(final String taskId) + { + final Matcher matcher = WORKER_PATTERN.matcher(taskId); + if (matcher.matches()) { + return Integer.parseInt(matcher.group(1)); + } else { + throw new ISE( + "Desired pattern %s to extract worker from task id %s did not match ", + WORKER_PATTERN.pattern(), + taskId + ); + } + } + /** * If "Object" is a Long, returns it. Otherwise, throws an appropriate exception assuming this operation is * being done to read the primary timestamp (__time) as part of an INSERT. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 6eddd3da8b3f..80b8a046d22b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -36,6 +36,7 @@ import org.apache.druid.msq.exec.ControllerContext; import org.apache.druid.msq.exec.ControllerImpl; import org.apache.druid.msq.exec.Limits; +import org.apache.druid.msq.exec.MSQTasks; import org.apache.druid.msq.exec.WorkerManagerClient; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQWarnings; @@ -282,6 +283,12 @@ public boolean isTaskCanceledByController(String taskId) return canceledWorkerTasks.contains(taskId); } + + public boolean isTaskRetried(String taskId) + { + return tasksToCleanup.contains(taskId) || workersToRelaunch.contains(MSQTasks.workerFromTaskId(taskId)); + } + private void mainLoop() { try { @@ -537,13 +544,17 @@ private void relaunchTasks() taskTrackers.remove(latestTaskId); currentRelaunchCount += 1; taskTrackers.put(relaunchedTask.getId(), new TaskTracker(relaunchedTask.getWorkerNumber(), relaunchedTask)); + synchronized (taskIds) { + fullyStartedTasks.remove(relaunchedTask.getWorkerNumber()); + taskIds.notifyAll(); + } + context.workerManager().run(relaunchedTask.getId(), relaunchedTask); taskHistory.add(relaunchedTask.getId()); synchronized (taskIds) { // replace taskId with the retry taskID for the same worker number taskIds.set(toRelaunch.getWorkerNumber(), relaunchedTask.getId()); - fullyStartedTasks.remove(relaunchedTask.getWorkerNumber()); taskIds.notifyAll(); } return taskHistory; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java index 35a97b0fc002..a3ede9726f7a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java @@ -44,7 +44,7 @@ public static String generateMessageWithErrorCode(MSQFault msqFault) */ public static String getErrorCodeFromMessage(String message) { - if (message == null || message.isEmpty()) { + if (message == null || message.isEmpty() || !message.contains(ERROR_CODE_DELIMITER)) { return UnknownFault.CODE; } return message.split(ERROR_CODE_DELIMITER, 2)[0]; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 8e58e13ce7a5..459ee7e9a8cb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -32,6 +32,7 @@ import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.QueryValidator; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.MSQException; @@ -74,6 +75,7 @@ */ public class ControllerQueryKernel { + private static final Logger log = new Logger(ControllerQueryKernel.class); private final QueryDefinition queryDef; private final int partitionStatisticsMaxRetainedBytes; @@ -636,6 +638,8 @@ public List getWorkInCaseWorkerElgibileForRetryElseThrow(int workerNu errorCode = msqFault.getErrorCode(); } + log.info("Parsed out errorCode[%s] to check eligibility for retry", errorCode); + if (retriableErrorCodes.contains(errorCode)) { return getWorkInCaseWorkerElgibileForRetryElseThrow(workerNumber); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java index 9acd49cb212c..f84bafec7672 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java @@ -23,6 +23,7 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; import org.apache.druid.msq.indexing.error.MSQErrorReport; @@ -139,6 +140,23 @@ public void test_makeErrorReport_workerPreferred() ); } + @Test + public void test_getWorkerFromTaskId() + { + Assert.assertEquals(1, MSQTasks.workerFromTaskId("xxxx-worker1_0")); + Assert.assertEquals(10, MSQTasks.workerFromTaskId("xxxx-worker10_0")); + Assert.assertEquals(0, MSQTasks.workerFromTaskId("xxdsadxx-worker0_0")); + Assert.assertEquals(90, MSQTasks.workerFromTaskId("dx-worker90_0")); + Assert.assertEquals(9, MSQTasks.workerFromTaskId("12dsa1-worker9_0")); + + Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worker-0")); + Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("worker-0")); + Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worker1-0")); + Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worker0-")); + Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worr1_0")); + Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worker-1-0")); + } + @Test public void test_queryWithoutEnoughSlots_shouldThrowException() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQFaultUtilsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQFaultUtilsTest.java new file mode 100644 index 000000000000..21b04623eda6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQFaultUtilsTest.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.util; + +import org.apache.druid.msq.indexing.error.MSQFaultUtils; +import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.error.WorkerFailedFault; +import org.junit.Assert; +import org.junit.Test; + +public class MSQFaultUtilsTest +{ + + + @Test + public void testGetErrorCodeFromMessage() + { + Assert.assertEquals(UnknownFault.CODE, MSQFaultUtils.getErrorCodeFromMessage( + "Task execution process exited unsuccessfully with code[137]. See middleManager logs for more details...")); + + Assert.assertEquals(UnknownFault.CODE, MSQFaultUtils.getErrorCodeFromMessage("")); + Assert.assertEquals(UnknownFault.CODE, MSQFaultUtils.getErrorCodeFromMessage(null)); + + Assert.assertEquals("ABC", MSQFaultUtils.getErrorCodeFromMessage("ABC: xyz xyz : xyz")); + + Assert.assertEquals( + WorkerFailedFault.CODE, + MSQFaultUtils.getErrorCodeFromMessage(MSQFaultUtils.generateMessageWithErrorCode(new WorkerFailedFault( + "123", + "error" + ))) + ); + } +} diff --git a/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml b/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml index f22ff3e6ddbe..2ce345063ac1 100644 --- a/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml +++ b/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml @@ -82,12 +82,15 @@ services: depends_on: - zookeeper - indexer: + middlemanager: extends: file: ../Common/druid.yaml - service: indexer + service: middlemanager environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_msq_intermediate_storage_enable=true + - druid_msq_intermediate_storage_type=local + - druid_msq_intermediate_storage_basePath=/shared/durablestorage/ volumes: # Test data - ../../resources:/resources diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java new file mode 100644 index 000000000000..74da22b136bd --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.msq; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.sql.SqlTaskStatus; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.clients.SqlResourceTestClient; +import org.apache.druid.testing.utils.DataLoaderHelper; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.MsqTestQueryHelper; +import org.apache.druid.testsEx.categories.MultiStageQuery; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.utils.DruidClusterAdminClient; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category(MultiStageQuery.class) +public class ITMultiStageQueryHA +{ + private static final Logger LOG = new Logger(ITMultiStageQueryHA.class); + @Inject + private MsqTestQueryHelper msqHelper; + + @Inject + private SqlResourceTestClient msqClient; + + @Inject + private IntegrationTestingConfig config; + + @Inject + private ObjectMapper jsonMapper; + + @Inject + private DataLoaderHelper dataLoaderHelper; + + @Inject + private CoordinatorResourceTestClient coordinatorClient; + + @Inject + private DruidClusterAdminClient druidClusterAdminClient; + + private static final String QUERY_FILE = "/multi-stage-query/wikipedia_msq_select_query_ha.json"; + + @Test + public void testMsqIngestionAndQuerying() throws Exception + { + String datasource = "dst"; + + // Clear up the datasource from the previous runs + coordinatorClient.unloadSegmentsForDataSource(datasource); + + String queryLocal = + StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot,\n" + + " diffUrl,\n" + + " added,\n" + + " countryIsoCode,\n" + + " regionName,\n" + + " channel,\n" + + " flags,\n" + + " delta,\n" + + " isUnpatrolled,\n" + + " isNew,\n" + + " deltaBucket,\n" + + " isMinor,\n" + + " isAnonymous,\n" + + " deleted,\n" + + " cityName,\n" + + " metroCode,\n" + + " namespace,\n" + + " comment,\n" + + " page,\n" + + " commentLength,\n" + + " countryName,\n" + + " user,\n" + + " regionIsoCode\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\",\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY DAY\n" + + "CLUSTERED BY \"__time\"", + datasource + ); + + // Submit the task and wait for the datasource to get loaded + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask( + queryLocal, + ImmutableMap.of( + MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, + "true", + MultiStageQueryContext.CTX_MAX_NUM_TASKS, + 3 + ) + ); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + + String taskIdToKill = sqlTaskStatus.getTaskId() + "-worker1_0"; + killTaskAbruptly(taskIdToKill); + + msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } + + private void killTaskAbruptly(String taskIdToKill) + { + + String command = "jps -mlv | grep -i peon | grep -i " + taskIdToKill + " |awk '{print $1}'"; + + ITRetryUtil.retryUntil(() -> { + + Pair stdOut = druidClusterAdminClient.runCommandInMiddleManagerContainer("/bin/bash", "-c", + command + ); + LOG.info(StringUtils.format( + "command %s \nstdout: %s\nstderr: %s", + command, + stdOut.lhs, + stdOut.rhs + )); + if (stdOut.rhs != null && stdOut.rhs.length() != 0) { + throw new ISE("Bad command"); + } + String pidToKill = stdOut.lhs.trim(); + if (pidToKill.length() != 0) { + LOG.info("Found PID to kill %s", pidToKill); + // kill worker after 5 seconds + Thread.sleep(5000); + LOG.info("Killing pid %s", pidToKill); + druidClusterAdminClient.runCommandInMiddleManagerContainer( + "/bin/bash", + "-c", + "kill -9 " + pidToKill + ); + return true; + } else { + return false; + } + }, true, 6000, 50, StringUtils.format("Figuring out PID for task[%s] to kill abruptly", taskIdToKill)); + + + } +} diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_msq_select_query_ha.json b/integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_msq_select_query_ha.json new file mode 100644 index 000000000000..992eda01a26a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_msq_select_query_ha.json @@ -0,0 +1,55 @@ +[ + { + "query": "SELECT __time, isRobot, added, delta, deleted, namespace FROM %%DATASOURCE%%", + "expectedResults": [ + { + "__time": 1377910953000, + "isRobot": null, + "added": 57, + "delta": -143, + "deleted": 200, + "namespace": "article" + }, + { + "__time": 1377910953000, + "isRobot": null, + "added": 57, + "delta": -143, + "deleted": 200, + "namespace": "article" + }, + { + "__time": 1377919965000, + "isRobot": null, + "added": 459, + "delta": 330, + "deleted": 129, + "namespace": "wikipedia" + }, + { + "__time": 1377919965000, + "isRobot": null, + "added": 459, + "delta": 330, + "deleted": 129, + "namespace": "wikipedia" + }, + { + "__time": 1377933081000, + "isRobot": null, + "added": 123, + "delta": 111, + "deleted": 12, + "namespace": "article" + }, + { + "__time": 1377933081000, + "isRobot": null, + "added": 123, + "delta": 111, + "deleted": 12, + "namespace": "article" + } + ] + } +] \ No newline at end of file From ecb1942c22813ab111cd6ca48be22a891cd0bf2a Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 29 Nov 2022 11:57:11 +0530 Subject: [PATCH 12/27] Review comments phase one --- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../druid/msq/guice/MSQIndexingModule.java | 8 +++---- .../msq/indexing/MSQWorkerTaskLauncher.java | 20 ++++++++++++----- .../msq/indexing/error/MSQFaultUtils.java | 2 +- ...dFault.java => TooManyAttemptsForJob.java} | 10 ++++----- ...mes.java => TooManyAttemptsForWorker.java} | 15 +++++-------- .../druid/msq/kernel/StageDefinition.java | 13 +++++++++++ .../controller/ControllerQueryKernel.java | 22 +++++++++---------- .../controller/ControllerStagePhase.java | 2 +- .../controller/ControllerStageTracker.java | 16 +++++++++----- .../apache/druid/msq/exec/MSQInsertTest.java | 2 +- .../druid/msq/indexing/MSQWorkerTaskTest.java | 1 - .../msq/indexing/error/MSQFaultSerdeTest.java | 4 ++-- .../BaseControllerQueryKernelTest.java | 2 +- .../apache/druid/msq/test/MSQTestBase.java | 10 ++++----- 15 files changed, 75 insertions(+), 54 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/{TotalRelaunchLimitExceededFault.java => TooManyAttemptsForJob.java} (90%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/{WorkerRelaunchedTooManyTimes.java => TooManyAttemptsForWorker.java} (88%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index ef491964fdb2..ff6fde668841 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -591,7 +591,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) */ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault fault) { - List retriableWorkOrders = kernel.getWorkInCaseWorkerElgibileForRetryElseThrow(worker, fault); + List retriableWorkOrders = kernel.getWorkInCaseWorkerEligibileForRetryElseThrow(worker, fault); if (retriableWorkOrders.size() != 0) { log.info("Submitting worker[%s] for relaunch because of fault[%s]", worker, fault); workerTaskLauncher.submitForRelaunch(worker); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 35af54d8be26..f037667a398a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -58,16 +58,16 @@ import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; import org.apache.druid.msq.indexing.error.RowTooLargeFault; import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; +import org.apache.druid.msq.indexing.error.TooManyAttemptsForJob; +import org.apache.druid.msq.indexing.error.TooManyAttemptsForWorker; import org.apache.druid.msq.indexing.error.TooManyBucketsFault; import org.apache.druid.msq.indexing.error.TooManyColumnsFault; import org.apache.druid.msq.indexing.error.TooManyInputFilesFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; import org.apache.druid.msq.indexing.error.TooManyWarningsFault; import org.apache.druid.msq.indexing.error.TooManyWorkersFault; -import org.apache.druid.msq.indexing.error.TotalRelaunchLimitExceededFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; -import org.apache.druid.msq.indexing.error.WorkerRelaunchedTooManyTimes; import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.input.NilInputSlice; @@ -124,10 +124,10 @@ public class MSQIndexingModule implements DruidModule TooManyPartitionsFault.class, TooManyWarningsFault.class, TooManyWorkersFault.class, - TotalRelaunchLimitExceededFault.class, + TooManyAttemptsForJob.class, UnknownFault.class, WorkerFailedFault.class, - WorkerRelaunchedTooManyTimes.class, + TooManyAttemptsForWorker.class, WorkerRpcFailedFault.class ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 80b8a046d22b..98801252a24a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -41,10 +41,10 @@ import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; -import org.apache.druid.msq.indexing.error.TotalRelaunchLimitExceededFault; +import org.apache.druid.msq.indexing.error.TooManyAttemptsForJob; +import org.apache.druid.msq.indexing.error.TooManyAttemptsForWorker; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; -import org.apache.druid.msq.indexing.error.WorkerRelaunchedTooManyTimes; import org.apache.druid.msq.util.MultiStageQueryContext; import javax.annotation.Nullable; @@ -538,10 +538,18 @@ private void relaunchTasks() MSQWorkerTask relaunchedTask = toRelaunch.getRetryTask(); // check relaunch limits - checkRelaunchLimits(tracker, toRelaunch); + checkRelaunchLimitsOrThrow(tracker, toRelaunch); // clean up trackers and tasks tasksToCleanup.add(latestTaskId); taskTrackers.remove(latestTaskId); + log.info( + "Relauching worker[%d] with new task id[%s] with worker relaunch count[%d] and job relaunch count[%d]", + relaunchedTask.getWorkerNumber(), + relaunchedTask.getId(), + toRelaunch.getRetryCount(), + currentRelaunchCount + ); + currentRelaunchCount += 1; taskTrackers.put(relaunchedTask.getId(), new TaskTracker(relaunchedTask.getWorkerNumber(), relaunchedTask)); synchronized (taskIds) { @@ -564,10 +572,10 @@ private void relaunchTasks() } } - private void checkRelaunchLimits(TaskTracker tracker, MSQWorkerTask relaunchTask) + private void checkRelaunchLimitsOrThrow(TaskTracker tracker, MSQWorkerTask relaunchTask) { if (relaunchTask.getRetryCount() > Limits.PER_WORKER_RELAUNCH_LIMIT) { - throw new MSQException(new WorkerRelaunchedTooManyTimes( + throw new MSQException(new TooManyAttemptsForWorker( Limits.PER_WORKER_RELAUNCH_LIMIT, relaunchTask.getId(), relaunchTask.getWorkerNumber(), @@ -575,7 +583,7 @@ private void checkRelaunchLimits(TaskTracker tracker, MSQWorkerTask relaunchTask )); } if (currentRelaunchCount > Limits.TOTAL_RELAUNCH_LIMIT) { - throw new MSQException(new TotalRelaunchLimitExceededFault( + throw new MSQException(new TooManyAttemptsForJob( Limits.TOTAL_RELAUNCH_LIMIT, currentRelaunchCount, relaunchTask.getId(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java index a3ede9726f7a..781639b17adc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQFaultUtils.java @@ -39,7 +39,7 @@ public static String generateMessageWithErrorCode(MSQFault msqFault) } /** - * Gets the error code from the message. If the messay is empty or null, {@link UnknownFault#CODE} is returned. This method + * Gets the error code from the message. If the message is empty or null, {@link UnknownFault#CODE} is returned. This method * does not gurantee that the error code we get out of the message is a valid error code. */ public static String getErrorCodeFromMessage(String message) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRelaunchLimitExceededFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyAttemptsForJob.java similarity index 90% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRelaunchLimitExceededFault.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyAttemptsForJob.java index b26548c1db4c..a2cb9c3d1fb0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TotalRelaunchLimitExceededFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyAttemptsForJob.java @@ -25,10 +25,10 @@ import java.util.Objects; -@JsonTypeName(TotalRelaunchLimitExceededFault.CODE) -public class TotalRelaunchLimitExceededFault extends BaseMSQFault +@JsonTypeName(TooManyAttemptsForJob.CODE) +public class TooManyAttemptsForJob extends BaseMSQFault { - static final String CODE = "TotalRelaunchLimitExceededFault"; + static final String CODE = "TooManyAttemptsForJob"; private final int maxRelaunchCount; @@ -41,7 +41,7 @@ public class TotalRelaunchLimitExceededFault extends BaseMSQFault private final String rootErrorMessage; @JsonCreator - public TotalRelaunchLimitExceededFault( + public TooManyAttemptsForJob( @JsonProperty("maxRelaunchCount") int maxRelaunchCount, @JsonProperty("currentRelaunchCount") int currentRelaunchCount, @JsonProperty("taskId") String taskId, @@ -99,7 +99,7 @@ public boolean equals(Object o) if (!super.equals(o)) { return false; } - TotalRelaunchLimitExceededFault that = (TotalRelaunchLimitExceededFault) o; + TooManyAttemptsForJob that = (TooManyAttemptsForJob) o; return maxRelaunchCount == that.maxRelaunchCount && currentRelaunchCount == that.currentRelaunchCount && Objects.equals( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRelaunchedTooManyTimes.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyAttemptsForWorker.java similarity index 88% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRelaunchedTooManyTimes.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyAttemptsForWorker.java index 83d6412f97af..dbf9b607e23c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/WorkerRelaunchedTooManyTimes.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyAttemptsForWorker.java @@ -25,10 +25,10 @@ import java.util.Objects; -@JsonTypeName(WorkerRelaunchedTooManyTimes.CODE) -public class WorkerRelaunchedTooManyTimes extends BaseMSQFault +@JsonTypeName(TooManyAttemptsForWorker.CODE) +public class TooManyAttemptsForWorker extends BaseMSQFault { - static final String CODE = "WorkerRelaunchedTooManyTimes"; + static final String CODE = "TooManyAttemptsForWorker"; private final int maxPerWorkerRelaunchCount; @@ -41,7 +41,7 @@ public class WorkerRelaunchedTooManyTimes extends BaseMSQFault private final String rootErrorMessage; @JsonCreator - public WorkerRelaunchedTooManyTimes( + public TooManyAttemptsForWorker( @JsonProperty("maxPerWorkerRelaunchCount") int maxPerWorkerRelaunchCount, @JsonProperty("taskId") String taskId, @JsonProperty("workerNumber") int workerNumber, @@ -98,13 +98,10 @@ public boolean equals(Object o) if (!super.equals(o)) { return false; } - WorkerRelaunchedTooManyTimes that = (WorkerRelaunchedTooManyTimes) o; + TooManyAttemptsForWorker that = (TooManyAttemptsForWorker) o; return maxPerWorkerRelaunchCount == that.maxPerWorkerRelaunchCount && workerNumber == that.workerNumber - && Objects.equals( - taskId, - that.taskId - ) + && Objects.equals(taskId, that.taskId) && Objects.equals(rootErrorMessage, that.rootErrorMessage); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java index c79bf46ddc9d..a55f862d5ad7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java @@ -259,6 +259,19 @@ public int getStageNumber() return id.getStageNumber(); } + /** + * Returns true, if the shuffling stage requires key statistics from the workers. + *

+ * Returns false, if the stage does not shuffle. + *

+ *

+ * It's possible we're shuffling using partition boundaries that are known ahead of time + * For eg: we know there's exactly one partition in query shapes like `select with limit`. + *

+ * In such cases, we return a false. + * + * @return + */ public boolean mustGatherResultKeyStatistics() { return shuffleSpec != null && shuffleSpec.needsStatistics(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 459ee7e9a8cb..ac88a18304fc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -61,7 +61,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; /** @@ -121,15 +120,16 @@ public class ControllerQueryKernel /** - * Store the work orders for the stage so that we can retrieve that in case of worker retry + * Map> + * Stores the work order per worker per stage so that we can retrieve that in case of worker retry */ private final Map> stageWorkOrders; /** * {@link MSQFault#getErrorCode()} which are retried. */ - private final Set retriableErrorCodes = ImmutableSet.of(CanceledFault.CODE, UnknownFault.CODE, - WorkerRpcFailedFault.CODE + private static final Set retriableErrorCodes = ImmutableSet.of(CanceledFault.CODE, UnknownFault.CODE, + WorkerRpcFailedFault.CODE ); public ControllerQueryKernel(final QueryDefinition queryDef, final int partitionStatisticsMaxRetainedBytes) @@ -143,7 +143,7 @@ public ControllerQueryKernel(final QueryDefinition queryDef, final int partition this.pendingInflowMap = computeStageInflowMap(queryDef); this.pendingOutflowMap = computeStageOutflowMap(queryDef); - stageWorkOrders = new ConcurrentHashMap<>(); + stageWorkOrders = new HashMap<>(); initializeReadyToRunStages(); } @@ -403,7 +403,7 @@ public Object getResultObjectForStage(final StageId stageId) /** * Checks if the stage can be started, delegates call to {@link ControllerStageTracker#start()} for internal phase - * transition and registers the transition in this queryKernel. Work orders need to created via {@link ControllerQueryKernel#createWorkOrders(int, Int2ObjectMap)} before calling this method. + * transition and registers the transition in this queryKernel. Work orders need to be created via {@link ControllerQueryKernel#createWorkOrders(int, Int2ObjectMap)} before calling this method. */ public void startStage(final StageId stageId) { @@ -628,7 +628,7 @@ private static Map> computeStageOutflowMap(final QueryDefi * @param msqFault * @return List of {@link WorkOrder} that needs to be retried. */ - public List getWorkInCaseWorkerElgibileForRetryElseThrow(int workerNumber, MSQFault msqFault) + public List getWorkInCaseWorkerEligibileForRetryElseThrow(int workerNumber, MSQFault msqFault) { final String errorCode; @@ -641,7 +641,7 @@ public List getWorkInCaseWorkerElgibileForRetryElseThrow(int workerNu log.info("Parsed out errorCode[%s] to check eligibility for retry", errorCode); if (retriableErrorCodes.contains(errorCode)) { - return getWorkInCaseWorkerElgibileForRetryElseThrow(workerNumber); + return getWorkInCaseWorkerEligibileForRetry(workerNumber); } else { throw new MSQException(msqFault); @@ -658,12 +658,10 @@ public List getWorkInCaseWorkerElgibileForRetryElseThrow(int workerNu * @param worker * @return List of {@link WorkOrder} that needs to be retried. */ - private List getWorkInCaseWorkerElgibileForRetryElseThrow(int worker) + private List getWorkInCaseWorkerEligibileForRetry(int worker) { List trackedSet = new ArrayList<>(getActiveStages()); - // no need to retry effectively finished stages - List getEffictivelyFinishedStages = getEffectivelyFinishedStageIds(); - trackedSet.removeAll(getEffictivelyFinishedStages); + trackedSet.removeAll(getEffectivelyFinishedStageIds()); List workOrders = new ArrayList<>(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index 151df2a93065..56d73950048f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java @@ -35,7 +35,7 @@ public enum ControllerStagePhase @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return true; + return false; } }, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index cfce3fdfd84b..b84e2713ed5c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -232,7 +232,7 @@ void workOrderSentForWorker(int worker) return WorkerStagePhase.READING_INPUT; }); if (phase != ControllerStagePhase.READING_INPUT) { - if (workOrdersNeedToBeSent()) { + if (allWorkOrdersSent()) { // if no more work orders need to be sent, change state to reading input from retrying. transitionTo(ControllerStagePhase.READING_INPUT); } @@ -369,7 +369,7 @@ ControllerStagePhase addResultKeyStatisticsForWorker( if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.canTransitionFrom(currentPhase)) { workerToPhase.put(workerNumber, WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); - // if stats already recieved for worker, donot update the sketch. + // if stats already received for worker, do not update the sketch. if (workersWithResultKeyStatistics.add(workerNumber)) { resultKeyStatisticsCollector.addAll(snapshot); } @@ -497,6 +497,8 @@ void fail() private void generateResultPartitionsAndBoundaries() { if (resultPartitions != null) { + // In case of retrying workers, we are perfectly fine using the partition boundaries generated before the retry + // took place. Hence, ignoring the request to generate result partitions. log.debug("Partition boundaries already generated for stage %d", stageDef.getStageNumber()); return; } @@ -544,10 +546,14 @@ private void generateResultPartitionsAndBoundaries() } /** - * True if all partitions stats are present, else false. + * True if all partitions stats are present for a shuffling stage which require statistics, else false. + * If the stage does not gather result statistics, we return a true. */ private boolean allPartitionStatisticsPresent() { + if (!stageDef.mustGatherResultKeyStatistics()) { + return true; + } return workerToPhase.values() .stream() .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) @@ -558,9 +564,9 @@ private boolean allPartitionStatisticsPresent() } /** - * True if work orders needs to be sent else false. + * True if all work orders are sent else false. */ - private boolean workOrdersNeedToBeSent() + private boolean allWorkOrdersSent() { return workerToPhase.values() .stream() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 0a9f3ea24348..d164d8eb58d3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -211,7 +211,7 @@ public void testInsertOnFoo1WithMultiValueDimGroupByWithoutGroupByEnable() } @Test - public void testRolltestRollUpOnFoo1UpOnFoo1() + public void testRollUpOnFoo1UpOnFoo1() { RowSignature rowSignature = RowSignature.builder() .add("__time", ColumnType.LONG) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java index 81cbcdc843c4..4884fda4c77d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQWorkerTaskTest.java @@ -103,5 +103,4 @@ public void testGetter() Assert.assertEquals(retryCount, msqWorkerTask.getRetryCount()); } - } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index 147dd2d07991..e46663f6bd84 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -74,8 +74,8 @@ public void testFaultSerde() throws IOException assertFaultSerde(new TooManyPartitionsFault(10)); assertFaultSerde(new TooManyWarningsFault(10, "the error")); assertFaultSerde(new TooManyWorkersFault(10, 5)); - assertFaultSerde(new WorkerRelaunchedTooManyTimes(2, "taskId", 1, "rootError")); - assertFaultSerde(new TotalRelaunchLimitExceededFault(2, 2, "taskId", "rootError")); + assertFaultSerde(new TooManyAttemptsForWorker(2, "taskId", 1, "rootError")); + assertFaultSerde(new TooManyAttemptsForJob(2, 2, "taskId", "rootError")); assertFaultSerde(UnknownFault.forMessage(null)); assertFaultSerde(UnknownFault.forMessage("the message")); assertFaultSerde(new WorkerFailedFault("the worker task", "the error msg")); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java index 212a3a1d49aa..e68079c66a39 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -361,7 +361,7 @@ public void sendPartitionBoundariesForStageAndWorkers(int stageNumber, int... wo public List getRetriableWorkOrdersAndChangeState(int workeNumber, MSQFault msqFault) { - return controllerQueryKernel.getWorkInCaseWorkerElgibileForRetryElseThrow(workeNumber, msqFault); + return controllerQueryKernel.getWorkInCaseWorkerEligibileForRetryElseThrow(workeNumber, msqFault); } public void failWorkerAndAssertWorkOrderes(int workeNumber, int retriedStage) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 6903cb104acf..3979d3f176c2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -78,7 +78,7 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.MSQFaultUtils; -import org.apache.druid.msq.indexing.error.WorkerRelaunchedTooManyTimes; +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; @@ -902,8 +902,8 @@ public void verifyResults() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { - String errorMessage = msqErrorReport.getFault() instanceof WorkerRelaunchedTooManyTimes - ? ((WorkerRelaunchedTooManyTimes) msqErrorReport.getFault()).getRootErrorMessage() + String errorMessage = msqErrorReport.getFault() instanceof TooManyAttemptsForWorker + ? ((TooManyAttemptsForWorker) msqErrorReport.getFault()).getRootErrorMessage() : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), @@ -1073,8 +1073,8 @@ public Pair>> runQueryWithResult() if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { - String errorMessage = msqErrorReport.getFault() instanceof WorkerRelaunchedTooManyTimes - ? ((WorkerRelaunchedTooManyTimes) msqErrorReport.getFault()).getRootErrorMessage() + String errorMessage = msqErrorReport.getFault() instanceof TooManyAttemptsForWorker + ? ((TooManyAttemptsForWorker) msqErrorReport.getFault()).getRootErrorMessage() : MSQFaultUtils.generateMessageWithErrorCode(msqErrorReport.getFault()); Assert.assertEquals( MSQFaultUtils.generateMessageWithErrorCode(expectedMSQFault), From 48c844ad132dd9abaf6f253ebe23de2a9c8b7f7f Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 29 Nov 2022 12:47:01 +0530 Subject: [PATCH 13/27] Adding doc changes --- docs/multi-stage-query/known-issues.md | 2 +- docs/multi-stage-query/reference.md | 8 ++++---- .../java/org/apache/druid/msq/exec/ControllerImpl.java | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md index 6d5fab7bf88a..c12227be4d97 100644 --- a/docs/multi-stage-query/known-issues.md +++ b/docs/multi-stage-query/known-issues.md @@ -29,7 +29,7 @@ sidebar_label: Known issues ## Multi-stage query task runtime -- Fault tolerance is partially implemented. Workers get relaunched when they are killed unexpectedly. The controller does not get relaunched if its killed unexpectedly. +- Fault tolerance is partially implemented. Workers get relaunched when they are killed unexpectedly. The controller does not get relaunched if it is killed unexpectedly. - Worker task stage outputs are stored in the working directory given by `druid.indexer.task.baseDir`. Stages that generate a large amount of output data may exhaust all available disk space. In this case, the query fails with diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 3c471514b2ec..6c80aff05190 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -234,8 +234,8 @@ The following table lists query limits: | Number of output columns for any one stage. | 2,000 | `TooManyColumns` | | Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | `TooManyWorkers` | | Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](concepts.md#memory-usage). | `BroadcastTablesTooLarge` | -| Maximum relaunches per worker. Initial run is not a relaunch. The worker will be spawned 1 + workerRelaunchLimit times before erroring out. | 2 | `WorkerRelaunchedTooManyTimes` | -| Maximum relaunches across all workers. | 30 | `TotalRelaunchLimitExceededFault` | +| Maximum relaunch attempts per worker. Initial run is not a relaunch. The worker will be spawned 1 + workerRelaunchLimit times before erroring out. | 2 | `TooManyAttemptsForWorker` | +| Maximum relaunch attempts for a job across all workers. | 100 | `TooManyAttemptsForJob` | @@ -262,15 +262,15 @@ The following table describes error codes you may encounter in the `multiStageQu | `QueryNotSupported` | QueryKit could not translate the provided native query to a multi-stage query.

This can happen if the query uses features that aren't supported, like GROUPING SETS. | | | `RowTooLarge` | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for the specific limit on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. | | `TaskStartTimeout` | Unable to launch all the worker tasks in time.

There might be insufficient available slots to start all the worker tasks simultaneously.

Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity. | `numTasks`: The number of tasks attempted to launch. | +| `TooManyAttemptsForJob` | Total relaunch attempt count across all workers exceeded max relaunch attempt limit. See the [Limits](#limits) table for the specific limit. | `maxRelaunchCount`: Max number of relaunches across all the workers defined in the [Limits](#limits) section.

`currentRelaunchCount`: current relaunch counter for the job across all workers.

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| +| `TooManyAttemptsForWorker` | Worker exceeded maximum relaunch attempt count as definied in the [Limits](#limits) section. |`maxPerWorkerRelaunchCount`: Max number of relaunches allower per worker as defined in the [Limits](#limits) section.

`workerNumber`: the worker number for which the task failed

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| | `TooManyBuckets` | Exceeded the number of partition buckets for a stage. Partition buckets are only used for `segmentGranularity` during INSERT queries. The most common reason for this error is that your `segmentGranularity` is too narrow relative to the data. See the [Limits](#limits) table for the specific limit. | `maxBuckets`: The limit on buckets. | | `TooManyInputFiles` | Exceeded the number of input files/segments per worker. See the [Limits](#limits) table for the specific limit. | `numInputFiles`: The total number of input files/segments for the stage.

`maxInputFiles`: The maximum number of input files/segments per worker per stage.

`minNumWorker`: The minimum number of workers required for a successful run. | | `TooManyPartitions` | Exceeded the number of partitions for a stage. The most common reason for this is that the final stage of an INSERT or REPLACE query generated too many segments. See the [Limits](#limits) table for the specific limit. | `maxPartitions`: The limit on partitions which was exceeded | | `TooManyColumns` | Exceeded the number of columns for a stage. See the [Limits](#limits) table for the specific limit. | `numColumns`: The number of columns requested.

`maxColumns`: The limit on columns which was exceeded. | | `TooManyWarnings` | Exceeded the allowed number of warnings of a particular type. | `rootErrorCode`: The error code corresponding to the exception that exceeded the required limit.

`maxWarnings`: Maximum number of warnings that are allowed for the corresponding `rootErrorCode`. | | `TooManyWorkers` | Exceeded the supported number of workers running simultaneously. See the [Limits](#limits) table for the specific limit. | `workers`: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously.

`maxWorkers`: The hard or soft limit on workers that was exceeded. | -| `TotalRelaunchLimitExceededFault` | Total relaunch count across all workers exceeded max relaunch limit. See the [Limits](#limits) table for the specific limit. | `maxRelaunchCount`: Max number of relaunches across all the workers defined in the [Limits](#limits) section.

`currentRelaunchCount`: current relaunch counter for the job.

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| | `NotEnoughMemory` | Insufficient memory to launch a stage. | `serverMemory`: The amount of memory available to a single process.

`serverWorkers`: The number of workers running in a single process.

`serverThreads`: The number of threads in a single process. | | `WorkerFailed` | A worker task failed unexpectedly. | `errorMsg`

`workerTaskId`: The ID of the worker task. | -| `WorkerRelaunchedTooManyTimes` | Worker exceeded maximum relaunch count as definied in the [Limits](#limits) section. |`maxPerWorkerRelaunchCount`: Max number of relaunches allower per worker as defined in the [Limits](#limits) section.

`workerNumber`: the worker number for which the task failed

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| | `WorkerRpcFailed` | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task | | `UnknownError` | All other errors. | `message` | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index ff6fde668841..3bad7550ea6c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1087,7 +1087,7 @@ public void onFailure(Throwable t) if (retryOnFailure) { log.info( t, - "Detected failure while contacting task[%s]. Iniitiating relaunch of worker[%d] if applicable", + "Detected failure while contacting task[%s]. Initiating relaunch of worker[%d] if applicable", taskId, workerNumber ); From ac417540324a404f8256142b97d207fd0ddeab80 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 29 Nov 2022 19:16:39 +0530 Subject: [PATCH 14/27] Adjusting for single threaded execution. --- .../apache/druid/msq/exec/ControllerImpl.java | 62 +++++++++++-------- .../org/apache/druid/msq/exec/Limits.java | 2 +- .../controller/ControllerStagePhase.java | 4 +- 3 files changed, 39 insertions(+), 29 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 3bad7550ea6c..149ea6e06779 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -270,8 +270,9 @@ public class ControllerImpl implements Controller // Time at which the query started. // For live reports. Written by the main controller thread, read by HTTP threads. - // WorkerNumber -> WorkOrders - private final ConcurrentHashMap> workOrdersToRetry = new ConcurrentHashMap<>(); + // WorkerNumber -> WorkOrders which need to be retried and our determined by the controller. + // Map is always populated in the main controller thread by addToRetryQueue, and pruned in retryFailedTasks. + private final Map> workOrdersToRetry = new HashMap<>(); private volatile DateTime queryStartTime = null; private volatile DruidNode selfDruidNode; @@ -287,9 +288,10 @@ public ControllerImpl( { this.task = task; this.context = context; - this.isDurableStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec() - .getQuery() - .context()); + this.isDurableStorageEnabled = + MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec() + .getQuery() + .context()); } @@ -586,8 +588,10 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) } /** - * Adds the workorders for worker to {@link ControllerImpl#workOrdersToRetry} if the {@link ControllerQueryKernel} determines that there + * Adds the work orders for worker to {@link ControllerImpl#workOrdersToRetry} if the {@link ControllerQueryKernel} determines that there * are work orders which needs reprocessing. + *

+ * This method is not thread safe, so it should always be called inside the main controller thread. */ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault fault) { @@ -597,9 +601,7 @@ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault workerTaskLauncher.submitForRelaunch(worker); workOrdersToRetry.compute(worker, (workerNumber, workOrders) -> { if (workOrders == null) { - Set orders = ConcurrentHashMap.newKeySet(); - orders.addAll(retriableWorkOrders); - return orders; + return new HashSet<>(retriableWorkOrders); } else { workOrders.addAll(retriableWorkOrders); return workOrders; @@ -1045,14 +1047,15 @@ private Int2ObjectMap> makeSegmentGeneratorWorkerFa * @param queryKernel * @param contactFn * @param workers set of workers to contact - * @param successCallBack on successfull api call, custom callback - * @param retryOnFailure if set to true, adds this worker to retry queue. If false, cancel all the futures and propergate the exception to the caller. + * @param successCallBack After contacting all the tasks, a custom callback is invoked in the main thread for each successfully contacted task. + * @param retryOnFailure If true, after contacting all the tasks, adds this worker to retry queue in the main thread. + * If false, cancel all the futures and propagate the exception to the caller. */ private void contactWorkersForStage( final ControllerQueryKernel queryKernel, final TaskContactFn contactFn, final IntSet workers, - final TaskContactSuccesss successCallBack, + final TaskContactSuccess successCallBack, final boolean retryOnFailure ) { @@ -1066,7 +1069,8 @@ private void contactWorkersForStage( throw new RuntimeException(e); } - Set> failedCalls = ConcurrentHashMap.newKeySet(); + Set failedCalls = ConcurrentHashMap.newKeySet(); + Set successfulCalls = ConcurrentHashMap.newKeySet(); for (int workerNumber : workers) { final String taskId = taskIds.get(workerNumber); @@ -1077,7 +1081,7 @@ private void contactWorkersForStage( @Override public void onSuccess(@Nullable Void result) { - successCallBack.onSuccess(taskId, workerNumber); + successfulCalls.add(taskId); settableFuture.set(true); } @@ -1089,9 +1093,9 @@ public void onFailure(Throwable t) t, "Detected failure while contacting task[%s]. Initiating relaunch of worker[%d] if applicable", taskId, - workerNumber + MSQTasks.workerFromTaskId(taskId) ); - failedCalls.add(new Pair<>(workerNumber, taskId)); + failedCalls.add(taskId); settableFuture.set(false); } else { settableFuture.setException(t); @@ -1104,9 +1108,13 @@ public void onFailure(Throwable t) FutureUtils.getUnchecked(MSQFutureUtils.allAsList(taskFutures, true), true); + for (String taskId : successfulCalls) { + successCallBack.onSuccess(taskId); + } + if (retryOnFailure) { - for (Pair workerIdTask : failedCalls) { - addToRetryQueue(queryKernel, workerIdTask.lhs, new WorkerRpcFailedFault(workerIdTask.rhs)); + for (String taskId : failedCalls) { + addToRetryQueue(queryKernel, MSQTasks.workerFromTaskId(taskId), new WorkerRpcFailedFault(taskId)); } } } @@ -1134,7 +1142,7 @@ private void startWorkForStage( queryKernel, (netClient, taskId, workerNumber) -> ( netClient.postWorkOrder(taskId, workOrders.get(workerNumber))), workOrders.keySet(), - (taskId, workerNumber) -> queryKernel.workOrdersSentForWorker(stageId, workerNumber), + (taskId) -> queryKernel.workOrdersSentForWorker(stageId, MSQTasks.workerFromTaskId(taskId)), isDurableStorageEnabled ); @@ -1158,7 +1166,7 @@ private void postResultPartitionBoundariesForStage( resultPartitionBoundaries ), workers, - (taskId, workerNumber) -> queryKernel.partitionBoundariesSentForWorker(stageId, workerNumber), + (taskId) -> queryKernel.partitionBoundariesSentForWorker(stageId, MSQTasks.workerFromTaskId(taskId)), isDurableStorageEnabled ); @@ -2151,10 +2159,11 @@ private void retryFailedTasks() throws InterruptedException stageWorkOrders.getValue().get(workerNumber) ), new IntArraySet(stageWorkOrders.getValue().keySet()), - (taskId, workerNumber) -> { + (taskId) -> { + int workerNumber = MSQTasks.workerFromTaskId(taskId); queryKernel.workOrdersSentForWorker(stageWorkOrders.getKey(), workerNumber); - // remove sucessfully contacted workOrders from workOrdersToRetry + // remove successfully contacted workOrders from workOrdersToRetry workOrdersToRetry.compute(workerNumber, (task, workOrderSet) -> { if (workOrderSet == null || workOrderSet.size() == 0 || !workOrderSet.remove(stageWorkOrders.getValue() .get( @@ -2168,7 +2177,6 @@ private void retryFailedTasks() throws InterruptedException }); }, isDurableStorageEnabled - ); } } @@ -2384,7 +2392,7 @@ private void cleanUpEffectivelyFinishedStages() queryKernel, (netClient, taskId, workerNumber) -> netClient.postCleanupStage(taskId, stageId), queryKernel.getWorkerInputsForStage(stageId).workers(), - (ignore1, ignore2) -> {}, + (ignore1) -> {}, false ); queryKernel.finishStage(stageId, true); @@ -2420,11 +2428,11 @@ private interface TaskContactFn } /** - * Interface used when {@link TaskContactFn#contactTask(WorkerClient, String, int)} return future is successfull. + * Interface used when {@link TaskContactFn#contactTask(WorkerClient, String, int)} returns a successful future. */ - private interface TaskContactSuccesss + private interface TaskContactSuccess { - void onSuccess(String taskId, int workerNumber); + void onSuccess(String taskId); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index ed8bb6c4ec43..b41cc585e427 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -62,7 +62,7 @@ public class Limits /** * Maximum relaunches across all workers. */ - public static final int TOTAL_RELAUNCH_LIMIT = 30; + public static final int TOTAL_RELAUNCH_LIMIT = 100; /** * Maximum relaunches per worker. Initial run is not a relaunch. The worker will be spawned 1 + workerRelaunchLimit times before erroring out. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index 56d73950048f..a9c8644e769f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java @@ -87,7 +87,9 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) } }, - // Stage currently under retry. priorPhase did not publish its final results yet. + // Stages whose workers are currently under relaunch. We can transition out of Retrying state only when all the work orders + // of this stage have been sent. + // We can transition into Retrying phase when the prior phase did not publish its final results yet. RETRYING { @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) From 7c31923ee5d5511085d218ba2414369ea1b86107 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 8 Dec 2022 11:10:37 +0530 Subject: [PATCH 15/27] Adding Sequential Merge PR state handling --- .../util/common/function/TriConsumer.java | 60 ++ .../util/common/function/TriConsumerTest.java | 45 ++ docs/multi-stage-query/reference.md | 4 +- .../apache/druid/msq/exec/ControllerImpl.java | 175 ++++-- .../org/apache/druid/msq/exec/WorkerImpl.java | 2 +- .../druid/msq/exec/WorkerSketchFetcher.java | 406 +++++-------- .../druid/msq/kernel/StageDefinition.java | 16 +- .../controller/ControllerQueryKernel.java | 126 +++- .../controller/ControllerStagePhase.java | 14 +- .../controller/ControllerStageTracker.java | 493 +++++++++++++-- .../ControllerWorkerStagePhase.java | 102 ++++ .../CompleteKeyStatisticsInformation.java | 26 +- .../msq/util/MultiStageQueryContext.java | 2 +- .../druid/msq/exec/ControllerImplTest.java | 106 ++++ .../apache/druid/msq/exec/MSQSelectTest.java | 6 +- .../exec/WorkerSketchFetcherAutoModeTest.java | 139 ----- .../msq/exec/WorkerSketchFetcherTest.java | 107 ++-- .../BaseControllerQueryKernelTest.java | 104 ++-- .../controller/ControllerQueryKernelTest.java | 58 +- ...onShufflingWorkersWithRetryKernelTest.java | 32 +- .../ShufflingWorkersWithRetryKernelTest.java | 570 +++++++++++++++--- 21 files changed, 1851 insertions(+), 742 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java create mode 100644 core/src/test/java/org/apache/druid/java/util/common/function/TriConsumerTest.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerWorkerStagePhase.java delete mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java diff --git a/core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java b/core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java new file mode 100644 index 000000000000..01d2ac5aebfb --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.java.util.common.function; + +import java.util.Objects; + +/** + * Based on {@link java.util.function.BiConsumer} + */ +@FunctionalInterface +public interface TriConsumer +{ + /** + * Performs this operation on the given arguments. + * + * @param t the first input argument + * @param u the second input argument + * @param v the third input argument + */ + void accept(T t, U u, V v); + + /** + * Returns a composed {@code BiConsumer} that performs, in sequence, this + * operation followed by the {@code after} operation. If performing either + * operation throws an exception, it is relayed to the caller of the + * composed operation. If performing this operation throws an exception, + * the {@code after} operation will not be performed. + * + * @param after the operation to perform after this operation + * @return a composed {@code BiConsumer} that performs in sequence this + * operation followed by the {@code after} operation + * @throws NullPointerException if {@code after} is null + */ + default TriConsumer andThen(TriConsumer after) + { + Objects.requireNonNull(after); + + return (t, u, v) -> { + accept(t, u, v); + after.accept(t, u, v); + }; + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/common/function/TriConsumerTest.java b/core/src/test/java/org/apache/druid/java/util/common/function/TriConsumerTest.java new file mode 100644 index 000000000000..3e3cafbeca46 --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/common/function/TriConsumerTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.java.util.common.function; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashSet; +import java.util.Set; + +public class TriConsumerTest +{ + + @Test + public void sanityTest() + { + Set sumSet = new HashSet<>(); + TriConsumer consumerA = (arg1, arg2, arg3) -> { + sumSet.add(arg1 + arg2 + arg3); + }; + TriConsumer consumerB = (arg1, arg2, arg3) -> { + sumSet.remove(arg1 + arg2 + arg3); + }; + consumerA.andThen(consumerB).accept(1, 2, 3); + + Assert.assertTrue(sumSet.isEmpty()); + } +} diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 060978898ee0..2fa2ea12c247 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -214,12 +214,12 @@ reading rows from the datasource. These statistics must be transferred to the co `PARALLEL` mode fetches the key statistics for all time chunks from all workers together and the controller then downsamples the sketch if it does not fit in memory. This is faster than `SEQUENTIAL` mode as there is less over head in fetching sketches -for all time chunks together. This is good for small sketches which won't be downsampled even if merged together or if +for all time chunks together. This is good for small sketches which won't be down sampled even if merged together or if accuracy in segment sizing for the ingestion is not very important. `SEQUENTIAL` mode fetches the sketches in ascending order of time and generates the partition boundaries for one time chunk at a time. This gives more working memory to the controller for merging sketches, which results in less -downsampling and thus, more accuracy. There is, however, a time overhead on fetching sketches in sequential order. This is +down sampling and thus, more accuracy. There is, however, a time overhead on fetching sketches in sequential order. This is good for cases where accuracy is important. `AUTO` mode tries to find the best approach based on number of workers and size of input rows. If there are more diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index a316d46d4d81..080cf733549d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -68,7 +68,6 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -113,7 +112,6 @@ import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; -import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; import org.apache.druid.msq.indexing.error.TooManyWarningsFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; @@ -157,7 +155,6 @@ import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory; import org.apache.druid.msq.shuffle.DurableStorageUtils; import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; -import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.msq.util.DimensionSchemaUtils; import org.apache.druid.msq.util.IntervalUtils; @@ -210,7 +207,6 @@ import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ThreadLocalRandom; @@ -286,6 +282,9 @@ public class ControllerImpl implements Controller private volatile FaultsExceededChecker faultsExceededChecker = null; + private Map stageToStatsMergingMode; + private WorkerMemoryParameters workerMemoryParameters; + public ControllerImpl( final MSQControllerTask task, final ControllerContext context @@ -526,7 +525,7 @@ public TaskStatus runTask(final Closer closer) *

* If the consumer throws an exception, the query fails. */ - private void addToKernelManipulationQueue(Consumer kernelConsumer) + public void addToKernelManipulationQueue(Consumer kernelConsumer) { if (!kernelManipulationQueue.offer(kernelConsumer)) { final String message = "Controller kernel queue is full. Main controller loop may be delayed or stuck."; @@ -541,15 +540,6 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); - ClusterStatisticsMergeMode clusterStatisticsMergeMode = - MultiStageQueryContext.getClusterStatisticsMergeMode(task.getQuerySpec().getQuery().context()); - - log.debug("Query [%s] cluster statistics merge mode is set to %s.", id(), clusterStatisticsMergeMode); - - int statisticsMaxRetainedBytes = WorkerMemoryParameters.createProductionInstanceForController(context.injector()) - .getPartitionStatisticsMaxRetainedBytes(); - this.workerSketchFetcher = new WorkerSketchFetcher(netClient, clusterStatisticsMergeMode, statisticsMaxRetainedBytes); - closer.register(netClient::close); final QueryDefinition queryDef = makeQueryDefinition( @@ -598,6 +588,28 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) ImmutableMap.of(CannotParseExternalDataFault.CODE, maxParseExceptions) ); + stageToStatsMergingMode = new HashMap<>(); + queryDef.getStageDefinitions().forEach( + stageDefinition -> + stageToStatsMergingMode.put( + stageDefinition.getId().getStageNumber(), + finalizeClusterStatisticsMergeMode( + stageDefinition, + MultiStageQueryContext.getClusterStatisticsMergeMode( + task.getQuerySpec() + .getQuery() + .context()) + ) + ) + ); + this.workerMemoryParameters = WorkerMemoryParameters.createProductionInstanceForController(context.injector()); + + this.workerSketchFetcher = new WorkerSketchFetcher( + netClient, + workerTaskLauncher, + isDurableStorageEnabled + ); + return queryDef; } @@ -609,7 +621,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) */ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault fault) { - List retriableWorkOrders = kernel.getWorkInCaseWorkerEligibileForRetryElseThrow(worker, fault); + List retriableWorkOrders = kernel.getWorkInCaseWorkerEligibleForRetryElseThrow(worker, fault); if (retriableWorkOrders.size() != 0) { log.info("Submitting worker[%s] for relaunch because of fault[%s]", worker, fault); workerTaskLauncher.submitForRelaunch(worker); @@ -635,7 +647,11 @@ private void addToRetryQueue(ControllerQueryKernel kernel, int worker, MSQFault * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ @Override - public void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject) + public void updatePartialKeyStatisticsInformation( + int stageNumber, + int workerNumber, + Object partialKeyStatisticsInformationObject + ) { addToKernelManipulationQueue( queryKernel -> { @@ -651,7 +667,10 @@ public void updatePartialKeyStatisticsInformation(int stageNumber, int workerNum final PartialKeyStatisticsInformation partialKeyStatisticsInformation; try { - partialKeyStatisticsInformation = mapper.convertValue(partialKeyStatisticsInformationObject, PartialKeyStatisticsInformation.class); + partialKeyStatisticsInformation = mapper.convertValue( + partialKeyStatisticsInformationObject, + PartialKeyStatisticsInformation.class + ); } catch (IllegalArgumentException e) { throw new IAE( @@ -663,39 +682,11 @@ public void updatePartialKeyStatisticsInformation(int stageNumber, int workerNum } queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation); - - if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) { - List workerTaskIds = workerTaskLauncher.getTaskList(); - CompleteKeyStatisticsInformation completeKeyStatisticsInformation = - queryKernel.getCompleteKeyStatisticsInformation(stageId); - - // Queue the sketch fetching task into the worker sketch fetcher. - CompletableFuture> clusterByPartitionsCompletableFuture = - workerSketchFetcher.submitFetcherTask( - completeKeyStatisticsInformation, - workerTaskIds, - stageDef - ); - - // Add the listener to handle completion. - clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> { - addToKernelManipulationQueue(holder -> { - if (throwable != null) { - holder.failStageForReason(stageId, UnknownFault.forException(throwable)); - } else if (clusterByPartitionsEither.isError()) { - holder.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); - } else { - log.debug("Query [%s] Partition boundaries generated for stage %s", id(), stageId); - holder.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow()); - } - holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId)); - }); - }); - } } ); } + @Override public void workerError(MSQErrorReport errorReport) { @@ -2129,7 +2120,11 @@ public RunQueryUntilDone( this.queryDef = queryDef; this.inputSpecSlicerFactory = inputSpecSlicerFactory; this.closer = closer; - this.queryKernel = new ControllerQueryKernel(queryDef); + this.queryKernel = new ControllerQueryKernel( + queryDef, + workerMemoryParameters.getPartitionStatisticsMaxRetainedBytes(), + isDurableStorageEnabled + ); } /** @@ -2141,6 +2136,7 @@ private Pair> run() throws IOExceptio while (!queryKernel.isDone()) { startStages(); + fetchStatsFromWorkers(); sendPartitionBoundaries(); updateLiveReportMaps(); cleanUpEffectivelyFinishedStages(); @@ -2259,6 +2255,66 @@ private void startTaskLauncher() ); } + /** + * Enqueues the fetching {@link org.apache.druid.msq.statistics.ClusterByStatisticsCollector} + * from each worker via {@link WorkerSketchFetcher} + */ + private void fetchStatsFromWorkers() + { + + for (Map.Entry> stageToWorker : queryKernel.getStagesAndWorkersToFetchClusterStats() + .entrySet()) { + List allTasks = workerTaskLauncher.getActiveTasks(); + Set tasks = stageToWorker.getValue().stream().map(allTasks::get).collect(Collectors.toSet()); + + ClusterStatisticsMergeMode clusterStatisticsMergeMode = stageToStatsMergingMode.get(stageToWorker.getKey() + .getStageNumber()); + switch (clusterStatisticsMergeMode) { + case SEQUENTIAL: + submitSequentialMergeFetchRequests(stageToWorker.getKey(), tasks); + break; + case PARALLEL: + submitParallelMergeRequests(stageToWorker.getKey(), tasks); + break; + default: + throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode); + } + } + } + + private void submitParallelMergeRequests(StageId stageId, Set tasks) + { + + // eagerly change state of workers whose state is being fetched so that we do not keep on queuing fetch requests. + queryKernel.startFetchingStatsFromWorker( + stageId, + tasks.stream().map(MSQTasks::workerFromTaskId).collect(Collectors.toSet()) + ); + workerSketchFetcher.inMemoryFullSketchMerging(ControllerImpl.this::addToKernelManipulationQueue, + stageId, tasks, + ControllerImpl.this::addToRetryQueue + ); + } + + private void submitSequentialMergeFetchRequests(StageId stageId, Set tasks) + { + if (queryKernel.allPartialKeyInformationPresent(stageId)) { + // eagerly change state of workers whose state is being fetched so that we do not keep on queuing fetch requests. + queryKernel.startFetchingStatsFromWorker( + stageId, + tasks.stream() + .map(MSQTasks::workerFromTaskId) + .collect(Collectors.toSet()) + ); + workerSketchFetcher.sequentialTimeChunkMerging( + ControllerImpl.this::addToKernelManipulationQueue, + queryKernel.getCompleteKeyStatisticsInformation(stageId), + stageId, tasks, + ControllerImpl.this::addToRetryQueue + ); + } + } + /** * Start up any stages that are ready to start. */ @@ -2457,6 +2513,31 @@ private void throwKernelExceptionIfNotUnknown() } } + static ClusterStatisticsMergeMode finalizeClusterStatisticsMergeMode( + StageDefinition stageDef, + ClusterStatisticsMergeMode initialMode + ) + { + ClusterStatisticsMergeMode mergeMode = initialMode; + if (initialMode == ClusterStatisticsMergeMode.AUTO) { + ClusterBy clusterBy = stageDef.getClusterBy(); + if (clusterBy.getBucketByCount() == 0) { + // If there is no time clustering, there is no scope for sequential merge + mergeMode = ClusterStatisticsMergeMode.PARALLEL; + } else if (stageDef.getMaxWorkerCount() > WorkerSketchFetcher.WORKER_THRESHOLD) { + mergeMode = ClusterStatisticsMergeMode.SEQUENTIAL; + } else { + mergeMode = ClusterStatisticsMergeMode.PARALLEL; + } + log.info( + "Stage [%d] AUTO mode: chose %s mode to merge key statistics", + stageDef.getStageNumber(), + mergeMode + ); + } + return mergeMode; + } + /** * Interface used by {@link #contactWorkersForStage}. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 74576d9fc879..3c5297e49ebd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -517,7 +517,7 @@ public void postWorkOrder(final WorkOrder workOrder) throw new ISE("Worker number mismatch: expected [%d]", task.getWorkerNumber()); } - // Dont add to queue if workerOrder already present. + // Do not add to queue if workerOrder already present. kernelManipulationQueue.add( kernelHolder -> kernelHolder.getStageKernelMap().putIfAbsent( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java index 3482b50daaff..bcbb899d57a2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -19,28 +19,34 @@ package org.apache.druid.msq.exec; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.frame.key.ClusterBy; -import org.apache.druid.frame.key.ClusterByPartition; -import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.java.util.common.Either; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.function.TriConsumer; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; +import org.apache.druid.msq.indexing.error.MSQFault; +import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernel; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; +import javax.annotation.Nullable; import java.util.Set; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.stream.IntStream; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Supplier; +import java.util.stream.Collectors; /** * Queues up fetching sketches from workers and progressively generates partitions boundaries. @@ -50,291 +56,189 @@ public class WorkerSketchFetcher private static final Logger log = new Logger(WorkerSketchFetcher.class); private static final int DEFAULT_THREAD_COUNT = 4; // If the combined size of worker sketches is more than this threshold, SEQUENTIAL merging mode is used. - static final long BYTES_THRESHOLD = 1_000_000_000L; - // If there are more workers than this threshold, SEQUENTIAL merging mode is used. - static final long WORKER_THRESHOLD = 100; + public static final long WORKER_THRESHOLD = 100; - private final ClusterStatisticsMergeMode clusterStatisticsMergeMode; - private final int statisticsMaxRetainedBytes; private final WorkerClient workerClient; private final ExecutorService executorService; + private final MSQWorkerTaskLauncher workerTaskLauncher; - public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode clusterStatisticsMergeMode, int statisticsMaxRetainedBytes) + private final boolean retryEnabled; + + AtomicReference isError = new AtomicReference<>(); + + public WorkerSketchFetcher( + WorkerClient workerClient, + MSQWorkerTaskLauncher workerTaskLauncher, + boolean retryEnabled + ) { this.workerClient = workerClient; - this.clusterStatisticsMergeMode = clusterStatisticsMergeMode; this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); - this.statisticsMaxRetainedBytes = statisticsMaxRetainedBytes; + this.workerTaskLauncher = workerTaskLauncher; + this.retryEnabled = retryEnabled; } /** * Submits a request to fetch and generate partitions for the given worker statistics and returns a future for it. It * decides based on the statistics if it should fetch sketches one by one or together. + * The future can successfully return a null signaling that partition statistics from all the workers have not been fetched yet. + * */ - public CompletableFuture> submitFetcherTask( - CompleteKeyStatisticsInformation completeKeyStatisticsInformation, - List workerTaskIds, - StageDefinition stageDefinition - ) - { - ClusterBy clusterBy = stageDefinition.getClusterBy(); - switch (clusterStatisticsMergeMode) { - case SEQUENTIAL: - return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); - case PARALLEL: - return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); - case AUTO: - if (clusterBy.getBucketByCount() == 0) { - log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); - // If there is no time clustering, there is no scope for sequential merge - return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); - } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || completeKeyStatisticsInformation.getBytesRetained() > BYTES_THRESHOLD) { - log.debug("Query [%s] AUTO mode: chose SEQUENTIAL mode to merge key statistics", stageDefinition.getId().getQueryId()); - return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); - } - log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); - return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); - default: - throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode); - } - } /** * Fetches the full {@link ClusterByStatisticsCollector} from all workers and generates partition boundaries from them. * This is faster than fetching them timechunk by timechunk but the collector will be downsampled till it can fit * on the controller, resulting in less accurate partition boundries. */ - CompletableFuture> inMemoryFullSketchMerging( - StageDefinition stageDefinition, - List workerTaskIds + public void inMemoryFullSketchMerging( + Consumer> kernelActions, + StageId stageId, + Set taskIds, + TriConsumer retryOperation ) { - CompletableFuture> partitionFuture = new CompletableFuture<>(); - // Create a new key statistics collector to merge worker sketches into - final ClusterByStatisticsCollector mergedStatisticsCollector = - stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); - final int workerCount = workerTaskIds.size(); - // Guarded by synchronized mergedStatisticsCollector - final Set finishedWorkers = new HashSet<>(); + for (String taskId : taskIds) { + try { + int workerNumber = MSQTasks.workerFromTaskId(taskId); + executorService.submit(() -> { + fetchStatsFromWorker( + kernelActions, + () -> workerClient.fetchClusterByStatisticsSnapshot( + taskId, + stageId.getQueryId(), + stageId.getStageNumber() + ), + taskId, + (kernel, snapshot) -> kernel.mergeClusterByStatisticsCollectorForAllTimeChunks( + stageId, + workerNumber, + snapshot + ), + retryOperation + ); + }); + } + catch (RejectedExecutionException rejectedExecutionException) { + if (isError.get() == null) { + throw rejectedExecutionException; + } else { + // throw worker error exception + throw new ISE("Unable to fetch partitions ", isError.get()); + } + } + } + } - // Submit a task for each worker to fetch statistics - IntStream.range(0, workerCount).forEach(workerNo -> { - executorService.submit(() -> { - ListenableFuture snapshotFuture = - workerClient.fetchClusterByStatisticsSnapshot( - workerTaskIds.get(workerNo), - stageDefinition.getId().getQueryId(), - stageDefinition.getStageNumber() - ); - partitionFuture.whenComplete((result, exception) -> { - if (exception != null || (result != null && result.isError())) { - snapshotFuture.cancel(true); - } + private void fetchStatsFromWorker( + Consumer> kernelActions, + Supplier> fetchStatsSupplier, + String taskId, + BiConsumer successKernelOperation, + TriConsumer retryOperation + ) + { + if (isError.get() != null) { + executorService.shutdownNow(); + return; + } + try { + workerTaskLauncher.waitUntilWorkersReady(ImmutableSet.of(MSQTasks.workerFromTaskId(taskId))); + } + catch (InterruptedException interruptedException) { + isError.compareAndSet(null, interruptedException); + executorService.shutdownNow(); + return; + } + + ListenableFuture fetchFuture = fetchStatsSupplier.get(); + + SettableFuture kernelActionFuture = SettableFuture.create(); + + Futures.addCallback(fetchFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable ClusterByStatisticsSnapshot result) + { + kernelActions.accept((queryKernel) -> { + successKernelOperation.accept(queryKernel, result); + // we do not want to have too many key collector sketches in the event queue as that cause memory issues + // blocking the executor service thread until the kernel operation is finished. + // so we would have utmost DEFAULT_THREAD_COUNT number of sketches in the queue. + kernelActionFuture.set(true); }); - try { - ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); - if (clusterByStatisticsSnapshot == null) { - throw new ISE("Worker %s returned null sketch, this should never happen", workerNo); - } - synchronized (mergedStatisticsCollector) { - mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); - finishedWorkers.add(workerNo); + } - if (finishedWorkers.size() == workerCount) { - log.debug("Query [%s] Received all statistics, generating partitions", stageDefinition.getId().getQueryId()); - partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); - } - } - } - catch (Exception e) { - synchronized (mergedStatisticsCollector) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); + @Override + public void onFailure(Throwable t) + { + if (retryEnabled) { + //add to retry queue + kernelActions.accept((kernel) -> { + retryOperation.accept(kernel, MSQTasks.workerFromTaskId(taskId), new WorkerRpcFailedFault(taskId)); + kernelActionFuture.set(false); + }); + kernelActionFuture.set(false); + + } else { + if (isError.compareAndSet(null, t)) { + log.error(t, "Failed while fetching stats from task[%s]", taskId); } + executorService.shutdownNow(); + kernelActionFuture.setException(t); } - }); + } }); - return partitionFuture; + + FutureUtils.getUnchecked(kernelActionFuture, true); } /** * Fetches cluster statistics from all workers and generates partition boundaries from them one time chunk at a time. * This takes longer due to the overhead of fetching sketches, however, this prevents any loss in accuracy from - * downsampling on the controller. + * down sampling on the controller. */ - CompletableFuture> sequentialTimeChunkMerging( + public void sequentialTimeChunkMerging( + Consumer> kernelActions, CompleteKeyStatisticsInformation completeKeyStatisticsInformation, - StageDefinition stageDefinition, - List workerTaskIds + StageId stageId, + Set tasks, + TriConsumer retryOperation ) { - SequentialFetchStage sequentialFetchStage = new SequentialFetchStage( - stageDefinition, - workerTaskIds, - completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().entrySet().iterator() - ); - sequentialFetchStage.submitFetchingTasksForNextTimeChunk(); - return sequentialFetchStage.getPartitionFuture(); - } - - private class SequentialFetchStage - { - private final StageDefinition stageDefinition; - private final List workerTaskIds; - private final Iterator>> timeSegmentVsWorkerIdIterator; - private final CompletableFuture> partitionFuture; - // Final sorted list of partition boundaries. This is appended to after statistics for each time chunk are gathered. - private final List finalPartitionBoundries; - - public SequentialFetchStage( - StageDefinition stageDefinition, - List workerTaskIds, - Iterator>> timeSegmentVsWorkerIdIterator - ) - { - this.finalPartitionBoundries = new ArrayList<>(); - this.stageDefinition = stageDefinition; - this.workerTaskIds = workerTaskIds; - this.timeSegmentVsWorkerIdIterator = timeSegmentVsWorkerIdIterator; - this.partitionFuture = new CompletableFuture<>(); + if (!completeKeyStatisticsInformation.isComplete()) { + throw new ISE("All worker partial key information not received for stage[%d]", stageId.getStageNumber()); } + Set workers = tasks.stream().map(MSQTasks::workerFromTaskId).collect(Collectors.toSet()); + completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().forEach((timeChunk, wks) -> { - /** - * Submits the tasks to fetch key statistics for the time chunk pointed to by {@link #timeSegmentVsWorkerIdIterator}. - * Once the statistics have been gathered from all workers which have them, generates partitions and adds it to - * {@link #finalPartitionBoundries}, stiching the partitions between time chunks using - * {@link #abutAndAppendPartitionBoundries(List, List)} to make them continuous. - * - * The time chunks returned by {@link #timeSegmentVsWorkerIdIterator} should be in ascending order for the partitions - * to be generated correctly. - * - * If {@link #timeSegmentVsWorkerIdIterator} doesn't have any more values, assumes that partition boundaries have - * been successfully generated and completes {@link #partitionFuture} with the result. - * - * Completes the future with an error as soon as the number of partitions exceed max partition count for the stage - * definition. - */ - public void submitFetchingTasksForNextTimeChunk() - { - if (!timeSegmentVsWorkerIdIterator.hasNext()) { - partitionFuture.complete(Either.value(new ClusterByPartitions(finalPartitionBoundries))); - } else { - Map.Entry> entry = timeSegmentVsWorkerIdIterator.next(); - // Time chunk for which partition boundries are going to be generated for - Long timeChunk = entry.getKey(); - Set workerIdsWithTimeChunk = entry.getValue(); - // Create a new key statistics collector to merge worker sketches into - ClusterByStatisticsCollector mergedStatisticsCollector = - stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); - // Guarded by synchronized mergedStatisticsCollector - Set finishedWorkers = new HashSet<>(); - - log.debug("Query [%s]. Submitting request for statistics for time chunk %s to %s workers", - stageDefinition.getId().getQueryId(), - timeChunk, - workerIdsWithTimeChunk.size()); - - // Submits a task for every worker which has a certain time chunk - for (int workerNo : workerIdsWithTimeChunk) { + for (String taskId : tasks) { + int workerNumber = MSQTasks.workerFromTaskId(taskId); + if (workers.contains(workerNumber)) { executorService.submit(() -> { - ListenableFuture snapshotFuture = - workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( - workerTaskIds.get(workerNo), - stageDefinition.getId().getQueryId(), - stageDefinition.getStageNumber(), + fetchStatsFromWorker( + kernelActions, + () -> workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( + taskId, + stageId.getQueryId(), + stageId.getStageNumber(), timeChunk - ); - partitionFuture.whenComplete((result, exception) -> { - if (exception != null || (result != null && result.isError())) { - snapshotFuture.cancel(true); - } - }); - - try { - ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get(); - if (snapshotForTimeChunk == null) { - throw new ISE("Worker %s returned null sketch for %s, this should never happen", workerNo, timeChunk); - } - synchronized (mergedStatisticsCollector) { - mergedStatisticsCollector.addAll(snapshotForTimeChunk); - finishedWorkers.add(workerNo); - - if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) { - Either longClusterByPartitionsEither = - stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector); - - log.debug("Query [%s]. Received all statistics for time chunk %s, generating partitions", - stageDefinition.getId().getQueryId(), - timeChunk); + ), + taskId, + (kernel, snapshot) -> kernel.mergeClusterByStatisticsCollectorForTimeChunk( + stageId, + workerNumber, + timeChunk, + snapshot + ), + retryOperation + ); - long totalPartitionCount = finalPartitionBoundries.size() + getPartitionCountFromEither(longClusterByPartitionsEither); - if (totalPartitionCount > stageDefinition.getMaxPartitionCount()) { - // Fail fast if more partitions than the maximum have been reached. - partitionFuture.complete(Either.error(totalPartitionCount)); - mergedStatisticsCollector.clear(); - } else { - List timeSketchPartitions = longClusterByPartitionsEither.valueOrThrow().ranges(); - abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchPartitions); - log.debug("Query [%s]. Finished generating partitions for time chunk %s, total count so far %s", - stageDefinition.getId().getQueryId(), - timeChunk, - finalPartitionBoundries.size()); - submitFetchingTasksForNextTimeChunk(); - } - } - } - } - catch (Exception e) { - synchronized (mergedStatisticsCollector) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); - } - } }); } } - } - - /** - * Takes a list of sorted {@link ClusterByPartitions} {@param timeSketchPartitions} and adds it to a sorted list - * {@param finalPartitionBoundries}. If {@param finalPartitionBoundries} is not empty, the end time of the last - * partition of {@param finalPartitionBoundries} is changed to abut with the starting time of the first partition - * of {@param timeSketchPartitions}. - * - * This is used to make the partitions generated continuous. - */ - private void abutAndAppendPartitionBoundries( - List finalPartitionBoundries, - List timeSketchPartitions - ) - { - if (!finalPartitionBoundries.isEmpty()) { - // Stitch up the end time of the last partition with the start time of the first partition. - ClusterByPartition clusterByPartition = finalPartitionBoundries.remove(finalPartitionBoundries.size() - 1); - finalPartitionBoundries.add(new ClusterByPartition(clusterByPartition.getStart(), timeSketchPartitions.get(0).getStart())); - } - finalPartitionBoundries.addAll(timeSketchPartitions); - } - - public CompletableFuture> getPartitionFuture() - { - return partitionFuture; - } - } - - /** - * Gets the partition size from an {@link Either}. If it is an error, the long denotes the number of partitions - * (in the case of creating too many partitions), otherwise checks the size of the list. - */ - private static long getPartitionCountFromEither(Either either) - { - if (either.isError()) { - return either.error(); - } else { - return either.valueOrThrow().size(); - } + }); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java index a55f862d5ad7..5fe3518af223 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java @@ -48,24 +48,24 @@ /** * Definition of a stage in a multi-stage {@link QueryDefinition}. - * + *

* Each stage has a list of {@link InputSpec} describing its inputs. The position of each spec within the list is * its "input number". Some inputs are broadcast to all workers (see {@link #getBroadcastInputNumbers()}). Other, * non-broadcast inputs are split up across workers. - * + *

* The number of workers in a stage is at most {@link #getMaxWorkerCount()}. It may be less, depending on the * {@link WorkerAssignmentStrategy} in play and depending on the number of distinct inputs available. (For example: * if there is only one input file, then there can be only one worker.) - * + *

* Each stage has a {@link FrameProcessorFactory} describing the work it does. Output frames written by these * processors have the signature given by {@link #getSignature()}. - * + *

* Each stage has a {@link ShuffleSpec} describing the shuffle that occurs as part of the stage. The shuffle spec is * optional: if none is provided, then the {@link FrameProcessorFactory} directly writes to output partitions. If a * shuffle spec is provided, then the {@link FrameProcessorFactory} is expected to sort each output frame individually * according to {@link ShuffleSpec#getClusterBy()}. The execution system handles the rest, including sorting data across * frames and producing the appropriate output partitions. - * + *

* The rarely-used parameter {@link #getShuffleCheckHasMultipleValues()} controls whether the execution system * checks, while shuffling, if the key used for shuffling has any multi-value fields. When this is true, the method * {@link ClusterByStatisticsCollector#hasMultipleValues} is enabled on collectors @@ -282,11 +282,11 @@ public Either generatePartitionsForShuffle( ) { if (shuffleSpec == null) { - throw new ISE("No shuffle"); + throw new ISE("No shuffle for stage[%d]", getStageNumber()); } else if (mustGatherResultKeyStatistics() && collector == null) { - throw new ISE("Statistics required, but not gathered"); + throw new ISE("Statistics required, but not gathered for stage[%d]", getStageNumber()); } else if (!mustGatherResultKeyStatistics() && collector != null) { - throw new ISE("Statistics gathered, but not required"); + throw new ISE("Statistics gathered, but not required for stage[%d]", getStageNumber()); } else { return shuffleSpec.generatePartitions(collector, MAX_PARTITIONS); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 82a3c6e34960..97939c20828e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -50,6 +50,7 @@ import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; @@ -127,13 +128,23 @@ public class ControllerQueryKernel /** * {@link MSQFault#getErrorCode()} which are retried. */ - private static final Set retriableErrorCodes = ImmutableSet.of(CanceledFault.CODE, UnknownFault.CODE, - WorkerRpcFailedFault.CODE + private static final Set RETRIABLE_ERROR_CODES = ImmutableSet.of( + CanceledFault.CODE, + UnknownFault.CODE, + WorkerRpcFailedFault.CODE ); + private final int maxRetainedPartitionSketchBytes; + private final boolean faultToleranceEnabled; - public ControllerQueryKernel(final QueryDefinition queryDef) + public ControllerQueryKernel( + final QueryDefinition queryDef, + int maxRetainedPartitionSketchBytes, + boolean faultToleranceEnabled + ) { this.queryDef = queryDef; + this.maxRetainedPartitionSketchBytes = maxRetainedPartitionSketchBytes; + this.faultToleranceEnabled = faultToleranceEnabled; this.inflowMap = ImmutableMap.copyOf(computeStageInflowMap(queryDef)); this.outflowMap = ImmutableMap.copyOf(computeStageOutflowMap(queryDef)); @@ -291,7 +302,8 @@ private void createNewKernels( stageDef, stageWorkerCountMap, slicer, - assignmentStrategy + assignmentStrategy, + maxRetainedPartitionSketchBytes ); stageTracker.put(nextStage, stageKernel); } @@ -351,11 +363,11 @@ public ReadablePartitions getResultPartitionsForStage(final StageId stageId) } /** - * Delegates call to {@link ControllerStageTracker#getWorkersToSendParitionBoundaries()} + * Delegates call to {@link ControllerStageTracker#getWorkersToSendPartitionBoundaries()} */ public IntSet getWorkersToSendPartitionBoundaries(final StageId stageId) { - return getStageKernelOrThrow(stageId).getWorkersToSendParitionBoundaries(); + return getStageKernelOrThrow(stageId).getWorkersToSendPartitionBoundaries(); } /** @@ -390,14 +402,6 @@ public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation(fina return getStageKernelOrThrow(stageId).getCompleteKeyStatisticsInformation(); } - /** - * Delegates call to {@link ControllerStageTracker#setClusterByPartitionBoundaries(ClusterByPartitions)} ()} - */ - public void setClusterByPartitionBoundaries(final StageId stageId, ClusterByPartitions clusterByPartitions) - { - getStageKernelOrThrow(stageId).setClusterByPartitionBoundaries(clusterByPartitions); - } - /** * Delegates call to {@link ControllerStageTracker#collectorEncounteredAnyMultiValueField()} */ @@ -458,7 +462,7 @@ public WorkerInputs getWorkerInputsForStage(final StageId stageId) } /** - * Delegates call to {@link ControllerStageTracker#addPartialKeyStatisticsForWorker(int, PartialKeyStatisticsInformation)}. + * Delegates call to {@link ControllerStageTracker#addPartialKeyInformationForWorker(int, PartialKeyStatisticsInformation)}. * If calling this causes transition for the stage kernel, then this gets registered in this query kernel */ public void addPartialKeyStatisticsForStageAndWorker( @@ -468,7 +472,7 @@ public void addPartialKeyStatisticsForStageAndWorker( ) { ControllerStageTracker stageKernel = getStageKernelOrThrow(stageId); - ControllerStagePhase newPhase = stageKernel.addPartialKeyStatisticsForWorker( + ControllerStagePhase newPhase = stageKernel.addPartialKeyInformationForWorker( workerNumber, partialKeyStatisticsInformation ); @@ -552,7 +556,7 @@ private WorkOrder getWorkOrder(int workerNumber, StageId stageId) * Whenever a stage kernel changes its phase, the change must be "registered" by calling this method with the stageId * and the new phase */ - private void transitionStageKernel(StageId stageId, ControllerStagePhase newPhase) + public void transitionStageKernel(StageId stageId, ControllerStagePhase newPhase) { Preconditions.checkArgument( stageTracker.containsKey(stageId), @@ -575,8 +579,13 @@ private void transitionStageKernel(StageId stageId, ControllerStagePhase newPhas } } - // might need to change this if (ControllerStagePhase.isPostReadingPhase(newPhase)) { + + // when fault tolerance is enabled, we cannot delete the input data eagerly as we need the input stage for retry until + // results for the current stage are ready. + if (faultToleranceEnabled && newPhase == ControllerStagePhase.POST_READING) { + return; + } // Once the stage has consumed all the data/input from its dependent stages, we remove it from all the stages // whose input it was dependent on for (StageId inputStage : inflowMap.get(stageId)) { @@ -650,7 +659,7 @@ private static Map> computeStageOutflowMap(final QueryDefi * @param msqFault * @return List of {@link WorkOrder} that needs to be retried. */ - public List getWorkInCaseWorkerEligibileForRetryElseThrow(int workerNumber, MSQFault msqFault) + public List getWorkInCaseWorkerEligibleForRetryElseThrow(int workerNumber, MSQFault msqFault) { final String errorCode; @@ -662,9 +671,8 @@ public List getWorkInCaseWorkerEligibileForRetryElseThrow(int workerN log.info("Parsed out errorCode[%s] to check eligibility for retry", errorCode); - if (retriableErrorCodes.contains(errorCode)) { - return getWorkInCaseWorkerEligibileForRetry(workerNumber); - + if (RETRIABLE_ERROR_CODES.contains(errorCode)) { + return getWorkInCaseWorkerEligibleForRetry(workerNumber); } else { throw new MSQException(msqFault); } @@ -680,7 +688,7 @@ public List getWorkInCaseWorkerEligibileForRetryElseThrow(int workerN * @param worker * @return List of {@link WorkOrder} that needs to be retried. */ - private List getWorkInCaseWorkerEligibileForRetry(int worker) + private List getWorkInCaseWorkerEligibleForRetry(int worker) { List trackedSet = new ArrayList<>(getActiveStages()); trackedSet.removeAll(getEffectivelyFinishedStageIds()); @@ -692,11 +700,81 @@ private List getWorkInCaseWorkerEligibileForRetry(int worker) if (ControllerStagePhase.RETRYING.canTransitionFrom(controllerStageTracker.getPhase()) && controllerStageTracker.retryIfNeeded(worker)) { workOrders.add(getWorkOrder(worker, stageId)); - // should be a no-op. Calling for code patterns. + // should be a no-op. transitionStageKernel(stageId, ControllerStagePhase.RETRYING); } } return workOrders; } + /** + * For each stage, fetches the workers who are ready with their {@link ClusterByStatisticsSnapshot} + */ + public Map> getStagesAndWorkersToFetchClusterStats() + { + List trackedSet = new ArrayList<>(getActiveStages()); + trackedSet.removeAll(getEffectivelyFinishedStageIds()); + + Map> stageToWorkers = new HashMap<>(); + + for (StageId stageId : trackedSet) { + ControllerStageTracker controllerStageTracker = getStageKernelOrThrow(stageId); + stageToWorkers.put(stageId, controllerStageTracker.getWorkersToFetchClusterStatisticsFrom()); + } + return stageToWorkers; + } + + + /** + * Delegates call to {@link ControllerStageTracker#startFetchingStatsFromWorker(int)} for each worker + */ + public void startFetchingStatsFromWorker(StageId stageId, Set workers) + { + ControllerStageTracker controllerStageTracker = getStageKernelOrThrow(stageId); + + for (int worker : workers) { + controllerStageTracker.startFetchingStatsFromWorker(worker); + } + } + + /** + * Delegates call to {@link ControllerStageTracker#mergeClusterByStatisticsCollectorForAllTimeChunks(int, ClusterByStatisticsSnapshot)} + */ + public void mergeClusterByStatisticsCollectorForAllTimeChunks( + StageId stageId, + int workerNumber, + ClusterByStatisticsSnapshot clusterByStatsSnapshot + ) + { + getStageKernelOrThrow(stageId).mergeClusterByStatisticsCollectorForAllTimeChunks( + workerNumber, + clusterByStatsSnapshot + ); + } + + /** + * Delegates call to {@link ControllerStageTracker#mergeClusterByStatisticsCollectorForTimeChunk(int, Long, ClusterByStatisticsSnapshot)} + */ + + public void mergeClusterByStatisticsCollectorForTimeChunk( + StageId stageId, + int workerNumber, + Long timeChunk, + ClusterByStatisticsSnapshot clusterByStatsSnapshot + ) + { + getStageKernelOrThrow(stageId).mergeClusterByStatisticsCollectorForTimeChunk( + workerNumber, + timeChunk, + clusterByStatsSnapshot + ); + } + + /** + * Delegates call to {@link ControllerStageTracker#allPartialKeyInformationFetched()} + */ + public boolean allPartialKeyInformationPresent(StageId stageId) + { + return getStageKernelOrThrow(stageId).allPartialKeyInformationFetched(); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index 06562965e207..3f8f3d19b3f8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java @@ -49,8 +49,11 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) }, // Waiting to fetch key statistics in the background from the workers and incrementally generate partitions. - // This phase is only transitioned to once all partialKeyInformation are recieved from workers. - // Transitioning to this phase should also enqueue the task to fetch key statistics to WorkerSketchFetcher. + // This phase is only transitioned to once all partialKeyInformation are received from workers. + // Transitioning to this phase should also enqueue the task to fetch key statistics if `SEQUENTIAL` strategy is used. + // In `PARALLEL` strategy, we start fetching the key statistics as soon as they are available on the worker. + // This stage is not required in non-pre shuffle contexts + MERGING_STATISTICS { @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) @@ -59,7 +62,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) } }, - // Post the inputs have been read and mapped to frames, in the `POST_READING` stage, we pre-shuffle and determing the partition boundaries. + // Post the inputs have been read and mapped to frames, in the `POST_READING` stage, we pre-shuffle and determining the partition boundaries. // This step for a stage spits out the statistics of the data as a whole (and not just the individual records). This // phase is not required in non-pre shuffle contexts. POST_READING { @@ -105,7 +108,10 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return priorPhase == READING_INPUT || priorPhase == POST_READING || priorPhase == RETRYING; + return priorPhase == READING_INPUT + || priorPhase == POST_READING + || priorPhase == MERGING_STATISTICS + || priorPhase == RETRYING; } }; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index a4a72cb2889f..73041a9037c7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -26,11 +26,14 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.key.ClusterByPartition; import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.indexing.error.InsertTimeNullFault; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; @@ -42,13 +45,19 @@ import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; -import java.util.stream.IntStream; +import java.util.Map; +import java.util.Set; import java.util.TreeMap; +import java.util.stream.IntStream; /** * Controller-side state machine for each stage. Used by {@link ControllerQueryKernel} to form the overall state @@ -60,6 +69,7 @@ class ControllerStageTracker { private static final Logger log = new Logger(ControllerStageTracker.class); + private static final long STATIC_TIME_CHUNK_FOR_PARALLEL_MERGE = Granularities.ALL.bucketStart(-1); private final StageDefinition stageDef; private final int workerCount; @@ -67,9 +77,14 @@ class ControllerStageTracker private final WorkerInputs workerInputs; // worker-> workerStagePhase - private final Int2ObjectMap workerToPhase = new Int2ObjectOpenHashMap<>(); + private final Int2ObjectMap workerToPhase = new Int2ObjectOpenHashMap<>(); + + // workers which have reported partial key information. + private final IntSet workerReportedPartialKeyInformation = new IntAVLTreeSet(); - private final IntSet workersWithResultKeyStatistics = new IntAVLTreeSet(); + // workers from which key collector is fetched. + private final IntSet workersFromWhichKeyCollectorFetched = new IntAVLTreeSet(); + private final int maxRetainedPartitionSketchBytes; private ControllerStagePhase phase = ControllerStagePhase.NEW; @Nullable @@ -83,6 +98,15 @@ class ControllerStageTracker @Nullable private ClusterByPartitions resultPartitionBoundaries; + // created when is called. Should be cleared once resultPartitionBoundariesforTimeChunk is set + private final Map timeChunkToCollector = new HashMap<>(); + private Map timeChunkToBoundaries = new TreeMap<>(); + private final Map> timeChunkToWorkers = new HashMap<>(); + long totalPartitionCount; + + + private Map> workerToRemainingTimeChunks = null; + @Nullable private Object resultObject; @@ -91,12 +115,14 @@ class ControllerStageTracker private ControllerStageTracker( final StageDefinition stageDef, - final WorkerInputs workerInputs + final WorkerInputs workerInputs, + final int maxRetainedPartitionSketchBytes ) { this.stageDef = stageDef; this.workerCount = workerInputs.workerCount(); this.workerInputs = workerInputs; + this.maxRetainedPartitionSketchBytes = maxRetainedPartitionSketchBytes; initializeWorkerState(workerCount); @@ -110,13 +136,14 @@ private ControllerStageTracker( } /** - * Initalized stage for each worker to {@link WorkerStagePhase#NEW} + * Initialize stage for each worker to {@link ControllerWorkerStagePhase#NEW} * * @param workerCount */ private void initializeWorkerState(int workerCount) { - IntStream.range(0, workerCount).forEach(wokerNumber -> workerToPhase.put(wokerNumber, WorkerStagePhase.NEW)); + IntStream.range(0, workerCount) + .forEach(wokerNumber -> workerToPhase.put(wokerNumber, ControllerWorkerStagePhase.NEW)); } /** @@ -128,11 +155,16 @@ static ControllerStageTracker create( final StageDefinition stageDef, final Int2IntMap stageWorkerCountMap, final InputSpecSlicer slicer, - final WorkerAssignmentStrategy assignmentStrategy + final WorkerAssignmentStrategy assignmentStrategy, + final int maxRetainedPartitionSketchBytes ) { final WorkerInputs workerInputs = WorkerInputs.create(stageDef, stageWorkerCountMap, slicer, assignmentStrategy); - return new ControllerStageTracker(stageDef, workerInputs); + return new ControllerStageTracker( + stageDef, + workerInputs, + maxRetainedPartitionSketchBytes + ); } /** @@ -191,14 +223,14 @@ ClusterByPartitions getResultPartitionBoundaries() * * @return */ - IntSet getWorkersToSendParitionBoundaries() + IntSet getWorkersToSendPartitionBoundaries() { if (!getStageDefinition().doesShuffle()) { throw new ISE("Result partition information is not relevant to this stage because it does not shuffle"); } IntAVLTreeSet workers = new IntAVLTreeSet(); for (Integer worker : workerToPhase.keySet()) { - if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.equals(workerToPhase.get(worker))) { + if (ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.equals(workerToPhase.get(worker))) { workers.add(worker); } } @@ -206,7 +238,7 @@ IntSet getWorkersToSendParitionBoundaries() } /** - * Indicates that the work order for worker has been sent. Transistions the state to {@link WorkerStagePhase#READING_INPUT} + * Indicates that the work order for worker has been sent. Transitions the state to {@link ControllerWorkerStagePhase#READING_INPUT} * if no more work orders need to be sent. * * @param worker @@ -218,15 +250,15 @@ void workOrderSentForWorker(int worker) if (state == null) { throw new ISE("Worker[%d] not found for stage[%s]", wk, stageDef.getStageNumber()); } - if (!WorkerStagePhase.READING_INPUT.canTransitionFrom(state)) { + if (!ControllerWorkerStagePhase.READING_INPUT.canTransitionFrom(state)) { throw new ISE( "Worker[%d] cannot transistion from state[%s] to state[%s] while sending work order", worker, state, - WorkerStagePhase.READING_INPUT + ControllerWorkerStagePhase.READING_INPUT ); } - return WorkerStagePhase.READING_INPUT; + return ControllerWorkerStagePhase.READING_INPUT; }); if (phase != ControllerStagePhase.READING_INPUT) { if (allWorkOrdersSent()) { @@ -249,15 +281,15 @@ void partitionBoundariesSentForWorker(int worker) if (state == null) { throw new ISE("Worker[%d] not found for stage[%s]", wk, stageDef.getStageNumber()); } - if (!WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT.canTransitionFrom(state)) { + if (!ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT.canTransitionFrom(state)) { throw new ISE( "Worker[%d] cannot transistion from state[%s] to state[%s] while sending partition boundaries", worker, state, - WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT + ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT ); } - return WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT; + return ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT; }); } @@ -275,7 +307,7 @@ boolean collectorEncounteredAnyMultiValueField() { if (completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); - } else if (workersWithReportedKeyStatistics.size() != workerCount) { + } else if (workerReportedPartialKeyInformation.size() != workerCount) { throw new ISE("Result key statistics are not ready"); } else { return completeKeyStatisticsInformation.hasMultipleValues(); @@ -332,13 +364,13 @@ public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation() } /** - * Adds result key statistics for a particular worker number. If statistics have already been added for this worker, - * then this call ignores the new ones and does nothing. + * Adds partial key statistics information for a particular worker number. If information is already added for this worker, + * then this call ignores the new information. * - * @param workerNumber the worker + * @param workerNumber the worker * @param partialKeyStatisticsInformation partial key statistics */ - ControllerStagePhase addPartialKeyStatisticsForWorker( + ControllerStagePhase addPartialKeyInformationForWorker( final int workerNumber, final PartialKeyStatisticsInformation partialKeyStatisticsInformation ) @@ -346,7 +378,9 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( if (phase != ControllerStagePhase.READING_INPUT) { throw new ISE("Cannot add result key statistics from stage [%s]", phase); } - if (!stageDef.mustGatherResultKeyStatistics() || !stageDef.doesShuffle() || completeKeyStatisticsInformation == null) { + if (!stageDef.mustGatherResultKeyStatistics() + || !stageDef.doesShuffle() + || completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); } @@ -354,40 +388,69 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( throw new IAE("Invalid workerNumber [%s]", workerNumber); } - WorkerStagePhase currentPhase = workerToPhase.get(workerNumber); + ControllerWorkerStagePhase currentPhase = workerToPhase.get(workerNumber); if (currentPhase == null) { throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); } try { - if (WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.canTransitionFrom(currentPhase)) { - workerToPhase.put(workerNumber, WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); + if (ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED.canTransitionFrom(currentPhase)) { + workerToPhase.put(workerNumber, ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED); - // if stats already received for worker, do not update the sketch. - if (workersWithReportedKeyStatistics.add(workerNumber)) { + // if partial key stats already received for worker, do not update the sketch. + if (workerReportedPartialKeyInformation.add(workerNumber)) { if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) { // Time should not contain null value failForReason(InsertTimeNullFault.instance()); return getPhase(); } completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation); + } + + if (resultPartitions != null) { + // we already have result partitions. No need to fetch the stats from worker + // can happen in case of worker retry + workerToPhase.put( + workerNumber, + ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES + ); + } + + if (workersFromWhichKeyCollectorFetched.contains(workerNumber)) { + // we already have fetched the key collector from this worker. No need to fetch it again. + // can happen in case of worker retry + workerToPhase.put( + workerNumber, + ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES + ); } - if (allPartitionStatisticsPresent()) { + + if (allPartialKeyInformationFetched()) { + completeKeyStatisticsInformation.complete(); + if (workerToRemainingTimeChunks == null) { + initializeWorkerToRemainingTimeChunks(); + } // All workers have sent the partial key statistics information. // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers. if (phase != ControllerStagePhase.FAILED) { transitionTo(ControllerStagePhase.MERGING_STATISTICS); } + // if all the results have been fetched, we can straight way transition to post reading. + if (allResultsStatsFetched()) { + if (phase != ControllerStagePhase.FAILED) { + transitionTo(ControllerStagePhase.POST_READING); + } + } } } else { throw new ISE( "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", workerNumber, (stageDef.getStageNumber()), - WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES, + ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED, currentPhase ); @@ -401,6 +464,253 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( return getPhase(); } + private void initializeWorkerToRemainingTimeChunks() + { + workerToRemainingTimeChunks = new HashMap<>(); + completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().forEach((timeChunk, workers) -> { + for (int worker : workers) { + this.workerToRemainingTimeChunks.compute(worker, (wk, timeChunks) -> { + if (timeChunks == null) { + timeChunks = new HashSet<>(); + } + timeChunks.add(timeChunk); + return timeChunks; + }); + } + }); + } + + + /** + * Merges the {@link ClusterByStatisticsSnapshot} for the worker, time chunk with the stage {@link ClusterByStatisticsCollector} being + * tracked at {@link #timeChunkToCollector} for the same time chunk. This method is called when + * {@link ClusterStatisticsMergeMode#SEQUENTIAL} is chosen eventually. + *

+ *

+ * If all the stats from the worker are merged, we transition the worker to {@link ControllerWorkerStagePhase#PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES}; + *

+ * If all the stats from all the workers are merged, we transition the stage to {@link ControllerStagePhase#POST_READING} + */ + + void mergeClusterByStatisticsCollectorForTimeChunk( + int workerNumber, + Long timeChunk, + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot + ) + { + if (!stageDef.mustGatherResultKeyStatistics() + || !stageDef.doesShuffle()) { + throw new ISE("Stage does not gather result key statistics"); + } + + if (workerNumber < 0 || workerNumber >= workerCount) { + throw new IAE("Invalid workerNumber [%s]", workerNumber); + } + + if (completeKeyStatisticsInformation == null || !completeKeyStatisticsInformation.isComplete()) { + throw new ISE( + "Cannot merge worker[%d] time chunk until all the key information is received for stage[%d]", + workerNumber, + stageDef.getStageNumber() + ); + } + + ControllerWorkerStagePhase workerStagePhase = workerToPhase.get(workerNumber); + + if (workerStagePhase == null) { + throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); + } + + // only merge in case this worker has remaining timechunks + workerToRemainingTimeChunks.computeIfPresent(workerNumber, (wk, timeChunks) -> { + if (timeChunks.remove(timeChunk)) { + + // merge the key collector + timeChunkToCollector.compute( + timeChunk, + (ignored, collector) -> { + if (collector == null) { + collector = stageDef.createResultKeyStatisticsCollector(maxRetainedPartitionSketchBytes); + } + collector.addAll(clusterByStatisticsSnapshot); + return collector; + } + ); + + // if work for one time chunk is finished, generate the ClusterByPartitions for that timeChunk and clear the collector so that we free up controller memory. + timeChunkToWorkers.compute(timeChunk, (tc, workers) -> { + if (workers == null || workers.isEmpty()) { + throw new ISE("Workers should not be empty until all the work is finished for time chunk[%d]", timeChunk); + } + workers.remove(workerNumber); + if (workers.isEmpty()) { + ClusterByStatisticsCollector collector = timeChunkToCollector.get(tc); + Either countOrPartitions = stageDef.generatePartitionsForShuffle(collector); + totalPartitionCount += getPartitionCountFromEither(countOrPartitions); + if (totalPartitionCount > stageDef.getMaxPartitionCount()) { + failForReason(new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); + return null; + } + timeChunkToBoundaries.put(tc, countOrPartitions.valueOrThrow()); + collector.clear(); + timeChunkToCollector.remove(tc); + return null; + } + return workers; + }); + } + return timeChunks.isEmpty() ? null : timeChunks; + }); + + + // if all time chunks for worker are taken care off transition worker. + if (workerToRemainingTimeChunks.get(workerNumber) == null) { + // adding worker to a set so that we do not fetch the worker collectors again. + workersFromWhichKeyCollectorFetched.add(workerNumber); + if (ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.canTransitionFrom( + workerStagePhase)) { + workerToPhase.put(workerNumber, ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); + } else { + throw new ISE( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + workerNumber, + (stageDef.getStageNumber()), + ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES, + workerStagePhase + + ); + } + } + + + // if all time chunks have the partition boundaries, merge them to set resultPartitionBoundaries + if (workerToRemainingTimeChunks.isEmpty()) { + if (resultPartitionBoundaries == null) { + timeChunkToBoundaries.forEach((ignored, partitions) -> { + if (resultPartitionBoundaries == null) { + resultPartitionBoundaries = partitions; + } else { + abutAndAppendPartitionBoundaries(resultPartitionBoundaries.ranges(), partitions.ranges()); + } + }); + timeChunkToBoundaries.clear(); + setClusterByPartitionBoundaries(resultPartitionBoundaries); + } else { + // we already have result partitions. We can safely transition to POST READING and submit the result boundaries to the workers. + transitionTo(ControllerStagePhase.POST_READING); + } + } + + } + + /** + * Merges the entire {@link ClusterByStatisticsSnapshot} for the worker with the stage {@link ClusterByStatisticsCollector} being + * tracked at {@link #timeChunkToCollector} with key {@link ControllerStageTracker#STATIC_TIME_CHUNK_FOR_PARALLEL_MERGE}. This method is called when + * {@link ClusterStatisticsMergeMode#PARALLEL} is chosen eventually. + *

+ *

+ * If all the stats from the worker are merged, we transition the worker to {@link ControllerWorkerStagePhase#PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES}. + *

+ * If all the stats from all the workers are merged, we transition the stage to {@link ControllerStagePhase#POST_READING}. + */ + + void mergeClusterByStatisticsCollectorForAllTimeChunks( + int workerNumber, + ClusterByStatisticsSnapshot clusterByStatsSnapshot + ) + { + if (!stageDef.mustGatherResultKeyStatistics() + || !stageDef.doesShuffle()) { + throw new ISE("Stage does not gather result key statistics"); + } + + if (workerNumber < 0 || workerNumber >= workerCount) { + throw new IAE("Invalid workerNumber [%s]", workerNumber); + } + + ControllerWorkerStagePhase workerStagePhase = workerToPhase.get(workerNumber); + + if (workerStagePhase == null) { + throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); + } + + + // To prevent the case where we do not fetch the collector twice, like when worker is retried, we should be okay with the + // older collector from the previous run of the worker. + + if (workersFromWhichKeyCollectorFetched.add(workerNumber)) { + // in case of parallel merge we use the "ALL" granularity start time to put the sketches + timeChunkToCollector.compute( + STATIC_TIME_CHUNK_FOR_PARALLEL_MERGE, + (timeChunk, stats) -> { + if (stats == null) { + stats = stageDef.createResultKeyStatisticsCollector(maxRetainedPartitionSketchBytes); + } + stats.addAll(clusterByStatsSnapshot); + return stats; + } + ); + } else { + log.debug("Already have key collector for worker[%d] stage[%d]", workerNumber, stageDef.getStageNumber()); + } + + if (ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.canTransitionFrom(workerStagePhase)) { + workerToPhase.put(workerNumber, ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); + } else { + throw new ISE( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + workerNumber, + (stageDef.getStageNumber()), + ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES, + workerStagePhase + + ); + } + + if (allResultsStatsFetched()) { + if (completeKeyStatisticsInformation == null || !completeKeyStatisticsInformation.isComplete()) { + throw new ISE( + "Cannot generate partition boundaries until all the key information is received for stage[%d]", + workerNumber, + stageDef.getStageNumber() + ); + } + if (resultPartitions == null) { + Either countOrPartitions = stageDef.generatePartitionsForShuffle(timeChunkToCollector.get( + STATIC_TIME_CHUNK_FOR_PARALLEL_MERGE)); + totalPartitionCount += getPartitionCountFromEither(countOrPartitions); + if (totalPartitionCount > stageDef.getMaxPartitionCount()) { + failForReason(new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); + return; + } + resultPartitionBoundaries = countOrPartitions.valueOrThrow(); + setClusterByPartitionBoundaries(resultPartitionBoundaries); + } else { + log.debug("Already have result partitions for stage[%d]", stageDef.getStageNumber()); + } + timeChunkToCollector.computeIfPresent( + STATIC_TIME_CHUNK_FOR_PARALLEL_MERGE, + (key, collector) -> collector.clear() + ); + timeChunkToCollector.clear(); + + } + } + + /** + * Returns true if all {@link ClusterByStatisticsSnapshot} are fetched from each worker else false. + */ + private boolean allResultsStatsFetched() + { + return workerToPhase.values().stream() + .filter(stagePhase -> stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) + || stagePhase.equals(ControllerWorkerStagePhase.RESULTS_READY)) + .count() + == workerCount; + } + + /** * Sets the {@link #resultPartitions} and {@link #resultPartitionBoundaries} and transitions the phase to POST_READING. */ @@ -415,7 +725,7 @@ void setClusterByPartitionBoundaries(ClusterByPartitions clusterByPartitions) } if (!ControllerStagePhase.MERGING_STATISTICS.equals(getPhase())) { - throw new ISE("Cannot set partition boundires from key statistics from stage [%s]", getPhase()); + throw new ISE("Cannot set partition boundaries from key statistics from stage [%s]", getPhase()); } this.resultPartitionBoundaries = clusterByPartitions; @@ -444,23 +754,23 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO throw new NullPointerException("resultObject must not be null"); } - WorkerStagePhase currentPhase = workerToPhase.get(workerNumber); + ControllerWorkerStagePhase currentPhase = workerToPhase.get(workerNumber); if (currentPhase == null) { throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); } - if (WorkerStagePhase.RESULTS_READY.canTransitionFrom(currentPhase)) { + if (ControllerWorkerStagePhase.RESULTS_READY.canTransitionFrom(currentPhase)) { - if (stageDef.mustGatherResultKeyStatistics() && currentPhase == WorkerStagePhase.READING_INPUT) { + if (stageDef.mustGatherResultKeyStatistics() && currentPhase == ControllerWorkerStagePhase.READING_INPUT) { throw new ISE( "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", workerNumber, (stageDef.getStageNumber()), - WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT, + ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT, currentPhase ); } - workerToPhase.put(workerNumber, WorkerStagePhase.RESULTS_READY); + workerToPhase.put(workerNumber, ControllerWorkerStagePhase.RESULTS_READY); if (this.resultObject == null) { this.resultObject = resultObject; } else { @@ -474,8 +784,8 @@ boolean setResultsCompleteForWorker(final int workerNumber, final Object resultO workerNumber, (stageDef.getStageNumber()), stageDef.mustGatherResultKeyStatistics() - ? WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT - : WorkerStagePhase.READING_INPUT, + ? ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT + : ControllerWorkerStagePhase.READING_INPUT, currentPhase ); @@ -492,7 +802,7 @@ private boolean allResultsPresent() { return workerToPhase.values() .stream() - .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.RESULTS_READY)) + .filter(stagePhase -> stagePhase.equals(ControllerWorkerStagePhase.RESULTS_READY)) .count() == workerCount; } @@ -518,7 +828,7 @@ void fail() /** * Sets {@link #resultPartitions} (always) and {@link #resultPartitionBoundaries} without using key statistics. - * + *

* If {@link StageDefinition#mustGatherResultKeyStatistics()} is true, this method should not be called. */ private void generateResultPartitionsAndBoundariesWithoutKeyStatistics() @@ -533,7 +843,7 @@ private void generateResultPartitionsAndBoundariesWithoutKeyStatistics() final int stageNumber = stageDef.getStageNumber(); if (stageDef.doesShuffle()) { - if (stageDef.mustGatherResultKeyStatistics() && !allPartitionStatisticsPresent()) { + if (stageDef.mustGatherResultKeyStatistics() && !allPartialKeyInformationFetched()) { throw new ISE("Cannot generate result partitions without all worker key statistics"); } @@ -573,34 +883,39 @@ private void generateResultPartitionsAndBoundariesWithoutKeyStatistics() } /** - * True if all partitions stats are present for a shuffling stage which require statistics, else false. + * True if all {@link PartialKeyStatisticsInformation} are present for a shuffling stage which require statistics, else false. * If the stage does not gather result statistics, we return a true. */ - private boolean allPartitionStatisticsPresent() + public boolean allPartialKeyInformationFetched() { if (!stageDef.mustGatherResultKeyStatistics()) { return true; } return workerToPhase.values() .stream() - .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) - || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) - || stagePhase.equals(WorkerStagePhase.RESULTS_READY)) + .filter(stagePhase -> stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_FETCHING_ALL_KEY_STATS) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) + || stagePhase.equals(ControllerWorkerStagePhase.RESULTS_READY)) .count() == workerCount; } /** - * True if all work orders are sent else false. + * True if all {@link org.apache.druid.msq.kernel.WorkOrder} are sent else false. */ private boolean allWorkOrdersSent() { return workerToPhase.values() .stream() - .filter(stagePhase -> stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) - || stagePhase.equals(WorkerStagePhase.READING_INPUT) - || stagePhase.equals(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) - || stagePhase.equals(WorkerStagePhase.RESULTS_READY) + .filter(stagePhase -> + stagePhase.equals(ControllerWorkerStagePhase.READING_INPUT) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_FETCHING_ALL_KEY_STATS) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES) + || stagePhase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT) + || stagePhase.equals(ControllerWorkerStagePhase.RESULTS_READY) ) .count() == workerCount; @@ -643,12 +958,12 @@ public boolean retryIfNeeded(int workerNumber) return false; } - if (workerToPhase.get(workerNumber).equals(WorkerStagePhase.RESULTS_READY) - || workerToPhase.get(workerNumber).equals(WorkerStagePhase.FINISHED)) { + if (workerToPhase.get(workerNumber).equals(ControllerWorkerStagePhase.RESULTS_READY) + || workerToPhase.get(workerNumber).equals(ControllerWorkerStagePhase.FINISHED)) { // do nothing return false; } - workerToPhase.put(workerNumber, WorkerStagePhase.NEW); + workerToPhase.put(workerNumber, ControllerWorkerStagePhase.NEW); transitionTo(ControllerStagePhase.RETRYING); return true; } @@ -659,4 +974,74 @@ private boolean isTrackingWorker(int workerNumber) return workerToPhase.get(workerNumber) != null; } + /** + * Returns the workers who are ready with {@link ClusterByStatisticsSnapshot} + */ + public Set getWorkersToFetchClusterStatisticsFrom() + { + Set workersToFetchStats = new HashSet<>(); + workerToPhase.forEach((worker, phase) -> { + if (phase.equals(ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED)) { + workersToFetchStats.add(worker); + } + }); + return workersToFetchStats; + } + + /** + * Transitions the worker to {@link ControllerWorkerStagePhase#PRESHUFFLE_FETCHING_ALL_KEY_STATS) indicating fetching has begun. + */ + public void startFetchingStatsFromWorker(int worker) + { + ControllerWorkerStagePhase workerStagePhase = workerToPhase.get(worker); + if (ControllerWorkerStagePhase.PRESHUFFLE_FETCHING_ALL_KEY_STATS.canTransitionFrom(workerStagePhase)) { + workerToPhase.put(worker, ControllerWorkerStagePhase.PRESHUFFLE_FETCHING_ALL_KEY_STATS); + } else { + throw new ISE( + "Worker[%d] for stage[%d] expected to be in state[%s]. Found state[%s]", + worker, + (stageDef.getStageNumber()), + ControllerWorkerStagePhase.PRESHUFFLE_FETCHING_ALL_KEY_STATS, + workerStagePhase + + ); + } + } + + /** + * Takes a list of sorted {@link ClusterByPartitions} {@param timeSketchPartitions} and adds it to a sorted list + * {@param finalPartitionBoundaries}. If {@param finalPartitionBoundaries} is not empty, the end time of the last + * partition of {@param finalPartitionBoundaries} is changed to abut with the starting time of the first partition + * of {@param timeSketchPartitions}. + *

+ * This is used to make the partitions generated continuous. + */ + private void abutAndAppendPartitionBoundaries( + List finalPartitionBoundaries, + List timeSketchPartitions + ) + { + if (!finalPartitionBoundaries.isEmpty()) { + // Stitch up the end time of the last partition with the start time of the first partition. + ClusterByPartition clusterByPartition = finalPartitionBoundaries.remove(finalPartitionBoundaries.size() - 1); + finalPartitionBoundaries.add(new ClusterByPartition( + clusterByPartition.getStart(), + timeSketchPartitions.get(0).getStart() + )); + } + finalPartitionBoundaries.addAll(timeSketchPartitions); + } + + /** + * Gets the partition size from an {@link Either}. If it is an error, the long denotes the number of partitions + * (in the case of creating too many partitions), otherwise checks the size of the list. + */ + private static long getPartitionCountFromEither(Either either) + { + if (either.isError()) { + return either.error(); + } else { + return either.valueOrThrow().size(); + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerWorkerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerWorkerStagePhase.java new file mode 100644 index 000000000000..1c3e370dc80e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerWorkerStagePhase.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.kernel.controller; + +/** + * Worker phases that a stage can be in being tracked by the controller. + *

+ * Used by {@link ControllerStageTracker}. + */ +public enum ControllerWorkerStagePhase +{ + NEW { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return false; + } + }, + + READING_INPUT { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return priorPhase == NEW; + } + }, + PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return priorPhase == READING_INPUT; + } + }, + + PRESHUFFLE_FETCHING_ALL_KEY_STATS { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return priorPhase == PRESHUFFLE_WAITING_FOR_ALL_KEY_STATS_TO_BE_FETCHED; + } + }, + + PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return priorPhase == PRESHUFFLE_FETCHING_ALL_KEY_STATS; + } + }, + + PRESHUFFLE_WRITING_OUTPUT { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES; + } + }, + + RESULTS_READY { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return priorPhase == READING_INPUT || priorPhase == PRESHUFFLE_WRITING_OUTPUT; + } + }, + + FINISHED { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return priorPhase == RESULTS_READY; + } + }, + + // Something went wrong. + FAILED { + @Override + public boolean canTransitionFrom(final ControllerWorkerStagePhase priorPhase) + { + return true; + } + }; + + public abstract boolean canTransitionFrom(ControllerWorkerStagePhase priorPhase); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java index 8a915fc8cd8e..6af0b8292a50 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java @@ -19,7 +19,7 @@ package org.apache.druid.msq.statistics; -import com.google.common.collect.ImmutableSortedMap; +import org.apache.druid.java.util.common.ISE; import java.util.HashSet; import java.util.Set; @@ -36,6 +36,8 @@ public class CompleteKeyStatisticsInformation private double bytesRetained; + private boolean complete; + public CompleteKeyStatisticsInformation( final SortedMap> timeChunks, boolean multipleValues, @@ -53,9 +55,13 @@ public CompleteKeyStatisticsInformation( * {@param partialKeyStatisticsInformation}, {@link #multipleValues} is set to true if * {@param partialKeyStatisticsInformation} contains multipleValues and the bytes retained by the partial sketch * is added to {@link #bytesRetained}. + * This method should not be called after {@link CompleteKeyStatisticsInformation#complete()} is called. */ public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation) { + if (complete) { + throw new ISE("Key stats for all workers have been received. This method should not be called."); + } for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) { this.timeSegmentVsWorkerMap .computeIfAbsent(timeSegment, key -> new HashSet<>()) @@ -67,7 +73,10 @@ public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInform public SortedMap> getTimeSegmentVsWorkerMap() { - return ImmutableSortedMap.copyOfSorted(timeSegmentVsWorkerMap); + if (!complete) { + throw new ISE("Key stats for all the workers have not been received. This method cant be called yet."); + } + return timeSegmentVsWorkerMap; } public boolean hasMultipleValues() @@ -79,4 +88,17 @@ public double getBytesRetained() { return bytesRetained; } + + /** + * Does not allow update via {@link CompleteKeyStatisticsInformation#mergePartialInformation(int, PartialKeyStatisticsInformation)} once this method is called. + */ + public void complete() + { + complete = true; + } + + public boolean isComplete() + { + return complete; + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 7c589f2326f1..3dc622870f43 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -60,7 +60,7 @@ public class MultiStageQueryContext public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode"; - public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.AUTO.toString(); + public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.PARALLEL.toString(); private static final boolean DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = false; public static final String CTX_DESTINATION = "destination"; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java index b7b0c4b825fa..c6a657d97a00 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java @@ -19,21 +19,47 @@ package org.apache.druid.msq.exec; +import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault; import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; import org.easymock.EasyMock; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; import java.io.IOException; import java.util.Collections; +import static org.mockito.Mockito.doReturn; + public class ControllerImplTest { + + @Mock + private StageDefinition stageDefinition; + @Mock + private ClusterBy clusterBy; + private AutoCloseable mocks; + + + @Before + public void setUp() + { + mocks = MockitoAnnotations.openMocks(this); + doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); + doReturn(clusterBy).when(stageDefinition).getClusterBy(); + + } + @Test public void test_performSegmentPublish_ok() throws IOException { @@ -101,4 +127,84 @@ public void test_performSegmentPublish_publishLockPreemptedException() throws IO Assert.assertEquals(InsertLockPreemptedFault.instance(), e.getFault()); } + + + @Test + public void test_belowThresholds_ShouldBeParallel() + { + // Cluster by bucket count not 0 + doReturn(1).when(clusterBy).getBucketByCount(); + + // Worker count below threshold + doReturn(1).when(stageDefinition).getMaxWorkerCount(); + + Assert.assertEquals( + ClusterStatisticsMergeMode.PARALLEL, + ControllerImpl.finalizeClusterStatisticsMergeMode( + stageDefinition, + ClusterStatisticsMergeMode.AUTO + ) + ); + } + + + @Test + public void test_noClusterByColumns_shouldBeParallel() + { + + // Cluster by bucket count 0 + doReturn(ClusterBy.none()).when(stageDefinition).getClusterBy(); + + // Worker count above threshold + doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); + + Assert.assertEquals( + ClusterStatisticsMergeMode.PARALLEL, + ControllerImpl.finalizeClusterStatisticsMergeMode( + stageDefinition, + ClusterStatisticsMergeMode.AUTO + ) + ); + + } + + @Test + public void test_wokersAboveThreshold_shouldBeSequential() + { + // Cluster by bucket count not 0 + doReturn(1).when(clusterBy).getBucketByCount(); + + // Worker count above threshold + doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); + + Assert.assertEquals( + ClusterStatisticsMergeMode.SEQUENTIAL, + ControllerImpl.finalizeClusterStatisticsMergeMode( + stageDefinition, + ClusterStatisticsMergeMode.AUTO + ) + ); + + } + + @Test + public void test_mode_should_not_change() + { + + Assert.assertEquals( + ClusterStatisticsMergeMode.SEQUENTIAL, + ControllerImpl.finalizeClusterStatisticsMergeMode(null, ClusterStatisticsMergeMode.SEQUENTIAL) + ); + Assert.assertEquals( + ClusterStatisticsMergeMode.PARALLEL, + ControllerImpl.finalizeClusterStatisticsMergeMode(null, ClusterStatisticsMergeMode.PARALLEL) + ); + } + + + @After + public void tearDown() throws Exception + { + mocks.close(); + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index dd9bf05d2094..f3b88a741d47 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -996,7 +996,7 @@ public void testGroupByWithMultiValue() public void testGroupByWithMultiValueWithoutGroupByEnable() { Map context = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE) .put("groupByEnableMultiValueUnnesting", false) .build(); @@ -1140,7 +1140,7 @@ public void testGroupByArrayWithMultiValueMvToArray() public void testGroupByWithMultiValueMvToArrayWithoutGroupByEnable() { Map context = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE) .put("groupByEnableMultiValueUnnesting", false) .build(); @@ -1258,7 +1258,7 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio @Test public void testMultiValueStringWithIncorrectType() throws IOException { - final File toRead = getResourceAsTemporaryFile("/unparseable-mv-string-array.json"); + final File toRead = MSQTestFileUtils.getResourceAsTemporaryFile(this, "/unparseable-mv-string-array.json"); final String toReadAsJson = queryFramework().queryJsonMapper().writeValueAsString(toRead.getAbsolutePath()); RowSignature rowSignature = RowSignature.builder() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java deleted file mode 100644 index 42f6f0437f59..000000000000 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.msq.exec; - -import org.apache.druid.frame.key.ClusterBy; -import org.apache.druid.msq.kernel.StageDefinition; -import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -import java.util.Collections; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -public class WorkerSketchFetcherAutoModeTest -{ - @Mock - private CompleteKeyStatisticsInformation completeKeyStatisticsInformation; - @Mock - private StageDefinition stageDefinition; - @Mock - private ClusterBy clusterBy; - private AutoCloseable mocks; - private WorkerSketchFetcher target; - - @Before - public void setUp() - { - mocks = MockitoAnnotations.openMocks(this); - - target = spy(new WorkerSketchFetcher(mock(WorkerClient.class), ClusterStatisticsMergeMode.AUTO, 300_000_000)); - // Don't actually try to fetch sketches - doReturn(null).when(target).inMemoryFullSketchMerging(any(), any()); - doReturn(null).when(target).sequentialTimeChunkMerging(any(), any(), any()); - - doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); - doReturn(clusterBy).when(stageDefinition).getClusterBy(); - } - - @After - public void tearDown() throws Exception - { - mocks.close(); - } - - @Test - public void test_submitFetcherTask_belowThresholds_ShouldBeParallel() - { - // Bytes below threshold - doReturn(10.0).when(completeKeyStatisticsInformation).getBytesRetained(); - - // Cluster by bucket count not 0 - doReturn(1).when(clusterBy).getBucketByCount(); - - // Worker count below threshold - doReturn(1).when(stageDefinition).getMaxWorkerCount(); - - target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); - verify(target, times(1)).inMemoryFullSketchMerging(any(), any()); - verify(target, times(0)).sequentialTimeChunkMerging(any(), any(), any()); - } - - @Test - public void test_submitFetcherTask_workerCountAboveThreshold_shouldBeSequential() - { - // Bytes below threshold - doReturn(10.0).when(completeKeyStatisticsInformation).getBytesRetained(); - - // Cluster by bucket count not 0 - doReturn(1).when(clusterBy).getBucketByCount(); - - // Worker count below threshold - doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); - - target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); - verify(target, times(0)).inMemoryFullSketchMerging(any(), any()); - verify(target, times(1)).sequentialTimeChunkMerging(any(), any(), any()); - } - - @Test - public void test_submitFetcherTask_noClusterByColumns_shouldBeParallel() - { - // Bytes above threshold - doReturn(WorkerSketchFetcher.BYTES_THRESHOLD + 10.0).when(completeKeyStatisticsInformation).getBytesRetained(); - - // Cluster by bucket count 0 - doReturn(ClusterBy.none()).when(stageDefinition).getClusterBy(); - - // Worker count above threshold - doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); - - target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); - verify(target, times(1)).inMemoryFullSketchMerging(any(), any()); - verify(target, times(0)).sequentialTimeChunkMerging(any(), any(), any()); - } - - @Test - public void test_submitFetcherTask_bytesRetainedAboveThreshold_shouldBeSequential() - { - // Bytes above threshold - doReturn(WorkerSketchFetcher.BYTES_THRESHOLD + 10.0).when(completeKeyStatisticsInformation).getBytesRetained(); - - // Cluster by bucket count not 0 - doReturn(1).when(clusterBy).getBucketByCount(); - - // Worker count below threshold - doReturn(1).when(stageDefinition).getMaxWorkerCount(); - - target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); - verify(target, times(0)).inMemoryFullSketchMerging(any(), any()); - verify(target, times(1)).sequentialTimeChunkMerging(any(), any(), any()); - } -} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java index 54c9a792e558..af61ac322a64 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -29,6 +29,7 @@ import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.key.RowKey; import org.apache.druid.java.util.common.Either; +import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; @@ -39,18 +40,16 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mock; +import org.mockito.Mockito; import org.mockito.MockitoAnnotations; import java.util.List; import java.util.Queue; import java.util.Set; import java.util.SortedMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutionException; import static org.easymock.EasyMock.mock; import static org.mockito.ArgumentMatchers.any; @@ -117,7 +116,11 @@ public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCan final List workerIds = ImmutableList.of("0", "1", "2", "3"); final CountDownLatch latch = new CountDownLatch(workerIds.size()); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); + target = spy(new WorkerSketchFetcher( + Mockito.mock(WorkerClient.class), + Mockito.mock(MSQWorkerTaskLauncher.class), + true + )); // When fetching snapshots, return a mock and add future to queue doAnswer(invocation -> { @@ -136,17 +139,17 @@ public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCan return Futures.immediateFailedFuture(new InterruptedException("interrupted")); }).when(workerClient).fetchClusterByStatisticsSnapshot(eq("2"), any(), anyInt()); - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - workerIds, - stageDefinition - ); +// CompletableFuture> eitherCompletableFuture = target.sequentialTimeChunkMerging( +// completeKeyStatisticsInformation, +// workerIds, +// stageDefinition +// ); // Assert that the final result is failed and all other task futures are also cancelled. - Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + //Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); Thread.sleep(1000); - Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); + //Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); // Verify that the statistics collector was cleared due to the error. verify(mergedClusterByStatisticsCollector1, times(1)).clear(); // Verify that other task futures were requested to be cancelled. @@ -158,14 +161,18 @@ public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCan @Test public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() - throws ExecutionException, InterruptedException + throws InterruptedException { // Store snapshots in a queue final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); final List workerIds = ImmutableList.of("0", "1", "2", "3", "4"); final CountDownLatch latch = new CountDownLatch(workerIds.size()); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); + target = spy(new WorkerSketchFetcher( + Mockito.mock(WorkerClient.class), + Mockito.mock(MSQWorkerTaskLauncher.class), + true + )); // When fetching snapshots, return a mock and add it to queue doAnswer(invocation -> { @@ -175,23 +182,23 @@ public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() return Futures.immediateFuture(snapshot); }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - workerIds, - stageDefinition - ); +// CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( +// completeKeyStatisticsInformation, +// workerIds, +// stageDefinition +// ); // Assert that the final result is complete and all other sketches returned have been merged. - eitherCompletableFuture.join(); + // eitherCompletableFuture.join(); Thread.sleep(1000); - Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); + // Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); Assert.assertFalse(snapshotQueue.isEmpty()); // Verify that all statistics were added to controller. for (ClusterByStatisticsSnapshot snapshot : snapshotQueue) { verify(mergedClusterByStatisticsCollector1, times(1)).addAll(eq(snapshot)); } // Check that the partitions returned by the merged collector is returned by the final future. - Assert.assertEquals(expectedPartitions1, eitherCompletableFuture.get().valueOrThrow()); + // Assert.assertEquals(expectedPartitions1, eitherCompletableFuture.get().valueOrThrow()); } @Test @@ -200,11 +207,20 @@ public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldC // Store futures in a queue final Queue> futureQueue = new ConcurrentLinkedQueue<>(); - SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of(1L, ImmutableSet.of(0, 1, 2), 2L, ImmutableSet.of(0, 1, 4)); + SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of( + 1L, + ImmutableSet.of(0, 1, 2), + 2L, + ImmutableSet.of(0, 1, 4) + ); doReturn(timeSegmentVsWorkerMap).when(completeKeyStatisticsInformation).getTimeSegmentVsWorkerMap(); final CyclicBarrier barrier = new CyclicBarrier(3); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); + target = spy(new WorkerSketchFetcher( + Mockito.mock(WorkerClient.class), + Mockito.mock(MSQWorkerTaskLauncher.class), + true + )); // When fetching snapshots, return a mock and add future to queue doAnswer(invocation -> { @@ -221,17 +237,17 @@ public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldC return Futures.immediateFailedFuture(new InterruptedException("interrupted")); }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(eq("4"), any(), anyInt(), eq(2L)); - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - ImmutableList.of("0", "1", "2", "3", "4"), - stageDefinition - ); +// CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( +// completeKeyStatisticsInformation, +// ImmutableList.of("0", "1", "2", "3", "4"), +// stageDefinition +// ); // Assert that the final result is failed and all other task futures are also cancelled. - Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + //Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); Thread.sleep(1000); - Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); + //Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); // Verify that the correct statistics collector was cleared due to the error. verify(mergedClusterByStatisticsCollector1, times(0)).clear(); verify(mergedClusterByStatisticsCollector2, times(1)).clear(); @@ -244,16 +260,25 @@ public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldC @Test public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() - throws ExecutionException, InterruptedException + throws InterruptedException { // Store snapshots in a queue final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); - SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of(1L, ImmutableSet.of(0, 1, 2), 2L, ImmutableSet.of(0, 1, 4)); + SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of( + 1L, + ImmutableSet.of(0, 1, 2), + 2L, + ImmutableSet.of(0, 1, 4) + ); doReturn(timeSegmentVsWorkerMap).when(completeKeyStatisticsInformation).getTimeSegmentVsWorkerMap(); final CyclicBarrier barrier = new CyclicBarrier(3); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); + target = spy(new WorkerSketchFetcher( + Mockito.mock(WorkerClient.class), + Mockito.mock(MSQWorkerTaskLauncher.class), + true + )); // When fetching snapshots, return a mock and add it to queue doAnswer(invocation -> { @@ -263,17 +288,17 @@ public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() return Futures.immediateFuture(snapshot); }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyInt(), anyLong()); - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - ImmutableList.of("0", "1", "2", "3", "4"), - stageDefinition - ); +// CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( +// completeKeyStatisticsInformation, +// ImmutableList.of("0", "1", "2", "3", "4"), +// stageDefinition +// ); // Assert that the final result is complete and all other sketches returned have been merged. - eitherCompletableFuture.join(); + // eitherCompletableFuture.join(); Thread.sleep(1000); - Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); +// Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); Assert.assertFalse(snapshotQueue.isEmpty()); // Verify that all statistics were added to controller. snapshotQueue.stream().limit(3).forEach(snapshot -> { @@ -290,6 +315,6 @@ public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() ) ); // Check that the partitions returned by the merged collector is returned by the final future. - Assert.assertEquals(expectedResult, eitherCompletableFuture.get().valueOrThrow()); + //Assert.assertEquals(expectedResult, eitherCompletableFuture.get().valueOrThrow()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java index 3e31bbc1c276..1e0177ae1208 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -21,7 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import org.apache.druid.frame.key.ClusterByPartitions; +import com.google.common.collect.ImmutableSet; import org.apache.druid.frame.key.KeyTestUtils; import org.apache.druid.frame.key.RowKey; import org.apache.druid.java.util.common.IAE; @@ -34,7 +34,6 @@ import org.apache.druid.msq.input.stage.StageInputSpec; import org.apache.druid.msq.input.stage.StageInputSpecSlicer; import org.apache.druid.msq.kernel.QueryDefinition; -import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; @@ -88,7 +87,11 @@ private ControllerQueryKernelTester(int numWorkers) public ControllerQueryKernelTester queryDefinition(QueryDefinition queryDefinition) { this.queryDefinition = Preconditions.checkNotNull(queryDefinition); - this.controllerQueryKernel = new ControllerQueryKernel(queryDefinition); + this.controllerQueryKernel = new ControllerQueryKernel( + queryDefinition, + 100_000_000, + true + ); return this; } @@ -129,21 +132,36 @@ public ControllerQueryKernelTester setupStage( } break; - case POST_READING: + case MERGING_STATISTICS: setupStage(stageNumber, ControllerStagePhase.READING_INPUT, true); - if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { + final ClusterByStatisticsCollector collector = getMockCollector( + stageNumber); - final ClusterByStatisticsCollector keyStatsCollector = getMockCollector( - stageNumber); + for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { + controllerQueryKernel.addPartialKeyStatisticsForStageAndWorker( + stageId, + i, + collector.snapshot().partialKeyStatistics() + ); - for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { - controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( - stageId, - i, - keyStatsCollector.snapshot() - ); - } + controllerQueryKernel.startFetchingStatsFromWorker(stageId, ImmutableSet.of(i)); + } + + for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { + controllerQueryKernel.mergeClusterByStatisticsCollectorForAllTimeChunks( + stageId, + i, + collector.snapshot() + ); + } + + break; + + + case POST_READING: + if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { + setupStage(stageNumber, ControllerStagePhase.MERGING_STATISTICS, true); for (int i = 0; i < queryDefinition.getStageDefinition(stageId).getMaxWorkerCount(); ++i) { controllerQueryKernel.partitionBoundariesSentForWorker(stageId, i); @@ -273,46 +291,55 @@ public void finishStage(int stageNumber, boolean strict) } - public ClusterByStatisticsCollector addResultKeyStatisticsForStageAndWorkers(int stageNumber, int... workers) + public void addPartialKeyStatsInformation(int stageNumber, int... workers) { Preconditions.checkArgument(initialized); - + // Simulate 1000 keys being encountered in the data, so the kernel can generate some partitions. + final ClusterByStatisticsCollector keyStatsCollector = getMockCollector(stageNumber); + StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); for (int worker : workers) { - // Simulate 1000 keys being encountered in the data, so the kernel can generate some partitions. - final ClusterByStatisticsCollector keyStatsCollector = getMockCollector(stageNumber); - controllerQueryKernel.addPartialKeyStatisticsForStageAndWorker( - new StageId(queryDefinition.getQueryId(), stageNumber), + stageId, worker, keyStatsCollector.snapshot().partialKeyStatistics() ); } - return keyStatsCollector; } - public void setResultsCompleteForStageAndWorkerAndWorkers(int stageNumber, int... workers) + public void statsBeingFetchedForWorkers(int stageNumber, Integer... workers) { Preconditions.checkArgument(initialized); - final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + controllerQueryKernel.startFetchingStatsFromWorker( + new StageId(queryDefinition.getQueryId(), stageNumber), + ImmutableSet.copyOf(workers) + ); + } + + public void mergeClusterByStatsForAllTimeChunksForWorkers(int stageNumber, Integer... workers) + { + Preconditions.checkArgument(initialized); + StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + final ClusterByStatisticsCollector keyStatsCollector = getMockCollector(stageNumber); for (int worker : workers) { - controllerQueryKernel.setResultsCompleteForStageAndWorker( + controllerQueryKernel.mergeClusterByStatisticsCollectorForAllTimeChunks( stageId, worker, - new Object() + keyStatsCollector.snapshot() ); } } - public void setPartitionBoundaries(int stageNumber, ClusterByStatisticsCollector clusterByStatisticsCollector) + public void setResultsCompleteForStageAndWorkers(int stageNumber, int... workers) { Preconditions.checkArgument(initialized); - StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); - StageDefinition stageDefinition = controllerQueryKernel.getStageDefinition(stageId); - ClusterByPartitions clusterByPartitions = - stageDefinition - .generatePartitionsForShuffle(clusterByStatisticsCollector) - .valueOrThrow(); - controllerQueryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitions); + final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + for (int worker : workers) { + controllerQueryKernel.setResultsCompleteForStageAndWorker( + stageId, + worker, + new Object() + ); + } } public void failStage(int stageNumber) @@ -374,9 +401,18 @@ public void sendPartitionBoundariesForStageAndWorkers(int stageNumber, int... wo } } + public void sendPartitionBoundariesForStage(int stageNumber) + { + Preconditions.checkArgument(initialized); + final StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + for (int worker : controllerQueryKernel.getWorkersToSendPartitionBoundaries(stageId)) { + controllerQueryKernel.partitionBoundariesSentForWorker(stageId, worker); + } + } + public List getRetriableWorkOrdersAndChangeState(int workeNumber, MSQFault msqFault) { - return controllerQueryKernel.getWorkInCaseWorkerEligibileForRetryElseThrow(workeNumber, msqFault); + return controllerQueryKernel.getWorkInCaseWorkerEligibleForRetryElseThrow(workeNumber, msqFault); } public void failWorkerAndAssertWorkOrderes(int workeNumber, int retriedStage) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTest.java index 990c692de5cb..8e47c470bf82 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTest.java @@ -23,7 +23,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.kernel.worker.WorkerStagePhase; -import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.junit.Assert; import org.junit.Test; @@ -151,10 +150,15 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(0); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); @@ -163,16 +167,24 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(1); controllerQueryKernelTester.sendWorkOrdersForWorkers(1, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(1, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(1, 0); + controllerQueryKernelTester.statsBeingFetchedForWorkers(1, 0); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.sendWorkOrdersForWorkers(1, 1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(1, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(1, 1); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(1, 1); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(1, 1); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(1, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(1, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(1, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(1, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(1, 1); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.RESULTS_READY); newStageNumbers = controllerQueryKernelTester.createAndGetNewStageNumbers(); @@ -182,7 +194,7 @@ public void testCompleteDAGExecutionForMultipleWorkers() controllerQueryKernelTester.startStage(2); controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.sendWorkOrdersForWorkers(2, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(2, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(2, 0); controllerQueryKernelTester.assertStagePhase(2, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.FINISHED); @@ -194,16 +206,20 @@ public void testCompleteDAGExecutionForMultipleWorkers() controllerQueryKernelTester.startStage(3); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.startWorkOrder(3); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(3, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(3, 1); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(3, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(3, 0); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(3, 0, 1); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(3, 0, 1); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(3, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(3, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(3, 0); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(3, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(3, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(3, 1); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(1); @@ -233,19 +249,23 @@ public void testTransitionsInShufflingStagesAndMultipleWorkers() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); controllerQueryKernelTester.finishStage(0, false); @@ -270,7 +290,7 @@ public void testPrematureResultsComplete() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); @@ -283,7 +303,7 @@ public void testPrematureResultsComplete() WorkerStagePhase.NEW ), ISE.class, - () -> controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0) + () -> controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0) ); } @@ -407,7 +427,7 @@ private static void transitionNewToResultsComplete(ControllerQueryKernelTester q { queryKernelTester.startStage(stageNumber); queryKernelTester.startWorkOrder(stageNumber); - queryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(stageNumber, 0); + queryKernelTester.setResultsCompleteForStageAndWorkers(stageNumber, 0); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTest.java index d2f2180e7a4f..d408b47da8e1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/NonShufflingWorkersWithRetryKernelTest.java @@ -59,9 +59,9 @@ public void testWorkerFailedBeforeAnyWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); @@ -83,9 +83,9 @@ public void testWorkerFailedBeforeAllWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @@ -109,12 +109,12 @@ public void testWorkerFailedBeforeAnyResultsRecieved() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -132,7 +132,7 @@ public void testWorkerFailedBeforeAllResultsRecieved() controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); // fail one worker @@ -144,7 +144,7 @@ public void testWorkerFailedBeforeAllResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -213,9 +213,9 @@ public void testMultipleWorkersFailedBeforeAnyWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @@ -240,9 +240,9 @@ public void testMulttipleWorkerFailedBeforeAllWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @@ -270,12 +270,12 @@ public void testMultipleWorkersFailedBeforeAnyResultsRecieved() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -293,7 +293,7 @@ public void testMultipleWorkersFailedBeforeAllResultsRecieved() controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); @@ -308,7 +308,7 @@ public void testMultipleWorkersFailedBeforeAllResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java index 008fd81c1727..144033aebdeb 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java @@ -59,16 +59,24 @@ public void testWorkerFailedBeforeAnyWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @@ -92,50 +100,249 @@ public void testWorkerFailedBeforeAllWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + } + + @Test + public void testWorkerFailedBeforeAnyPartialKeyInfoReceived() + { + + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @Test - public void testWorkerFailedBeforeAnyResultsStatsArrive() + public void testWorkerFailedBeforeAllPartialKeyInfoReceived() { + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.init(); - // workorders sent for both stage - controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + } + + + @Test + public void testWorkerFailedBeforeAnyPartialKeyStatsFetchingFinishes() + { + + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + } + + + @Test + public void testWorkerFailedBeforeAllPartialKeyStatsFetchingFinishes() + { + + + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.init(); + controllerQueryKernelTester.createAndGetNewStageNumbers(); + controllerQueryKernelTester.startStage(0); + + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + + controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + + } + + + @Test + public void testWorkerFailedBeforeAnyStatsAreMerged() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); + + // work orders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); + + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -144,37 +351,46 @@ public void testWorkerFailedBeforeAnyResultsStatsArrive() @Test - public void testWorkerFailedBeforeAllResultsStatsArrive() + public void testWorkerFailedBeforeAllStatsAreMerged() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); - // workorders sent for both stage + // work orders sent for both stage controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); - // fail one worker controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + + + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -187,14 +403,10 @@ public void testWorkerFailedBeforeAnyPartitionBoundariesAreSent() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); - // workorders sent for both stage - controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + // work orders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.MERGING_STATISTICS); controllerQueryKernelTester.init(); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); // fail one worker controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); @@ -203,20 +415,21 @@ public void testWorkerFailedBeforeAnyPartitionBoundariesAreSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); + controllerQueryKernelTester.sendPartitionBoundariesForStage(0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -228,15 +441,10 @@ public void testWorkerFailedBeforeAllPartitionBoundariesAreSent() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); - // workorders sent for both stage - controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + // work orders sent for both stage + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.MERGING_STATISTICS); controllerQueryKernelTester.init(); - - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 1); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); // fail one worker @@ -246,7 +454,7 @@ public void testWorkerFailedBeforeAllPartitionBoundariesAreSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); @@ -255,12 +463,12 @@ public void testWorkerFailedBeforeAllPartitionBoundariesAreSent() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -268,7 +476,7 @@ public void testWorkerFailedBeforeAllPartitionBoundariesAreSent() @Test - public void testWorkerFailedBeforeAnyResultsRecieved() + public void testWorkerFailedBeforeAnyResultsReceived() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); @@ -280,9 +488,9 @@ public void testWorkerFailedBeforeAnyResultsRecieved() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); @@ -296,7 +504,7 @@ public void testWorkerFailedBeforeAnyResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -306,12 +514,12 @@ public void testWorkerFailedBeforeAnyResultsRecieved() @Test - public void testWorkerFailedBeforeAllResultsRecieved() + public void testWorkerFailedBeforeAllResultsReceived() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(2); controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.init(); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); @@ -321,7 +529,7 @@ public void testWorkerFailedBeforeAllResultsRecieved() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); @@ -335,7 +543,7 @@ public void testWorkerFailedBeforeAllResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -415,14 +623,18 @@ public void testMultipleWorkersFailedBeforeAnyWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @@ -451,21 +663,24 @@ public void testMultipleWorkersFailedBeforeAllWorkOrdersSent() controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 2); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); } @Test - public void testMultipleWorkersFailedBeforeAnyResultsStatsArrive() + public void testMultipleWorkersFailedBeforeAnyPartialKeyInfoReceived() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); @@ -485,36 +700,74 @@ public void testMultipleWorkersFailedBeforeAnyResultsStatsArrive() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0, 2); - + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1, 0, 2); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 1, 0, 2); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 1, 0, 2); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); } @Test - public void testMultipleWorkersFailedBeforeAllResultsStatsArrive() + public void testMultipleWorkersFailedBeforeAllPartialKeyInfoReceived() { - ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); - controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.init(); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 2); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 2); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1, 2); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + + @Test + public void testMultipleWorkersFailedBeforeAnyPartialKeyStatsFetchingFinishes() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1, 2); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); @@ -525,18 +778,142 @@ public void testMultipleWorkersFailedBeforeAllResultsStatsArrive() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1, 2); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + @Test + public void testMultipleWorkersFailedBeforeAllPartialKeyStatsFetchingFinishes() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1, 2); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 2); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 1); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 2); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + @Test + public void testMultipleWorkersFailedBeforeAnyStatsAreMerged() + { + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1, 2); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1, 2); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 2); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); + + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); + Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); + + } + + @Test + public void testMultipleWorkersFailedBeforeAllStatsAreMerged() + { + - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); + + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.init(); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1, 2); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1, 2); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 2); + + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); + controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); + controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0, 1); + controllerQueryKernelTester.mergeClusterByStatsForAllTimeChunksForWorkers(0, 0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); @@ -549,34 +926,31 @@ public void testMultipleWorkersFailedBeforeAnyPartitionBoundariesAreSent() ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); - controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.MERGING_STATISTICS); controllerQueryKernelTester.init(); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1, 2); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); - + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1, 0); - controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); } @@ -589,11 +963,8 @@ public void testMultipleWorkersFailedBeforeAllPartitionBoundariesAreSent() ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); - controllerQueryKernelTester.setupStage(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.setupStage(0, ControllerStagePhase.MERGING_STATISTICS); controllerQueryKernelTester.init(); - - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1, 2); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 2); @@ -606,24 +977,26 @@ public void testMultipleWorkersFailedBeforeAllPartitionBoundariesAreSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 1, 0); - + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); } @Test - public void testMultipleWorkersFailedBeforeAnyResultsRecieved() + public void testMultipleWorkersFailedBeforeAnyResultsReceived() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); @@ -638,7 +1011,9 @@ public void testMultipleWorkersFailedBeforeAnyResultsRecieved() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); @@ -649,21 +1024,21 @@ public void testMultipleWorkersFailedBeforeAnyResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 1, 2); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); } @Test - public void testMultipleWorkersFailedBeforeAllResultsRecieved() + public void testMultipleWorkersFailedBeforeAllResultsReceived() { ControllerQueryKernelTester controllerQueryKernelTester = getSimpleQueryDefinition(3); controllerQueryKernelTester.setupStage(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.init(); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 2); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 2); controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(0, 0); controllerQueryKernelTester.failWorkerAndAssertWorkOrderes(1, 0); @@ -672,8 +1047,11 @@ public void testMultipleWorkersFailedBeforeAllResultsRecieved() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorkers(0, 0, 1); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); + + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); controllerQueryKernelTester.sendPartitionBoundariesForStageAndWorkers(0, 0, 1); @@ -684,7 +1062,7 @@ public void testMultipleWorkersFailedBeforeAllResultsRecieved() Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 0); - controllerQueryKernelTester.setResultsCompleteForStageAndWorkerAndWorkers(0, 0, 1); + controllerQueryKernelTester.setResultsCompleteForStageAndWorkers(0, 0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); Assert.assertTrue(controllerQueryKernelTester.createAndGetNewStageNumbers().size() == 1); From d32397a454d5e8c8d8a6500ea96ba52ec92dda31 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 8 Dec 2022 12:05:48 +0530 Subject: [PATCH 16/27] Merge things --- .../apache/druid/msq/exec/ControllerImpl.java | 11 +- .../druid/msq/exec/WorkerSketchFetcher.java | 53 ++++--- .../controller/ControllerQueryKernel.java | 2 +- .../msq/exec/WorkerSketchFetcherTest.java | 141 ++++++++---------- 4 files changed, 104 insertions(+), 103 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 8cb719b7ab0c..98c8a748ecca 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -609,7 +609,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) workerTaskLauncher, isDurableStorageEnabled ); - closer.register(workerSketchFetcher::close) + closer.register(workerSketchFetcher::close); return queryDef; } @@ -2142,6 +2142,7 @@ private Pair> run() throws IOExceptio updateLiveReportMaps(); cleanUpEffectivelyFinishedStages(); retryFailedTasks(); + checkForErrorsInSketchFetcher(); runKernelCommands(); } @@ -2153,6 +2154,14 @@ private Pair> run() throws IOExceptio return Pair.of(queryKernel, workerTaskLauncherFuture); } + private void checkForErrorsInSketchFetcher() + { + Throwable throwable = workerSketchFetcher.getError(); + if (throwable != null) { + throw new ISE(throwable, "worker sketch fetch failed"); + } + } + private void retryFailedTasks() throws InterruptedException { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java index 1b43f2013a2c..859bd667d4ca 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -46,7 +46,6 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Supplier; -import java.util.stream.Collectors; /** * Queues up fetching sketches from workers and progressively generates partitions boundaries. @@ -211,37 +210,45 @@ public void sequentialTimeChunkMerging( if (!completeKeyStatisticsInformation.isComplete()) { throw new ISE("All worker partial key information not received for stage[%d]", stageId.getStageNumber()); } - Set workers = tasks.stream().map(MSQTasks::workerFromTaskId).collect(Collectors.toSet()); + completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().forEach((timeChunk, wks) -> { for (String taskId : tasks) { int workerNumber = MSQTasks.workerFromTaskId(taskId); - if (workers.contains(workerNumber)) { - executorService.submit(() -> { - fetchStatsFromWorker( - kernelActions, - () -> workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( - taskId, - stageId.getQueryId(), - stageId.getStageNumber(), - timeChunk - ), - taskId, - (kernel, snapshot) -> kernel.mergeClusterByStatisticsCollectorForTimeChunk( - stageId, - workerNumber, - timeChunk, - snapshot - ), - retryOperation - ); + executorService.submit(() -> { + fetchStatsFromWorker( + kernelActions, + () -> workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( + taskId, + stageId.getQueryId(), + stageId.getStageNumber(), + timeChunk + ), + taskId, + (kernel, snapshot) -> kernel.mergeClusterByStatisticsCollectorForTimeChunk( + stageId, + workerNumber, + timeChunk, + snapshot + ), + retryOperation + ); - }); - } + }); } }); } + + /** + * Returns {@link Throwable} if error, else null + */ + public Throwable getError() + { + return isError.get(); + } + + @Override public void close() { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 97939c20828e..36c2c3cbf206 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -245,7 +245,7 @@ public void markSuccessfulTerminalStagesAsFinished() } /** - * Returns true if all the stages comprising the query definition have been sucessful in producing their results + * Returns true if all the stages comprising the query definition have been successful in producing their results */ public boolean isSuccess() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java index 83fb73043bd9..d16cdbada382 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -28,8 +28,10 @@ import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.key.RowKey; import org.apache.druid.java.util.common.Either; +import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernel; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; @@ -41,14 +43,8 @@ import org.mockito.MockitoAnnotations; import java.util.List; -import java.util.Queue; -import java.util.Set; -import java.util.SortedMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import static org.easymock.EasyMock.mock; import static org.mockito.ArgumentMatchers.any; @@ -58,13 +54,18 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; public class WorkerSketchFetcherTest { @Mock private CompleteKeyStatisticsInformation completeKeyStatisticsInformation; + + @Mock + private MSQWorkerTaskLauncher workerTaskLauncher; + + @Mock + private ControllerQueryKernel kernel; + @Mock private StageDefinition stageDefinition; @Mock @@ -85,14 +86,23 @@ public void setUp() { mocks = MockitoAnnotations.openMocks(this); doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); - doReturn(clusterBy).when(stageDefinition).getClusterBy(); - doReturn(25_000).when(stageDefinition).getMaxPartitionCount(); - - expectedPartitions1 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition(mock(RowKey.class), mock(RowKey.class)))); - expectedPartitions2 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition(mock(RowKey.class), mock(RowKey.class)))); - - doReturn(Either.value(expectedPartitions1)).when(stageDefinition).generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector1)); - doReturn(Either.value(expectedPartitions2)).when(stageDefinition).generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector2)); + doReturn(true).when(completeKeyStatisticsInformation).isComplete(); + doReturn(ImmutableSortedMap.of(123L, ImmutableSet.of(1, 2))).when(completeKeyStatisticsInformation) + .getTimeSegmentVsWorkerMap(); + + expectedPartitions1 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition( + mock(RowKey.class), + mock(RowKey.class) + ))); + expectedPartitions2 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition( + mock(RowKey.class), + mock(RowKey.class) + ))); + + doReturn(Either.value(expectedPartitions1)).when(stageDefinition) + .generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector1)); + doReturn(Either.value(expectedPartitions2)).when(stageDefinition) + .generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector2)); doReturn( mergedClusterByStatisticsCollector1, @@ -110,91 +120,66 @@ public void tearDown() throws Exception } @Test - public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() - throws ExecutionException, InterruptedException + public void test_submitFetcherTask_parallelFetch() + throws InterruptedException { - // Store snapshots in a queue - final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); - final List workerIds = ImmutableList.of("0", "1", "2", "3", "4"); - final CountDownLatch latch = new CountDownLatch(workerIds.size()); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); + final List taskIds = ImmutableList.of("task-worker0_0", "task-worker1_0", "task-worker2_1"); + final CountDownLatch latch = new CountDownLatch(taskIds.size()); + + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, true)); // When fetching snapshots, return a mock and add it to queue doAnswer(invocation -> { ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class); - snapshotQueue.add(snapshot); - latch.countDown(); return Futures.immediateFuture(snapshot); }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - workerIds, - stageDefinition + target.inMemoryFullSketchMerging( + (kernelConsumer) -> + { + kernelConsumer.accept(kernel); + latch.countDown(); + } + , + stageDefinition.getId(), + ImmutableSet.copyOf(taskIds), + ((queryKernel, integer, msqFault) -> {}) ); - // Assert that the final result is complete and all other sketches returned have been merged. - eitherCompletableFuture.join(); - Thread.sleep(1000); - Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); - Assert.assertFalse(snapshotQueue.isEmpty()); - // Verify that all statistics were added to controller. - for (ClusterByStatisticsSnapshot snapshot : snapshotQueue) { - verify(mergedClusterByStatisticsCollector1, times(1)).addAll(eq(snapshot)); - } - // Check that the partitions returned by the merged collector is returned by the final future. - Assert.assertEquals(expectedPartitions1, eitherCompletableFuture.get().valueOrThrow()); + latch.await(1, TimeUnit.MINUTES); + Assert.assertEquals(0, latch.getCount()); + } @Test - public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() - throws ExecutionException, InterruptedException + public void test_submitFetcherTask_sequentialFetch() throws InterruptedException { - // Store snapshots in a queue - final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); - - SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of(1L, ImmutableSet.of(0, 1, 2), 2L, ImmutableSet.of(0, 1, 4)); - doReturn(timeSegmentVsWorkerMap).when(completeKeyStatisticsInformation).getTimeSegmentVsWorkerMap(); + final List taskIds = ImmutableList.of("task-worker0_0", "task-worker1_1", "task-worker2_1"); + final CountDownLatch latch = new CountDownLatch(taskIds.size() - 1); - final CyclicBarrier barrier = new CyclicBarrier(3); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, true)); // When fetching snapshots, return a mock and add it to queue doAnswer(invocation -> { ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class); - snapshotQueue.add(snapshot); - barrier.await(); return Futures.immediateFuture(snapshot); }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyInt(), anyLong()); - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - ImmutableList.of("0", "1", "2", "3", "4"), - stageDefinition + target.sequentialTimeChunkMerging( + (kernelConsumer) -> + { + kernelConsumer.accept(kernel); + latch.countDown(); + }, + completeKeyStatisticsInformation + , + stageDefinition.getId(), + ImmutableSet.copyOf(taskIds), + ((queryKernel, integer, msqFault) -> {}) ); - // Assert that the final result is complete and all other sketches returned have been merged. - eitherCompletableFuture.join(); - Thread.sleep(1000); - - Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); - Assert.assertFalse(snapshotQueue.isEmpty()); - // Verify that all statistics were added to controller. - snapshotQueue.stream().limit(3).forEach(snapshot -> { - verify(mergedClusterByStatisticsCollector1, times(1)).addAll(eq(snapshot)); - }); - snapshotQueue.stream().skip(3).limit(3).forEach(snapshot -> { - verify(mergedClusterByStatisticsCollector2, times(1)).addAll(eq(snapshot)); - }); - ClusterByPartitions expectedResult = - new ClusterByPartitions( - ImmutableList.of( - new ClusterByPartition(expectedPartitions1.get(0).getStart(), expectedPartitions2.get(0).getStart()), - new ClusterByPartition(expectedPartitions2.get(0).getStart(), expectedPartitions2.get(0).getEnd()) - ) - ); - // Check that the partitions returned by the merged collector is returned by the final future. - Assert.assertEquals(expectedResult, eitherCompletableFuture.get().valueOrThrow()); + latch.await(1, TimeUnit.MINUTES); + Assert.assertEquals(0, latch.getCount()); } } From a286480ea9dbdc66d8f484461700d4b9b66200f7 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 8 Dec 2022 14:14:55 +0530 Subject: [PATCH 17/27] Fixing checkstyle. --- .../druid/msq/exec/WorkerSketchFetcherTest.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java index d16cdbada382..777d3f239d01 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; import com.google.common.util.concurrent.Futures; -import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.ClusterByPartition; import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.key.RowKey; @@ -68,8 +67,7 @@ public class WorkerSketchFetcherTest @Mock private StageDefinition stageDefinition; - @Mock - private ClusterBy clusterBy; + @Mock private ClusterByStatisticsCollector mergedClusterByStatisticsCollector1; @Mock @@ -136,12 +134,10 @@ public void test_submitFetcherTask_parallelFetch() }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); target.inMemoryFullSketchMerging( - (kernelConsumer) -> - { + (kernelConsumer) -> { kernelConsumer.accept(kernel); latch.countDown(); - } - , + }, stageDefinition.getId(), ImmutableSet.copyOf(taskIds), ((queryKernel, integer, msqFault) -> {}) @@ -167,13 +163,11 @@ public void test_submitFetcherTask_sequentialFetch() throws InterruptedException }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyInt(), anyLong()); target.sequentialTimeChunkMerging( - (kernelConsumer) -> - { + (kernelConsumer) -> { kernelConsumer.accept(kernel); latch.countDown(); }, - completeKeyStatisticsInformation - , + completeKeyStatisticsInformation, stageDefinition.getId(), ImmutableSet.copyOf(taskIds), ((queryKernel, integer, msqFault) -> {}) From bf25675686cf835d04632f2f40de2784987b1cb8 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 9 Dec 2022 20:07:38 +0530 Subject: [PATCH 18/27] Adding new context param for fault tolerance. Adding stale task handling in sketchFetcher. Adding UT's. --- .../apache/druid/msq/exec/ControllerImpl.java | 66 +++-- .../org/apache/druid/msq/exec/Limits.java | 6 + .../druid/msq/exec/WorkerSketchFetcher.java | 133 +++++---- .../msq/indexing/MSQWorkerTaskLauncher.java | 11 +- .../DurableStorageConfigurationFault.java | 2 +- .../msq/util/MultiStageQueryContext.java | 20 +- .../druid/msq/exec/ControllerImplTest.java | 6 +- .../msq/exec/WorkerSketchFetcherTest.java | 274 ++++++++++++++---- .../druid/msq/kernel/StageDefinitionTest.java | 69 +++++ .../apache/druid/msq/test/MSQTestBase.java | 5 +- .../msq/util/MultiStageQueryContextTest.java | 19 +- .../testsEx/msq/ITMultiStageQueryHA.java | 2 +- 12 files changed, 471 insertions(+), 142 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 98c8a748ecca..7edaa4d1864c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -162,6 +162,7 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.msq.util.PassthroughAggregatorFactory; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; @@ -224,8 +225,6 @@ public class ControllerImpl implements Controller private final MSQControllerTask task; private final ControllerContext context; - private final boolean isDurableStorageEnabled; - /** * Queue of "commands" to run on the {@link ControllerQueryKernel}. Various threads insert into the queue * using {@link #addToKernelManipulationQueue}. The main thread running {@link RunQueryUntilDone#run()} reads @@ -284,6 +283,8 @@ public class ControllerImpl implements Controller private Map stageToStatsMergingMode; private WorkerMemoryParameters workerMemoryParameters; + private boolean isDurableStorageEnabled; + private boolean isFaultToleranceEnabled; public ControllerImpl( final MSQControllerTask task, @@ -292,11 +293,6 @@ public ControllerImpl( { this.task = task; this.context = context; - this.isDurableStorageEnabled = - MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec() - .getQuery() - .context()); - } @Override @@ -536,6 +532,36 @@ public void addToKernelManipulationQueue(Consumer kernelC private QueryDefinition initializeQueryDefAndState(final Closer closer) { + final QueryContext queryContext = task.getQuerySpec().getQuery().context(); + isFaultToleranceEnabled = MultiStageQueryContext.isFaultToleranceEnabled(queryContext); + + if (isFaultToleranceEnabled) { + if (!queryContext.containsKey(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE)) { + // if context key not set, enable durableStorage automatically. + isDurableStorageEnabled = true; + } else { + // if context key is set, and durableStorage is turned on. + if (MultiStageQueryContext.isDurableStorageEnabled(queryContext)) { + isDurableStorageEnabled = true; + } else { + throw new MSQException( + UnknownFault.forMessage( + StringUtils.format( + "Context param[%s] cannot be explicitly set to false when context param[%s] is" + + " set to true. Either remove the context param[%s] or explicitly set it to true.", + MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, + MultiStageQueryContext.CTX_FAULT_TOLERANCE, + MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE + ))); + } + } + } else { + isDurableStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(queryContext); + } + + log.debug("Task [%s] durable storage mode is set to %s.", task.getId(), isDurableStorageEnabled); + log.debug("Task [%s] fault tolerance mode is set to %s.", task.getId(), isFaultToleranceEnabled); + this.selfDruidNode = context.selfNode(); context.registerController(this, closer); @@ -552,9 +578,6 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) QueryValidator.validateQueryDef(queryDef); queryDefRef.set(queryDef); - log.debug("Query [%s] durable storage mode is set to %s.", queryDef.getQueryId(), isDurableStorageEnabled); - - long maxParseExceptions = -1; if (task.getSqlQueryContext() != null) { @@ -564,14 +587,13 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) .orElse(MSQWarnings.DEFAULT_MAX_PARSE_EXCEPTIONS_ALLOWED); } - this.workerTaskLauncher = new MSQWorkerTaskLauncher( id(), task.getDataSource(), context, (failedTask, fault) -> { addToKernelManipulationQueue((kernel) -> { - if (isDurableStorageEnabled) { + if (isFaultToleranceEnabled) { addToRetryQueue(kernel, failedTask.getWorkerNumber(), fault); } else { throw new MSQException(fault); @@ -595,19 +617,15 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) stageDefinition.getId().getStageNumber(), finalizeClusterStatisticsMergeMode( stageDefinition, - MultiStageQueryContext.getClusterStatisticsMergeMode( - task.getQuerySpec() - .getQuery() - .context()) + MultiStageQueryContext.getClusterStatisticsMergeMode(queryContext) ) ) ); this.workerMemoryParameters = WorkerMemoryParameters.createProductionInstanceForController(context.injector()); - this.workerSketchFetcher = new WorkerSketchFetcher( netClient, workerTaskLauncher, - isDurableStorageEnabled + isFaultToleranceEnabled ); closer.register(workerSketchFetcher::close); @@ -693,7 +711,7 @@ public void workerError(MSQErrorReport errorReport) { // move inside kernel if (workerTaskLauncher.isTaskCanceledByController(errorReport.getTaskId()) || - workerTaskLauncher.isTaskRetried(errorReport.getTaskId())) { + !workerTaskLauncher.isTaskLatest(errorReport.getTaskId())) { log.info("Ignoring task %s", errorReport.getTaskId()); } else { workerErrorRef.compareAndSet(null, errorReport); @@ -1179,7 +1197,7 @@ private void startWorkForStage( (netClient, taskId, workerNumber) -> ( netClient.postWorkOrder(taskId, workOrders.get(workerNumber))), workOrders.keySet(), (taskId) -> queryKernel.workOrdersSentForWorker(stageId, MSQTasks.workerFromTaskId(taskId)), - isDurableStorageEnabled + isFaultToleranceEnabled ); } @@ -1202,7 +1220,7 @@ private void postResultPartitionBoundariesForStage( ), workers, (taskId) -> queryKernel.partitionBoundariesSentForWorker(stageId, MSQTasks.workerFromTaskId(taskId)), - isDurableStorageEnabled + isFaultToleranceEnabled ); } @@ -2124,7 +2142,7 @@ public RunQueryUntilDone( this.queryKernel = new ControllerQueryKernel( queryDef, workerMemoryParameters.getPartitionStatisticsMaxRetainedBytes(), - isDurableStorageEnabled + isFaultToleranceEnabled ); } @@ -2220,7 +2238,7 @@ private void retryFailedTasks() throws InterruptedException return workOrderSet; }); }, - isDurableStorageEnabled + isFaultToleranceEnabled ); } } @@ -2534,7 +2552,7 @@ static ClusterStatisticsMergeMode finalizeClusterStatisticsMergeMode( if (clusterBy.getBucketByCount() == 0) { // If there is no time clustering, there is no scope for sequential merge mergeMode = ClusterStatisticsMergeMode.PARALLEL; - } else if (stageDef.getMaxWorkerCount() > WorkerSketchFetcher.WORKER_THRESHOLD) { + } else if (stageDef.getMaxWorkerCount() > Limits.MAX_WORKERS_FOR_PARALLEL_MERGE) { mergeMode = ClusterStatisticsMergeMode.SEQUENTIAL; } else { mergeMode = ClusterStatisticsMergeMode.PARALLEL; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index 63d3c5e09659..31a309afb1df 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -78,4 +78,10 @@ public class Limits * Maximum relaunches per worker. Initial run is not a relaunch. The worker will be spawned 1 + workerRelaunchLimit times before erroring out. */ public static final int PER_WORKER_RELAUNCH_LIMIT = 2; + + /** + * Max number of workers for {@link ClusterStatisticsMergeMode#PARALLEL}. If the number of workers is more than this, + * {@link ClusterStatisticsMergeMode#SEQUENTIAL} mode is chosen. + */ + public static final long MAX_WORKERS_FOR_PARALLEL_MERGE = 100; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java index 859bd667d4ca..4a072e9bca93 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -54,16 +54,15 @@ public class WorkerSketchFetcher implements AutoCloseable { private static final Logger log = new Logger(WorkerSketchFetcher.class); private static final int DEFAULT_THREAD_COUNT = 4; - // If the combined size of worker sketches is more than this threshold, SEQUENTIAL merging mode is used. - public static final long WORKER_THRESHOLD = 100; private final WorkerClient workerClient; - private final ExecutorService executorService; private final MSQWorkerTaskLauncher workerTaskLauncher; private final boolean retryEnabled; - AtomicReference isError = new AtomicReference<>(); + private AtomicReference isError = new AtomicReference<>(); + final ExecutorService executorService; + public WorkerSketchFetcher( WorkerClient workerClient, @@ -77,14 +76,6 @@ public WorkerSketchFetcher( this.retryEnabled = retryEnabled; } - /** - * Submits a request to fetch and generate partitions for the given worker statistics and returns a future for it. It - * decides based on the statistics if it should fetch sketches one by one or together. - * The future can successfully return a null signaling that partition statistics from all the workers have not been fetched yet. - * - */ - - /** * Fetches the full {@link ClusterByStatisticsCollector} from all workers and generates partition boundaries from them. * This is faster than fetching them timechunk by timechunk but the collector will be downsampled till it can fit @@ -142,8 +133,9 @@ private void fetchStatsFromWorker( executorService.shutdownNow(); return; } + int worker = MSQTasks.workerFromTaskId(taskId); try { - workerTaskLauncher.waitUntilWorkersReady(ImmutableSet.of(MSQTasks.workerFromTaskId(taskId))); + workerTaskLauncher.waitUntilWorkersReady(ImmutableSet.of(worker)); } catch (InterruptedException interruptedException) { isError.compareAndSet(null, interruptedException); @@ -151,6 +143,16 @@ private void fetchStatsFromWorker( return; } + // if task is not the latest task. It must have retried. + if (!workerTaskLauncher.isTaskLatest(taskId)) { + log.info( + "Task[%s] is no longer the latest task for worker[%d], hence ignoring fetching stats from this worker", + taskId, + worker + ); + return; + } + ListenableFuture fetchFuture = fetchStatsSupplier.get(); SettableFuture kernelActionFuture = SettableFuture.create(); @@ -160,40 +162,71 @@ private void fetchStatsFromWorker( @Override public void onSuccess(@Nullable ClusterByStatisticsSnapshot result) { - kernelActions.accept((queryKernel) -> { - successKernelOperation.accept(queryKernel, result); - // we do not want to have too many key collector sketches in the event queue as that cause memory issues - // blocking the executor service thread until the kernel operation is finished. - // so we would have utmost DEFAULT_THREAD_COUNT number of sketches in the queue. - kernelActionFuture.set(true); - }); + try { + kernelActions.accept((queryKernel) -> { + try { + successKernelOperation.accept(queryKernel, result); + // we do not want to have too many key collector sketches in the event queue as that cause memory issues + // blocking the executor service thread until the kernel operation is finished. + // so we would have utmost DEFAULT_THREAD_COUNT number of sketches in the queue. + kernelActionFuture.set(true); + } + catch (Exception e) { + failFutureAndShutDownExecutorService(e, taskId, kernelActionFuture); + } + }); + } + catch (Exception e) { + failFutureAndShutDownExecutorService(e, taskId, kernelActionFuture); + } } @Override public void onFailure(Throwable t) { + if (retryEnabled) { //add to retry queue - kernelActions.accept((kernel) -> { - retryOperation.accept(kernel, MSQTasks.workerFromTaskId(taskId), new WorkerRpcFailedFault(taskId)); - kernelActionFuture.set(false); - }); - kernelActionFuture.set(false); + try { + kernelActions.accept((kernel) -> { + try { + retryOperation.accept(kernel, worker, new WorkerRpcFailedFault(taskId)); + kernelActionFuture.set(false); - } else { - if (isError.compareAndSet(null, t)) { - log.error(t, "Failed while fetching stats from task[%s]", taskId); + } + catch (Exception e) { + failFutureAndShutDownExecutorService(e, taskId, kernelActionFuture); + } + }); + kernelActionFuture.set(false); } - executorService.shutdownNow(); - kernelActionFuture.setException(t); + catch (Exception e) { + failFutureAndShutDownExecutorService(e, taskId, kernelActionFuture); + } + } else { + failFutureAndShutDownExecutorService(t, taskId, kernelActionFuture); } + } }); FutureUtils.getUnchecked(kernelActionFuture, true); } + private void failFutureAndShutDownExecutorService( + Throwable t, + String taskId, + SettableFuture kernelActionFuture + ) + { + if (isError.compareAndSet(null, t)) { + log.error(t, "Failed while fetching stats from task[%s]", taskId); + } + executorService.shutdownNow(); + kernelActionFuture.setException(t); + } + /** * Fetches cluster statistics from all workers and generates partition boundaries from them one time chunk at a time. * This takes longer due to the overhead of fetching sketches, however, this prevents any loss in accuracy from @@ -215,26 +248,28 @@ public void sequentialTimeChunkMerging( for (String taskId : tasks) { int workerNumber = MSQTasks.workerFromTaskId(taskId); - executorService.submit(() -> { - fetchStatsFromWorker( - kernelActions, - () -> workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( - taskId, - stageId.getQueryId(), - stageId.getStageNumber(), - timeChunk - ), - taskId, - (kernel, snapshot) -> kernel.mergeClusterByStatisticsCollectorForTimeChunk( - stageId, - workerNumber, - timeChunk, - snapshot - ), - retryOperation - ); + if (wks.contains(workerNumber)) { + executorService.submit(() -> { + fetchStatsFromWorker( + kernelActions, + () -> workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( + taskId, + stageId.getQueryId(), + stageId.getStageNumber(), + timeChunk + ), + taskId, + (kernel, snapshot) -> kernel.mergeClusterByStatisticsCollectorForTimeChunk( + stageId, + workerNumber, + timeChunk, + snapshot + ), + retryOperation + ); - }); + }); + } } }); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 9f1e73fed755..1be409ed6a9f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -290,9 +290,12 @@ public boolean isTaskCanceledByController(String taskId) } - public boolean isTaskRetried(String taskId) + public boolean isTaskLatest(String taskId) { - return tasksToCleanup.contains(taskId) || workersToRelaunch.contains(MSQTasks.workerFromTaskId(taskId)); + int worker = MSQTasks.workerFromTaskId(taskId); + synchronized (taskIds) { + return taskId.equals(taskIds.get(worker)); + } } private void mainLoop() @@ -376,7 +379,7 @@ private void runNewTasks() final Map taskContext = new HashMap<>(); if (durableStageStorageEnabled) { - taskContext.put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, true); + taskContext.put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, true); } if (maxParseExceptions != null) { @@ -550,7 +553,7 @@ private void relaunchTasks() tasksToCleanup.add(latestTaskId); taskTrackers.remove(latestTaskId); log.info( - "Relauching worker[%d] with new task id[%s] with worker relaunch count[%d] and job relaunch count[%d]", + "Relaunching worker[%d] with new task id[%s] with worker relaunch count[%d] and job relaunch count[%d]", relaunchedTask.getWorkerNumber(), relaunchedTask.getId(), toRelaunch.getRetryCount(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.java index 2fe0d6eccc7d..cd65e58c7b7b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/DurableStorageConfigurationFault.java @@ -44,7 +44,7 @@ public DurableStorageConfigurationFault(@JsonProperty("message") final String er + "Check the documentation on how to enable durable storage mode. " + "If you want to still query without durable storage mode, set %s to false in the query context. Got error %s", MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_ENABLED, - MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, + MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, errorMessage ); this.errorMessage = errorMessage; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 3dc622870f43..a596879fad50 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -58,10 +58,14 @@ public class MultiStageQueryContext public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; - public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; + public static final String CTX_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; + private static final boolean DEFAULT_DURABLE_SHUFFLE_STORAGE = false; + + public static final String CTX_FAULT_TOLERANCE = "faultTolerance"; + public static final boolean DEFAULT_FAULT_TOLERANCE = false; + public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode"; public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.PARALLEL.toString(); - private static final boolean DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = false; public static final String CTX_DESTINATION = "destination"; private static final String DEFAULT_DESTINATION = null; @@ -91,8 +95,16 @@ public static String getMSQMode(final QueryContext queryContext) public static boolean isDurableStorageEnabled(final QueryContext queryContext) { return queryContext.getBoolean( - CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, - DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE + CTX_DURABLE_SHUFFLE_STORAGE, + DEFAULT_DURABLE_SHUFFLE_STORAGE + ); + } + + public static boolean isFaultToleranceEnabled(final QueryContext queryContext) + { + return queryContext.getBoolean( + CTX_FAULT_TOLERANCE, + DEFAULT_FAULT_TOLERANCE ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java index c6a657d97a00..dc399e6623dd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java @@ -156,7 +156,7 @@ public void test_noClusterByColumns_shouldBeParallel() doReturn(ClusterBy.none()).when(stageDefinition).getClusterBy(); // Worker count above threshold - doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); + doReturn((int) Limits.MAX_WORKERS_FOR_PARALLEL_MERGE + 1).when(stageDefinition).getMaxWorkerCount(); Assert.assertEquals( ClusterStatisticsMergeMode.PARALLEL, @@ -169,13 +169,13 @@ public void test_noClusterByColumns_shouldBeParallel() } @Test - public void test_wokersAboveThreshold_shouldBeSequential() + public void test_numWorkersAboveThreshold_shouldBeSequential() { // Cluster by bucket count not 0 doReturn(1).when(clusterBy).getBucketByCount(); // Worker count above threshold - doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); + doReturn((int) Limits.MAX_WORKERS_FOR_PARALLEL_MERGE + 1).when(stageDefinition).getMaxWorkerCount(); Assert.assertEquals( ClusterStatisticsMergeMode.SEQUENTIAL, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java index 777d3f239d01..20f8503cf9be 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -23,15 +23,11 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; import com.google.common.util.concurrent.Futures; -import org.apache.druid.frame.key.ClusterByPartition; -import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.frame.key.RowKey; -import org.apache.druid.java.util.common.Either; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.controller.ControllerQueryKernel; -import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; import org.junit.After; @@ -42,6 +38,7 @@ import org.mockito.MockitoAnnotations; import java.util.List; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -49,7 +46,6 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; @@ -67,45 +63,26 @@ public class WorkerSketchFetcherTest @Mock private StageDefinition stageDefinition; - - @Mock - private ClusterByStatisticsCollector mergedClusterByStatisticsCollector1; - @Mock - private ClusterByStatisticsCollector mergedClusterByStatisticsCollector2; @Mock private WorkerClient workerClient; - private ClusterByPartitions expectedPartitions1; - private ClusterByPartitions expectedPartitions2; private AutoCloseable mocks; private WorkerSketchFetcher target; + private static final String TASK_0 = "task-worker0_0"; + private static final String TASK_1 = "task-worker1_0"; + private static final String TASK_2 = "task-worker2_1"; + private static final List TASK_IDS = ImmutableList.of(TASK_0, TASK_1, TASK_2); + @Before public void setUp() { mocks = MockitoAnnotations.openMocks(this); doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); - doReturn(true).when(completeKeyStatisticsInformation).isComplete(); + doReturn(ImmutableSortedMap.of(123L, ImmutableSet.of(1, 2))).when(completeKeyStatisticsInformation) .getTimeSegmentVsWorkerMap(); - expectedPartitions1 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition( - mock(RowKey.class), - mock(RowKey.class) - ))); - expectedPartitions2 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition( - mock(RowKey.class), - mock(RowKey.class) - ))); - - doReturn(Either.value(expectedPartitions1)).when(stageDefinition) - .generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector1)); - doReturn(Either.value(expectedPartitions2)).when(stageDefinition) - .generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector2)); - - doReturn( - mergedClusterByStatisticsCollector1, - mergedClusterByStatisticsCollector2 - ).when(stageDefinition).createResultKeyStatisticsCollector(anyInt()); + doReturn(true).when(workerTaskLauncher).isTaskLatest(any()); } @After @@ -118,12 +95,10 @@ public void tearDown() throws Exception } @Test - public void test_submitFetcherTask_parallelFetch() - throws InterruptedException + public void test_submitFetcherTask_parallelFetch() throws InterruptedException { - final List taskIds = ImmutableList.of("task-worker0_0", "task-worker1_0", "task-worker2_1"); - final CountDownLatch latch = new CountDownLatch(taskIds.size()); + final CountDownLatch latch = new CountDownLatch(TASK_IDS.size()); target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, true)); @@ -133,17 +108,12 @@ public void test_submitFetcherTask_parallelFetch() return Futures.immediateFuture(snapshot); }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); - target.inMemoryFullSketchMerging( - (kernelConsumer) -> { - kernelConsumer.accept(kernel); - latch.countDown(); - }, - stageDefinition.getId(), - ImmutableSet.copyOf(taskIds), - ((queryKernel, integer, msqFault) -> {}) - ); + target.inMemoryFullSketchMerging((kernelConsumer) -> { + kernelConsumer.accept(kernel); + latch.countDown(); + }, stageDefinition.getId(), ImmutableSet.copyOf(TASK_IDS), ((queryKernel, integer, msqFault) -> {})); - latch.await(1, TimeUnit.MINUTES); + latch.await(5, TimeUnit.SECONDS); Assert.assertEquals(0, latch.getCount()); } @@ -151,8 +121,8 @@ public void test_submitFetcherTask_parallelFetch() @Test public void test_submitFetcherTask_sequentialFetch() throws InterruptedException { - final List taskIds = ImmutableList.of("task-worker0_0", "task-worker1_1", "task-worker2_1"); - final CountDownLatch latch = new CountDownLatch(taskIds.size() - 1); + doReturn(true).when(completeKeyStatisticsInformation).isComplete(); + final CountDownLatch latch = new CountDownLatch(TASK_IDS.size() - 1); target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, true)); @@ -169,11 +139,215 @@ public void test_submitFetcherTask_sequentialFetch() throws InterruptedException }, completeKeyStatisticsInformation, stageDefinition.getId(), - ImmutableSet.copyOf(taskIds), + ImmutableSet.copyOf(TASK_IDS), ((queryKernel, integer, msqFault) -> {}) ); - latch.await(1, TimeUnit.MINUTES); + latch.await(5, TimeUnit.SECONDS); Assert.assertEquals(0, latch.getCount()); } + + @Test + public void test_sequentialMerge_nonCompleteInformation() + { + + doReturn(false).when(completeKeyStatisticsInformation).isComplete(); + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, true)); + Assert.assertThrows(ISE.class, () -> target.sequentialTimeChunkMerging( + (ignore) -> {}, + completeKeyStatisticsInformation, + stageDefinition.getId(), + ImmutableSet.of(""), + ((queryKernel, integer, msqFault) -> {}) + )); + } + + @Test + public void test_inMemoryRetryEnabled_retryInvoked() throws InterruptedException + { + final CountDownLatch latch = new CountDownLatch(TASK_IDS.size()); + + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, true)); + + workersWithFailedFetchParallel(ImmutableSet.of(TASK_1)); + + CountDownLatch retryLatch = new CountDownLatch(1); + target.inMemoryFullSketchMerging( + (kernelConsumer) -> { + kernelConsumer.accept(kernel); + latch.countDown(); + }, + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + if (integer.equals(1) && msqFault.getErrorMessage().contains(TASK_1)) { + retryLatch.countDown(); + } + }) + ); + + latch.await(5, TimeUnit.SECONDS); + retryLatch.await(5, TimeUnit.SECONDS); + Assert.assertEquals(0, latch.getCount()); + Assert.assertEquals(0, retryLatch.getCount()); + } + + @Test + public void test_SequentialRetryEnabled_retryInvoked() throws InterruptedException + { + doReturn(true).when(completeKeyStatisticsInformation).isComplete(); + final CountDownLatch latch = new CountDownLatch(TASK_IDS.size() - 1); + + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, true)); + + workersWithFailedFetchSequential(ImmutableSet.of(TASK_1)); + CountDownLatch retryLatch = new CountDownLatch(1); + target.sequentialTimeChunkMerging( + (kernelConsumer) -> { + kernelConsumer.accept(kernel); + latch.countDown(); + }, + completeKeyStatisticsInformation, + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + if (integer.equals(1) && msqFault.getErrorMessage().contains(TASK_1)) { + retryLatch.countDown(); + } + }) + ); + + latch.await(5, TimeUnit.SECONDS); + retryLatch.await(5, TimeUnit.SECONDS); + Assert.assertEquals(0, latch.getCount()); + Assert.assertEquals(0, retryLatch.getCount()); + } + + @Test + public void test_InMemoryRetryDisabled_multipleFailures() throws InterruptedException + { + + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, false)); + + workersWithFailedFetchParallel(ImmutableSet.of(TASK_1, TASK_0)); + + target.inMemoryFullSketchMerging( + (kernelConsumer) -> kernelConsumer.accept(kernel), + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + + while (!target.executorService.isShutdown()) { + Thread.sleep(100); + } + Assert.assertNotNull(target.getError().getMessage().contains("Task fetch failed:")); + + } + + @Test + public void test_InMemoryRetryDisabled_singleFailure() throws InterruptedException + { + + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, false)); + + workersWithFailedFetchParallel(ImmutableSet.of(TASK_1)); + + target.inMemoryFullSketchMerging( + (kernelConsumer) -> kernelConsumer.accept(kernel), + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + + while (!target.executorService.isShutdown()) { + Thread.sleep(100); + } + Assert.assertNotNull(target.getError().getMessage().contains(TASK_1)); + + } + + + @Test + public void test_SequentialRetryDisabled_multipleFailures() throws InterruptedException + { + + doReturn(true).when(completeKeyStatisticsInformation).isComplete(); + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, false)); + + workersWithFailedFetchSequential(ImmutableSet.of(TASK_1, TASK_0)); + + target.sequentialTimeChunkMerging( + (kernelConsumer) -> { + kernelConsumer.accept(kernel); + }, + completeKeyStatisticsInformation, + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + + while (!target.executorService.isShutdown()) { + Thread.sleep(100); + } + Assert.assertNotNull(target.getError().getMessage().contains("Task fetch failed:")); + + } + + @Test + public void test_SequentialRetryDisabled_singleFailure() throws InterruptedException + { + doReturn(true).when(completeKeyStatisticsInformation).isComplete(); + target = spy(new WorkerSketchFetcher(workerClient, workerTaskLauncher, false)); + + workersWithFailedFetchSequential(ImmutableSet.of(TASK_1)); + + target.sequentialTimeChunkMerging( + (kernelConsumer) -> { + kernelConsumer.accept(kernel); + }, + completeKeyStatisticsInformation, + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + + while (!target.executorService.isShutdown()) { + Thread.sleep(100); + } + Assert.assertNotNull(target.getError().getMessage().contains(TASK_1)); + + } + + + private void workersWithFailedFetchSequential(Set failedTasks) + { + doAnswer(invocation -> { + ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class); + if (failedTasks.contains((String) invocation.getArgument(0))) { + return Futures.immediateFailedFuture(new Exception("Task fetch failed :" + invocation.getArgument(0))); + } + return Futures.immediateFuture(snapshot); + }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyInt(), anyLong()); + } + + private void workersWithFailedFetchParallel(Set failedTasks) + { + doAnswer(invocation -> { + ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class); + if (failedTasks.contains((String) invocation.getArgument(0))) { + return Futures.immediateFailedFuture(new Exception("Task fetch failed :" + invocation.getArgument(0))); + } + return Futures.immediateFuture(snapshot); + }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); + } + } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.java index b93cebd6fce6..c4552e4afeee 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/StageDefinitionTest.java @@ -19,7 +19,18 @@ package org.apache.druid.msq.kernel; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.SortColumn; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.input.stage.StageInputSpec; +import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollectorImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Assert; import org.junit.Test; public class StageDefinitionTest @@ -32,4 +43,62 @@ public void testEquals() .usingGetClass() .verify(); } + + @Test + public void testGeneratePartitionsForNullShuffle() + { + StageDefinition stageDefinition = new StageDefinition( + new StageId("query", 1), + ImmutableList.of(new StageInputSpec(0)), + ImmutableSet.of(), + new OffsetLimitFrameProcessorFactory(0, 1L), + RowSignature.empty(), + null, + 0, + false + ); + + Assert.assertThrows(ISE.class, () -> stageDefinition.generatePartitionsForShuffle(null)); + } + + @Test + public void testGeneratePartitionsForNonNullShuffleWithNullCollector() + { + StageDefinition stageDefinition = new StageDefinition( + new StageId("query", 1), + ImmutableList.of(new StageInputSpec(0)), + ImmutableSet.of(), + new OffsetLimitFrameProcessorFactory(0, 1L), + RowSignature.empty(), + new MaxCountShuffleSpec(new ClusterBy(ImmutableList.of(new SortColumn("test", false)), 1), 2, false), + 1, + false + ); + + Assert.assertThrows(ISE.class, () -> stageDefinition.generatePartitionsForShuffle(null)); + } + + + @Test + public void testGeneratePartitionsForNonNullShuffleWithNonNullCollector() + { + StageDefinition stageDefinition = new StageDefinition( + new StageId("query", 1), + ImmutableList.of(new StageInputSpec(0)), + ImmutableSet.of(), + new OffsetLimitFrameProcessorFactory(0, 1L), + RowSignature.empty(), + new MaxCountShuffleSpec(new ClusterBy(ImmutableList.of(new SortColumn("test", false)), 0), 1, false), + 1, + false + ); + + Assert.assertThrows( + ISE.class, + () -> stageDefinition.generatePartitionsForShuffle(ClusterByStatisticsCollectorImpl.create(new ClusterBy( + ImmutableList.of(new SortColumn("test", false)), + 1 + ), RowSignature.builder().add("test", ColumnType.STRING).build(), 1000, 100, false, false)) + ); + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 8d3a58dfa93a..7875bfe3df60 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -170,7 +170,6 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -213,7 +212,7 @@ public class MSQTestBase extends BaseCalciteQueryTest { public static final Map DEFAULT_MSQ_CONTEXT = ImmutableMap.builder() - .put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, true) + .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, true) .put(QueryContexts.CTX_SQL_QUERY_ID, "test-query") .put(QueryContexts.FINALIZE_KEY, true) .build(); @@ -229,7 +228,7 @@ public class MSQTestBase extends BaseCalciteQueryTest ImmutableMap.builder() .put(QueryContexts.CTX_SQL_QUERY_ID, "test-query") .put(QueryContexts.FINALIZE_KEY, true) - .put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, false) + .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, false) .build(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index 0153cbc38ed3..916e120f3919 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -34,13 +34,13 @@ import org.junit.internal.matchers.ThrowableMessageMatcher; import javax.annotation.Nullable; - import java.util.Collections; import java.util.List; import java.util.Map; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_DESTINATION; -import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_FAULT_TOLERANCE; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MAX_NUM_TASKS; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MSQ_MODE; @@ -61,7 +61,20 @@ public void isDurableStorageEnabled_noParameterSetReturnsDefaultValue() @Test public void isDurableStorageEnabled_parameterSetReturnsCorrectValue() { - Map propertyMap = ImmutableMap.of(CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, "true"); + Map propertyMap = ImmutableMap.of(CTX_DURABLE_SHUFFLE_STORAGE, "true"); + Assert.assertTrue(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(propertyMap))); + } + + @Test + public void isFaultToleranceEnabled_noParameterSetReturnsDefaultValue() + { + Assert.assertFalse(MultiStageQueryContext.isFaultToleranceEnabled(QueryContext.empty())); + } + + @Test + public void isFaultToleranceEnabled_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of(CTX_FAULT_TOLERANCE, "true"); Assert.assertTrue(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(propertyMap))); } diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java index 74da22b136bd..b15ee58c38e9 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java @@ -122,7 +122,7 @@ public void testMsqIngestionAndQuerying() throws Exception SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask( queryLocal, ImmutableMap.of( - MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, + MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, "true", MultiStageQueryContext.CTX_MAX_NUM_TASKS, 3 From e674cac6d7bdfb9682dde31be1d0582bb672a395 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 9 Dec 2022 20:14:55 +0530 Subject: [PATCH 19/27] Merge things --- docs/multi-stage-query/reference.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 887b2c579845..c5f5a5629972 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -405,8 +405,8 @@ The following table describes error codes you may encounter in the `multiStageQu | `QueryNotSupported` | QueryKit could not translate the provided native query to a multi-stage query.

This can happen if the query uses features that aren't supported, like GROUPING SETS. | | | `RowTooLarge` | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for specific limits on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. | | `TaskStartTimeout` | Unable to launch all the worker tasks in time.

There might be insufficient available slots to start all the worker tasks simultaneously.

Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity. | `numTasks`: The number of tasks attempted to launch. | -| `TooManyAttemptsForJob` | Total relaunch attempt count across all workers exceeded max relaunch attempt limit. See the [Limits](#limits) table for the specific limit. | `maxRelaunchCount`: Max number of relaunches across all the workers defined in the [Limits](#limits) section.

`currentRelaunchCount`: current relaunch counter for the job across all workers.

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| -| `TooManyAttemptsForWorker` | Worker exceeded maximum relaunch attempt count as definied in the [Limits](#limits) section. |`maxPerWorkerRelaunchCount`: Max number of relaunches allower per worker as defined in the [Limits](#limits) section.

`workerNumber`: the worker number for which the task failed

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| +| `TooManyAttemptsForJob` | Total relaunch attempt count across all workers exceeded max relaunch attempt limit. See the [Limits](#limits) table for the specific limit. | `maxRelaunchCount`: Max number of relaunches across all the workers defined in the [Limits](#limits) section.

`currentRelaunchCount`: current relaunch counter for the job across all workers.

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| +| `TooManyAttemptsForWorker` | Worker exceeded maximum relaunch attempt count as definied in the [Limits](#limits) section. |`maxPerWorkerRelaunchCount`: Max number of relaunches allower per worker as defined in the [Limits](#limits) section.

`workerNumber`: the worker number for which the task failed

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| | `TooManyBuckets` | Exceeded the maximum number of partition buckets for a stage (5,000 partition buckets).
< br />Partition buckets are created for each [`PARTITIONED BY`](#partitioned-by) time chunk for INSERT and REPLACE queries. The most common reason for this error is that your `PARTITIONED BY` is too narrow relative to your data. | `maxBuckets`: The limit on partition buckets. | | `TooManyInputFiles` | Exceeded the maximum number of input files or segments per worker (10,000 files or segments).

If you encounter this limit, consider adding more workers, or breaking up your query into smaller queries that process fewer files or segments per query. | `numInputFiles`: The total number of input files/segments for the stage.

`maxInputFiles`: The maximum number of input files/segments per worker per stage.

`minNumWorker`: The minimum number of workers required for a successful run. | | `TooManyPartitions` | Exceeded the maximum number of partitions for a stage (25,000 partitions).

This can occur with INSERT or REPLACE statements that generate large numbers of segments, since each segment is associated with a partition. If you encounter this limit, consider breaking up your INSERT or REPLACE statement into smaller statements that process less data per statement. | `maxPartitions`: The limit on partitions which was exceeded | From 0c22ce8d104bead7e7780f839d155827ed6d10b6 Mon Sep 17 00:00:00 2001 From: Karan Kumar Date: Mon, 9 Jan 2023 11:44:23 +0530 Subject: [PATCH 20/27] Merge things --- .../druid/msq/kernel/controller/ControllerStageTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index 73041a9037c7..ada5a8c36484 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -100,7 +100,7 @@ class ControllerStageTracker // created when is called. Should be cleared once resultPartitionBoundariesforTimeChunk is set private final Map timeChunkToCollector = new HashMap<>(); - private Map timeChunkToBoundaries = new TreeMap<>(); + private final Map timeChunkToBoundaries = new TreeMap<>(); private final Map> timeChunkToWorkers = new HashMap<>(); long totalPartitionCount; From a4f974c42dbed2dbaf1e1b9a2489e5971be83318 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 9 Jan 2023 18:30:39 +0530 Subject: [PATCH 21/27] Adding parameterized tests Created separate module for faultToleranceTests --- .travis.yml | 10 ++ .../controller/ControllerQueryKernel.java | 4 +- .../controller/ControllerStageTracker.java | 27 ++- .../apache/druid/msq/exec/MSQInsertTest.java | 87 +++++++-- .../apache/druid/msq/exec/MSQReplaceTest.java | 166 ++++++++++++++---- .../apache/druid/msq/exec/MSQSelectTest.java | 165 +++++++++++------ .../msq/indexing/error/MSQWarningsTest.java | 4 +- .../apache/druid/msq/test/MSQTestBase.java | 43 +++-- .../MultiStageQuery/docker-compose.yaml | 4 +- .../MultiStageQueryWithMM/docker-compose.yaml | 98 +++++++++++ integration-tests-ex/cases/pom.xml | 9 + .../categories/MultiStageQueryWithMM.java | 24 +++ ...TMultiStageQueryWorkerFaultTolerance.java} | 11 +- 13 files changed, 517 insertions(+), 135 deletions(-) create mode 100644 integration-tests-ex/cases/cluster/MultiStageQueryWithMM/docker-compose.yaml create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/MultiStageQueryWithMM.java rename integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/{ITMultiStageQueryHA.java => ITMultiStageQueryWorkerFaultTolerance.java} (96%) diff --git a/.travis.yml b/.travis.yml index 52a804bee871..003331be2e34 100644 --- a/.travis.yml +++ b/.travis.yml @@ -525,6 +525,16 @@ jobs: env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' script: ./it.sh travis MultiStageQuery + + - &integration_tests_ex + name: "(Compile=openjdk8, Run=openjdk8) multi stage query tests with MM" + stage: Tests - phase 2 + jdk: openjdk8 + services: *integration_test_services + env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' + script: ./it.sh travis MultiStageQueryWithMM + + - &integration_tests_ex name: "(Compile=openjdk8, Run=openjdk8) catalog integration tests" stage: Tests - phase 2 diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 36c2c3cbf206..ddd7b633b398 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -719,7 +719,9 @@ public Map> getStagesAndWorkersToFetchClusterStats() for (StageId stageId : trackedSet) { ControllerStageTracker controllerStageTracker = getStageKernelOrThrow(stageId); - stageToWorkers.put(stageId, controllerStageTracker.getWorkersToFetchClusterStatisticsFrom()); + if (controllerStageTracker.getStageDefinition().mustGatherResultKeyStatistics()) { + stageToWorkers.put(stageId, controllerStageTracker.getWorkersToFetchClusterStatisticsFrom()); + } } return stageToWorkers; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index ada5a8c36484..6494baa08feb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -98,14 +98,16 @@ class ControllerStageTracker @Nullable private ClusterByPartitions resultPartitionBoundaries; - // created when is called. Should be cleared once resultPartitionBoundariesforTimeChunk is set + + // created when mergingStatsForTimeChunk is called. Should be cleared once timeChunkToBoundaries is set for the timechunk private final Map timeChunkToCollector = new HashMap<>(); private final Map timeChunkToBoundaries = new TreeMap<>(); - private final Map> timeChunkToWorkers = new HashMap<>(); long totalPartitionCount; + // states used for tracking worker to timechunks and vice versa so that we know when to generate partition boundaries for (timeChunk,worker) private Map> workerToRemainingTimeChunks = null; + private Map> timeChunkToRemainingWorkers = null; @Nullable private Object resultObject; @@ -430,8 +432,8 @@ ControllerStagePhase addPartialKeyInformationForWorker( if (allPartialKeyInformationFetched()) { completeKeyStatisticsInformation.complete(); - if (workerToRemainingTimeChunks == null) { - initializeWorkerToRemainingTimeChunks(); + if (workerToRemainingTimeChunks == null && timeChunkToRemainingWorkers == null) { + initializeTimeChunkWorkerTrackers(); } // All workers have sent the partial key statistics information. // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers. @@ -464,9 +466,10 @@ ControllerStagePhase addPartialKeyInformationForWorker( return getPhase(); } - private void initializeWorkerToRemainingTimeChunks() + private void initializeTimeChunkWorkerTrackers() { workerToRemainingTimeChunks = new HashMap<>(); + timeChunkToRemainingWorkers = new HashMap<>(); completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().forEach((timeChunk, workers) -> { for (int worker : workers) { this.workerToRemainingTimeChunks.compute(worker, (wk, timeChunks) -> { @@ -477,6 +480,7 @@ private void initializeWorkerToRemainingTimeChunks() return timeChunks; }); } + timeChunkToRemainingWorkers.put(timeChunk, workers); }); } @@ -521,7 +525,7 @@ void mergeClusterByStatisticsCollectorForTimeChunk( throw new ISE("Worker[%d] not found for stage[%s]", workerNumber, stageDef.getStageNumber()); } - // only merge in case this worker has remaining timechunks + // only merge in case this worker has remaining time chunks workerToRemainingTimeChunks.computeIfPresent(workerNumber, (wk, timeChunks) -> { if (timeChunks.remove(timeChunk)) { @@ -538,12 +542,17 @@ void mergeClusterByStatisticsCollectorForTimeChunk( ); // if work for one time chunk is finished, generate the ClusterByPartitions for that timeChunk and clear the collector so that we free up controller memory. - timeChunkToWorkers.compute(timeChunk, (tc, workers) -> { + timeChunkToRemainingWorkers.compute(timeChunk, (tc, workers) -> { if (workers == null || workers.isEmpty()) { - throw new ISE("Workers should not be empty until all the work is finished for time chunk[%d]", timeChunk); + throw new ISE( + "Remaining workers should not be empty until all the work is finished for time chunk[%d] for stage[%d]", + timeChunk, + stageDef.getStageNumber() + ); } workers.remove(workerNumber); if (workers.isEmpty()) { + // generate partition boundaries since all work is finished for the time chunk ClusterByStatisticsCollector collector = timeChunkToCollector.get(tc); Either countOrPartitions = stageDef.generatePartitionsForShuffle(collector); totalPartitionCount += getPartitionCountFromEither(countOrPartitions); @@ -552,6 +561,8 @@ void mergeClusterByStatisticsCollectorForTimeChunk( return null; } timeChunkToBoundaries.put(tc, countOrPartitions.valueOrThrow()); + + // clear the collector to give back memory collector.clear(); timeChunkToCollector.remove(tc); return null; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 35371a928856..8fc08ad1d2ea 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -44,21 +44,46 @@ import org.hamcrest.CoreMatchers; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.mockito.Mockito; import javax.annotation.Nonnull; import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeSet; + +@RunWith(Parameterized.class) public class MSQInsertTest extends MSQTestBase { private final HashFunction fn = Hashing.murmur3_128(); + @Parameterized.Parameters(name = "{index}:with context {0}") + public static Collection data() + { + Object[][] data = new Object[][]{ + {DEFAULT, DEFAULT_MSQ_CONTEXT}, + {DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT}, + {FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT}, + {SEQUENTIAL_MERGE, SEQUENTIAL_MERGE_MSQ_CONTEXT} + }; + return Arrays.asList(data); + } + + @Parameterized.Parameter(0) + public String contextName; + + @Parameterized.Parameter(1) + public Map context; + + @Test public void testInsertOnFoo1() { @@ -70,6 +95,7 @@ public void testInsertOnFoo1() testIngestQuery().setSql( "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") .setExpectedDataSource("foo1") + .setQueryContext(context) .setExpectedRowSignature(rowSignature) .setExpectedSegment(expectedFooSegments()) .setExpectedResultRows(expectedFooRows()) @@ -100,6 +126,7 @@ public void testInsertOnExternalDataSource() throws IOException + ") group by 1 PARTITIONED by day ") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedSegment(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), @@ -129,6 +156,7 @@ public void testInsertOnFoo1WithTimeFunction() "insert into foo1 select floor(__time to day) as __time , dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedSegment(expectedFooSegments()) .setExpectedResultRows(expectedFooRows()) .verifyResults(); @@ -155,6 +183,7 @@ public void testInsertOnFoo1WithTimeFunctionWithSequential() .setQueryContext(context) .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(MSQInsertTest.this.context) .setExpectedSegment(expectedFooSegments()) .setExpectedResultRows(expectedFooRows()) .verifyResults(); @@ -172,6 +201,7 @@ public void testInsertOnFoo1WithMultiValueDim() "INSERT INTO foo1 SELECT dim3 FROM foo WHERE dim3 IS NOT NULL PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedMultiValueFooRows()) .verifyResults(); @@ -188,6 +218,7 @@ public void testInsertOnFoo1WithMultiValueDimGroupBy() "INSERT INTO foo1 SELECT dim3 FROM foo WHERE dim3 IS NOT NULL GROUP BY 1 PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedMultiValueFooRowsGroupBy()) .verifyResults(); @@ -199,6 +230,7 @@ public void testInsertOnFoo1WithMultiValueMeasureGroupBy() testIngestQuery().setSql( "INSERT INTO foo1 SELECT count(dim3) FROM foo WHERE dim3 IS NOT NULL GROUP BY 1 PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") + .setQueryContext(context) .setExpectedValidationErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( @@ -219,6 +251,7 @@ public void testInsertOnFoo1WithMultiValueToArrayGroupBy() "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedMultiValueFooRowsToArray()) .verifyResults(); @@ -227,18 +260,23 @@ public void testInsertOnFoo1WithMultiValueToArrayGroupBy() @Test public void testInsertOnFoo1WithMultiValueDimGroupByWithoutGroupByEnable() { - Map context = ImmutableMap.builder() - .putAll(MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE) - .put("groupByEnableMultiValueUnnesting", false) - .build(); + Map localContext = ImmutableMap.builder() + .putAll(context) + .put("groupByEnableMultiValueUnnesting", false) + .build(); + testIngestQuery().setSql( "INSERT INTO foo1 SELECT dim3, count(*) AS cnt1 FROM foo GROUP BY dim3 PARTITIONED BY ALL TIME") - .setQueryContext(context) + .setQueryContext(localContext) .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(ISE.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.")) + ThrowableMessageMatcher.hasMessage(!FAULT_TOLERANCE.equals(contextName) + ? CoreMatchers.containsString( + "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.") + : + CoreMatchers.containsString("exceeded max relaunch count") + ) )) .verifyExecutionError(); } @@ -254,7 +292,9 @@ public void testRollUpOnFoo1UpOnFoo1() testIngestQuery().setSql( "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") .setExpectedDataSource("foo1") - .setQueryContext(ROLLUP_CONTEXT) + .setQueryContext(new ImmutableMap.Builder().putAll(context) + .putAll(ROLLUP_CONTEXT_PARAMS) + .build()) .setExpectedRollUp(true) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) .setExpectedRowSignature(rowSignature) @@ -272,11 +312,11 @@ public void testRollUpOnFoo1WithTimeFunction() .add("dim1", ColumnType.STRING) .add("cnt", ColumnType.LONG).build(); - testIngestQuery().setSql( "insert into foo1 select floor(__time to day) as __time , dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") .setExpectedDataSource("foo1") - .setQueryContext(ROLLUP_CONTEXT) + .setQueryContext(new ImmutableMap.Builder().putAll(context).putAll( + ROLLUP_CONTEXT_PARAMS).build()) .setExpectedRollUp(true) .setExpectedQueryGranularity(Granularities.DAY) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) @@ -300,7 +340,8 @@ public void testRollUpOnFoo1WithTimeFunctionComplexCol() testIngestQuery().setSql( "insert into foo1 select floor(__time to day) as __time , dim1 , count(distinct m1) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") .setExpectedDataSource("foo1") - .setQueryContext(ROLLUP_CONTEXT) + .setQueryContext(new ImmutableMap.Builder().putAll(context).putAll( + ROLLUP_CONTEXT_PARAMS).build()) .setExpectedRollUp(true) .setExpectedQueryGranularity(Granularities.DAY) .addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true)) @@ -324,7 +365,8 @@ public void testRollUpOnFoo1ComplexCol() testIngestQuery().setSql( "insert into foo1 select __time , dim1 , count(distinct m1) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") .setExpectedDataSource("foo1") - .setQueryContext(ROLLUP_CONTEXT) + .setQueryContext(new ImmutableMap.Builder().putAll(context).putAll( + ROLLUP_CONTEXT_PARAMS).build()) .setExpectedRollUp(true) .addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true)) .setExpectedRowSignature(rowSignature) @@ -355,7 +397,8 @@ public void testRollUpOnExternalDataSource() throws IOException + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + " )\n" + ") group by 1 PARTITIONED by day ") - .setQueryContext(ROLLUP_CONTEXT) + .setQueryContext(new ImmutableMap.Builder().putAll(context).putAll( + ROLLUP_CONTEXT_PARAMS).build()) .setExpectedRollUp(true) .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) @@ -392,7 +435,8 @@ public void testRollUpOnExternalDataSourceWithCompositeKey() throws IOException + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"namespace\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + " )\n" + ") group by 1,2 PARTITIONED by day ") - .setQueryContext(ROLLUP_CONTEXT) + .setQueryContext(new ImmutableMap.Builder().putAll(context).putAll( + ROLLUP_CONTEXT_PARAMS).build()) .setExpectedRollUp(true) .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) @@ -433,6 +477,7 @@ public void testInsertWrongTypeTimestamp() + "CLUSTERED BY dim1") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedValidationErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( @@ -478,6 +523,7 @@ public void testInsertRestrictedColumns() + ") PARTITIONED by day") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedMSQFault(new ColumnNameRestrictedFault("__bucket")) .verifyResults(); } @@ -485,13 +531,13 @@ public void testInsertRestrictedColumns() @Test public void testInsertQueryWithInvalidSubtaskCount() { - Map context = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) - .put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 1) - .build(); + Map localContext = ImmutableMap.builder() + .putAll(context) + .put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 1) + .build(); testIngestQuery().setSql( "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") - .setQueryContext(context) + .setQueryContext(localContext) .setExpectedExecutionErrorMatcher( ThrowableMessageMatcher.hasMessage( CoreMatchers.startsWith( @@ -522,6 +568,7 @@ public void testInsertWithTooLargeRowShouldThrowException() throws IOException + " )\n" + ") group by 1 PARTITIONED by day ") .setExpectedDataSource("foo") + .setQueryContext(context) .setExpectedMSQFault(new RowTooLargeFault(500)) .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(ISE.class), @@ -544,6 +591,7 @@ public void testInsertLimitWithPeriodGranularityThrowsException() ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\"")) )) + .setQueryContext(context) .verifyPlanningErrors(); } @@ -561,6 +609,7 @@ public void testInsertOffsetThrowsException() ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( "INSERT and REPLACE queries cannot have an OFFSET")) )) + .setQueryContext(context) .verifyPlanningErrors(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 1cd8d66bbcbf..af524ace0cc4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -33,18 +33,42 @@ import org.hamcrest.CoreMatchers; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.annotation.Nonnull; import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.TreeSet; +@RunWith(Parameterized.class) public class MSQReplaceTest extends MSQTestBase { + @Parameterized.Parameters(name = "{index}:with context {0}") + public static Collection data() + { + Object[][] data = new Object[][]{ + {DEFAULT, DEFAULT_MSQ_CONTEXT}, + {DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT}, + {FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT}, + {SEQUENTIAL_MERGE, SEQUENTIAL_MERGE_MSQ_CONTEXT} + }; + return Arrays.asList(data); + } + + @Parameterized.Parameter(0) + public String contextName; + + @Parameterized.Parameter(1) + public Map context; + @Test public void testReplaceOnFooWithAll() { @@ -59,6 +83,7 @@ public void testReplaceOnFooWithAll() + "PARTITIONED BY DAY ") .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) .setExpectedSegment( ImmutableSet.of( @@ -91,15 +116,23 @@ public void testReplaceOnFooWithWhere() .add("m1", ColumnType.FLOAT) .build(); - testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE WHERE __time >= TIMESTAMP '2000-01-02' AND __time < TIMESTAMP '2000-01-03' " - + "SELECT __time, m1 " - + "FROM foo " - + "WHERE __time >= TIMESTAMP '2000-01-02' AND __time < TIMESTAMP '2000-01-03' " - + "PARTITIONED by DAY ") + testIngestQuery().setSql( + " REPLACE INTO foo OVERWRITE WHERE __time >= TIMESTAMP '2000-01-02' AND __time < TIMESTAMP '2000-01-03' " + + "SELECT __time, m1 " + + "FROM foo " + + "WHERE __time >= TIMESTAMP '2000-01-02' AND __time < TIMESTAMP '2000-01-03' " + + "PARTITIONED by DAY ") .setExpectedDataSource("foo") - .setExpectedDestinationIntervals(ImmutableList.of(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"))) + .setExpectedDestinationIntervals(ImmutableList.of(Intervals.of( + "2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"))) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-02T/P1D"), "test", 0))) + .setQueryContext(context) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-02T/P1D"), + "test", + 0 + ))) .setExpectedResultRows(ImmutableList.of(new Object[]{946771200000L, 2.0f})) .verifyResults(); } @@ -127,10 +160,27 @@ public void testReplaceOnFoo1WithAllExtern() throws IOException .setExpectedDataSource("foo1") .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedSegment(ImmutableSet.of( - SegmentId.of("foo1", Intervals.of("2016-06-27T00:00:00.000Z/2016-06-27T01:00:00.000Z"), "test", 0), - SegmentId.of("foo1", Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"), "test", 0), - SegmentId.of("foo1", Intervals.of("2016-06-27T02:00:00.000Z/2016-06-27T03:00:00.000Z"), "test", 0)) + SegmentId.of( + "foo1", + Intervals.of("2016-06-27T00:00:00.000Z/2016-06-27T01:00:00.000Z"), + "test", + 0 + ), + SegmentId.of( + "foo1", + Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"), + "test", + 0 + ), + SegmentId.of( + "foo1", + Intervals.of("2016-06-27T02:00:00.000Z/2016-06-27T03:00:00.000Z"), + "test", + 0 + ) + ) ) .setExpectedResultRows( ImmutableList.of( @@ -158,23 +208,31 @@ public void testReplaceOnFoo1WithWhereExtern() throws IOException final File toRead = MSQTestFileUtils.getResourceAsTemporaryFile(this, "/wikipedia-sampled.json"); final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(toRead.getAbsolutePath()); - testIngestQuery().setSql(" REPLACE INTO foo1 OVERWRITE WHERE __time >= TIMESTAMP '2016-06-27 01:00:00.00' AND __time < TIMESTAMP '2016-06-27 02:00:00.00' " - + " SELECT " - + " floor(TIME_PARSE(\"timestamp\") to hour) AS __time, " - + " user " - + "FROM TABLE(\n" - + " EXTERN(\n" - + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" - + " '{\"type\": \"json\"}',\n" - + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" - + " )\n" - + ") " - + "where \"timestamp\" >= TIMESTAMP '2016-06-27 01:00:00.00' AND \"timestamp\" < TIMESTAMP '2016-06-27 02:00:00.00' " - + "PARTITIONED BY HOUR ") + testIngestQuery().setSql( + " REPLACE INTO foo1 OVERWRITE WHERE __time >= TIMESTAMP '2016-06-27 01:00:00.00' AND __time < TIMESTAMP '2016-06-27 02:00:00.00' " + + " SELECT " + + " floor(TIME_PARSE(\"timestamp\") to hour) AS __time, " + + " user " + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + + " )\n" + + ") " + + "where \"timestamp\" >= TIMESTAMP '2016-06-27 01:00:00.00' AND \"timestamp\" < TIMESTAMP '2016-06-27 02:00:00.00' " + + "PARTITIONED BY HOUR ") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) - .setExpectedDestinationIntervals(ImmutableList.of(Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"), "test", 0))) + .setQueryContext(context) + .setExpectedDestinationIntervals(ImmutableList.of(Intervals.of( + "2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo1", + Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"), + "test", + 0 + ))) .setExpectedResultRows( ImmutableList.of( new Object[]{1466989200000L, "2001:DA8:207:E132:94DC:BA03:DFDF:8F9F"}, @@ -197,6 +255,7 @@ public void testReplaceIncorrectSyntax() { testIngestQuery().setSql("REPLACE INTO foo1 OVERWRITE SELECT * FROM foo PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") + .setQueryContext(context) .setExpectedValidationErrorMatcher( CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), @@ -222,8 +281,14 @@ public void testReplaceSegmentEntireTable() + "PARTITIONED BY ALL TIME ") .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) .setExpectedResultRows( ImmutableList.of( new Object[]{946684800000L, 1.0f}, @@ -253,8 +318,14 @@ public void testReplaceSegmentsRepartitionTable() + "PARTITIONED BY MONTH") .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) .setExpectedResultRows( ImmutableList.of( new Object[]{946684800000L, 1.0f}, @@ -266,8 +337,9 @@ public void testReplaceSegmentsRepartitionTable() ) ) .setExpectedSegment(ImmutableSet.of( - SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0), - SegmentId.of("foo", Intervals.of("2001-01-01T/P1M"), "test", 0)) + SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0), + SegmentId.of("foo", Intervals.of("2001-01-01T/P1M"), "test", 0) + ) ) .verifyResults(); } @@ -288,15 +360,26 @@ public void testReplaceWithWhereClause() + "PARTITIONED BY MONTH") .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) .setExpectedResultRows( ImmutableList.of( new Object[]{946684800000L, 1.0f}, new Object[]{946771200000L, 2.0f} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) .verifyResults(); } @@ -316,15 +399,26 @@ public void testReplaceWhereClauseLargerThanData() + "PARTITIONED BY MONTH") .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) .setExpectedResultRows( ImmutableList.of( new Object[]{946684800000L, 1.0f}, new Object[]{946771200000L, 2.0f} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0))) + .setExpectedSegment(ImmutableSet.of(SegmentId.of( + "foo", + Intervals.of("2000-01-01T/P1M"), + "test", + 0 + ))) .verifyResults(); } @@ -337,6 +431,7 @@ public void testReplaceLimitWithPeriodGranularityThrowsException() + "FROM foo " + "LIMIT 50" + "PARTITIONED BY MONTH") + .setQueryContext(context) .setExpectedValidationErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( @@ -360,6 +455,7 @@ public void testReplaceOffsetThrowsException() ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( "INSERT and REPLACE queries cannot have an OFFSET")) )) + .setQueryContext(context) .verifyPlanningErrors(); } @@ -380,6 +476,7 @@ public void testReplaceTimeChunks() .setExpectedDataSource("foo") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) .setExpectedSegment(ImmutableSet.of(SegmentId.of( "foo", @@ -413,6 +510,7 @@ public void testReplaceTimeChunksLargerThanData() .setExpectedDataSource("foo") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T"))) .setExpectedSegment(ImmutableSet.of(SegmentId.of( "foo", @@ -445,6 +543,7 @@ public void testInsertOnFoo1Range() .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedShardSpec(DimensionRangeShardSpec.class) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedSegment(expectedFooSegments()) .setExpectedResultRows(expectedFooRows()) .verifyResults(); @@ -466,6 +565,7 @@ public void testReplaceSegmentsInsertIntoNewTable() + "PARTITIONED BY ALL TIME ") .setExpectedDataSource("foobar") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) .setExpectedSegment(ImmutableSet.of(SegmentId.of("foobar", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index a0d057ce274e..97d4df2eaa29 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -66,6 +66,8 @@ import org.hamcrest.CoreMatchers; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @@ -74,12 +76,33 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +@RunWith(Parameterized.class) public class MSQSelectTest extends MSQTestBase { + + @Parameterized.Parameters(name = "{index}:with context {0}") + public static Collection data() + { + Object[][] data = new Object[][]{ + {DEFAULT, DEFAULT_MSQ_CONTEXT}, + {DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT}, + {FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT}, + {SEQUENTIAL_MERGE, SEQUENTIAL_MERGE_MSQ_CONTEXT} + }; + return Arrays.asList(data); + } + + @Parameterized.Parameter(0) + public String contextName; + + @Parameterized.Parameter(1) + public Map context; + @Test public void testCalculator() { @@ -101,7 +124,7 @@ public void testCalculator() ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") - .context(defaultScanQueryContext(resultSignature)) + .context(defaultScanQueryContext(context, resultSignature)) .build() ) .columnMappings(ColumnMappings.identity(resultSignature)) @@ -109,6 +132,7 @@ public void testCalculator() .build() ) .setExpectedRowSignature(resultSignature) + .setQueryContext(context) .setExpectedResultRows(ImmutableList.of(new Object[]{2L})).verifyResults(); } @@ -129,13 +153,14 @@ public void testSelectOnFoo() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .columns("cnt", "dim1") - .context(defaultScanQueryContext(resultSignature)) + .context(defaultScanQueryContext(context, resultSignature)) .build() ) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .build() ) + .setQueryContext(context) .setExpectedRowSignature(resultSignature) .setExpectedResultRows(ImmutableList.of( new Object[]{1L, !useDefault ? "" : null}, @@ -164,6 +189,7 @@ public void testSelectOnFoo2() .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2", "m1") .context(defaultScanQueryContext( + context, RowSignature.builder() .add("dim2", ColumnType.STRING) .add("m1", ColumnType.LONG) @@ -175,6 +201,7 @@ public void testSelectOnFoo2() .build() ) .setExpectedRowSignature(resultSignature) + .setQueryContext(context) .setExpectedResultRows(ImmutableList.of( new Object[]{1L, "en"}, new Object[]{1L, "ru"}, @@ -207,7 +234,7 @@ public void testGroupByOnFoo() )) .setAggregatorSpecs(aggregators(new CountAggregatorFactory( "a0"))) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build()) .columnMappings( new ColumnMappings(ImmutableList.of( @@ -219,6 +246,7 @@ public void testGroupByOnFoo() .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) + .setQueryContext(context) .verifyResults(); } @@ -249,7 +277,7 @@ public void testGroupByOrderByDimension() null ) ) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -266,6 +294,7 @@ public void testGroupByOrderByDimension() .tuningConfig(MSQTuningConfig.defaultConfig()) .build()) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedResultRows( ImmutableList.of( new Object[]{6f, 1L}, @@ -294,13 +323,13 @@ public void testSubquery() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT))) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build() ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -314,6 +343,7 @@ public void testSubquery() ) .setExpectedRowSignature(resultSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{6L})) + .setQueryContext(context) .verifyResults(); } @@ -353,6 +383,7 @@ public void testJoin() .columns("dim2", "m1", "m2") .context( defaultScanQueryContext( + context, RowSignature.builder() .add("dim2", ColumnType.STRING) .add("m1", ColumnType.FLOAT) @@ -371,6 +402,7 @@ public void testJoin() .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context( defaultScanQueryContext( + context, RowSignature.builder().add("m1", ColumnType.FLOAT).build() ) ) @@ -419,7 +451,7 @@ public void testJoin() ) ) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -442,6 +474,7 @@ public void testJoin() ) .setExpectedRowSignature(resultSignature) .setExpectedResultRows(expectedResults) + .setQueryContext(context) .verifyResults(); } @@ -482,6 +515,7 @@ public void testBroadcastJoin() .columns("dim2", "m1", "m2") .context( defaultScanQueryContext( + context, RowSignature.builder() .add("dim2", ColumnType.STRING) .add("m1", ColumnType.FLOAT) @@ -535,7 +569,7 @@ public void testBroadcastJoin() ) ) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -558,6 +592,7 @@ public void testBroadcastJoin() ) .setExpectedRowSignature(resultSignature) .setExpectedResultRows(expectedResults) + .setQueryContext(context) .verifyResults(); } @@ -588,7 +623,7 @@ public void testGroupByOrderByAggregation() null ) ) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -608,6 +643,7 @@ public void testGroupByOrderByAggregation() .build() ) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedResultRows( ImmutableList.of( new Object[]{6f, 6d}, @@ -647,7 +683,7 @@ public void testGroupByOrderByAggregationWithLimit() 3 ) ) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -667,6 +703,7 @@ public void testGroupByOrderByAggregationWithLimit() .build() ) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedResultRows( ImmutableList.of( new Object[]{6f, 6d}, @@ -704,7 +741,7 @@ public void testGroupByOrderByAggregationWithLimitAndOffset() 2 ) ) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -724,6 +761,7 @@ public void testGroupByOrderByAggregationWithLimitAndOffset() .build() ) .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedResultRows( ImmutableList.of( new Object[]{5f, 5d}, @@ -768,7 +806,7 @@ public void testExternSelect1() throws IOException ) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() @@ -783,6 +821,7 @@ public void testExternSelect1() throws IOException + " )\n" + ") group by 1") .setExpectedRowSignature(rowSignature) + .setQueryContext(context) .setExpectedResultRows(ImmutableList.of(new Object[]{1466985600000L, 20L})) .setExpectedMSQSpec( MSQSpec @@ -815,6 +854,7 @@ public void testIncorrectSelectQuery() CoreMatchers.instanceOf(SqlPlanningException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Encountered \"from \"")) )) + .setQueryContext(context) .verifyPlanningErrors(); } @@ -823,6 +863,7 @@ public void testSelectOnInformationSchemaSource() { testSelectQuery() .setSql("SELECT * FROM INFORMATION_SCHEMA.SCHEMATA") + .setQueryContext(context) .setExpectedValidationErrorMatcher( CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), @@ -838,6 +879,7 @@ public void testSelectOnSysSource() { testSelectQuery() .setSql("SELECT * FROM sys.segments") + .setQueryContext(context) .setExpectedValidationErrorMatcher( CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), @@ -853,6 +895,7 @@ public void testSelectOnSysSourceWithJoin() { testSelectQuery() .setSql("select s.segment_id, s.num_rows, f.dim1 from sys.segments as s, foo as f") + .setQueryContext(context) .setExpectedValidationErrorMatcher( CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), @@ -869,6 +912,7 @@ public void testSelectOnSysSourceContainingWith() testSelectQuery() .setSql("with segment_source as (SELECT * FROM sys.segments) " + "select segment_source.segment_id, segment_source.num_rows from segment_source") + .setQueryContext(context) .setExpectedValidationErrorMatcher( CoreMatchers.allOf( CoreMatchers.instanceOf(SqlPlanningException.class), @@ -899,6 +943,7 @@ public void testSelectOnUserDefinedSourceContainingWith() .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2", "m1") .context(defaultScanQueryContext( + context, RowSignature.builder() .add("dim2", ColumnType.STRING) .add("m1", ColumnType.LONG) @@ -911,6 +956,7 @@ public void testSelectOnUserDefinedSourceContainingWith() .build() ) .setExpectedRowSignature(resultSignature) + .setQueryContext(context) .setExpectedResultRows(ImmutableList.of( new Object[]{1L, "en"}, new Object[]{1L, "ru"}, @@ -933,12 +979,13 @@ public void testScanWithMultiValueSelectQuery() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim3") - .context(defaultScanQueryContext(resultSignature)) + .context(defaultScanQueryContext(context, resultSignature)) .build()) .columnMappings(ColumnMappings.identity(resultSignature)) .tuningConfig(MSQTuningConfig.defaultConfig()) .build()) .setExpectedRowSignature(resultSignature) + .setQueryContext(context) .setExpectedResultRows(ImmutableList.of( new Object[]{ImmutableList.of("a", "b")}, new Object[]{ImmutableList.of("b", "c")}, @@ -952,10 +999,7 @@ public void testScanWithMultiValueSelectQuery() @Test public void testGroupByWithMultiValue() { - Map context = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) - .put("groupByEnableMultiValueUnnesting", true) - .build(); + Map localContext = enableMultiValueUnnesting(context, true); RowSignature rowSignature = RowSignature.builder() .add("dim3", ColumnType.STRING) .add("cnt1", ColumnType.LONG) @@ -963,7 +1007,7 @@ public void testGroupByWithMultiValue() testSelectQuery() .setSql("select dim3, count(*) as cnt1 from foo group by dim3") - .setQueryContext(context) + .setQueryContext(localContext) .setExpectedMSQSpec( MSQSpec.builder() .query( @@ -981,7 +1025,7 @@ public void testGroupByWithMultiValue() ) .setAggregatorSpecs(aggregators(new CountAggregatorFactory( "a0"))) - .setContext(context) + .setContext(localContext) .build() ) .columnMappings( @@ -1003,18 +1047,20 @@ public void testGroupByWithMultiValue() @Test public void testGroupByWithMultiValueWithoutGroupByEnable() { - Map context = ImmutableMap.builder() - .putAll(MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE) - .put("groupByEnableMultiValueUnnesting", false) - .build(); + Map localContext = enableMultiValueUnnesting(context, false); testSelectQuery() .setSql("select dim3, count(*) as cnt1 from foo group by dim3") - .setQueryContext(context) + .setQueryContext(localContext) .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(ISE.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.")) + ThrowableMessageMatcher.hasMessage( + !FAULT_TOLERANCE.equals(contextName) + ? CoreMatchers.containsString( + "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.") + : + CoreMatchers.containsString("exceeded max relaunch count") + ) )) .verifyExecutionError(); } @@ -1022,10 +1068,7 @@ public void testGroupByWithMultiValueWithoutGroupByEnable() @Test public void testGroupByWithMultiValueMvToArray() { - Map context = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) - .put("groupByEnableMultiValueUnnesting", true) - .build(); + Map localContext = enableMultiValueUnnesting(context, true); RowSignature rowSignature = RowSignature.builder() .add("EXPR$0", ColumnType.STRING_ARRAY) @@ -1034,7 +1077,7 @@ public void testGroupByWithMultiValueMvToArray() testSelectQuery() .setSql("select MV_TO_ARRAY(dim3), count(*) as cnt1 from foo group by dim3") - .setQueryContext(context) + .setQueryContext(localContext) .setExpectedMSQSpec(MSQSpec.builder() .query(GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) @@ -1058,7 +1101,7 @@ public void testGroupByWithMultiValueMvToArray() ) ) ) - .setContext(context) + .setContext(localContext) .build() ) .columnMappings( @@ -1079,10 +1122,7 @@ public void testGroupByWithMultiValueMvToArray() @Test public void testGroupByArrayWithMultiValueMvToArray() { - Map context = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) - .put("groupByEnableMultiValueUnnesting", true) - .build(); + Map localContext = enableMultiValueUnnesting(context, true); RowSignature rowSignature = RowSignature.builder() .add("EXPR$0", ColumnType.STRING_ARRAY) @@ -1102,7 +1142,7 @@ public void testGroupByArrayWithMultiValueMvToArray() testSelectQuery() .setSql("select MV_TO_ARRAY(dim3), count(*) as cnt1 from foo group by MV_TO_ARRAY(dim3)") - .setQueryContext(context) + .setQueryContext(localContext) .setExpectedMSQSpec(MSQSpec.builder() .query(GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) @@ -1126,7 +1166,7 @@ public void testGroupByArrayWithMultiValueMvToArray() ) ) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(context) + .setContext(localContext) .build() ) .columnMappings( @@ -1144,21 +1184,24 @@ public void testGroupByArrayWithMultiValueMvToArray() .verifyResults(); } + @Test public void testGroupByWithMultiValueMvToArrayWithoutGroupByEnable() { - Map context = ImmutableMap.builder() - .putAll(MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE) - .put("groupByEnableMultiValueUnnesting", false) - .build(); + Map localContext = enableMultiValueUnnesting(context, false); testSelectQuery() .setSql("select MV_TO_ARRAY(dim3), count(*) as cnt1 from foo group by dim3") - .setQueryContext(context) + .setQueryContext(localContext) .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(ISE.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.")) + ThrowableMessageMatcher.hasMessage( + !FAULT_TOLERANCE.equals(contextName) + ? CoreMatchers.containsString( + "Encountered multi-value dimension [dim3] that cannot be processed with 'groupByEnableMultiValueUnnesting' set to false.") + : + CoreMatchers.containsString("exceeded max relaunch count") + ) )) .verifyExecutionError(); } @@ -1186,12 +1229,12 @@ public void testGroupByMultiValueMeasureQuery() ) ) ) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build(); testSelectQuery() .setSql("select __time, count(dim3) as cnt1 from foo group by __time") - .setQueryContext(DEFAULT_MSQ_CONTEXT) + .setQueryContext(context) .setExpectedMSQSpec(MSQSpec.builder() .query(expectedQuery) .columnMappings( @@ -1224,8 +1267,10 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio .add("cnt1", ColumnType.LONG) .build(); + testSelectQuery() .setSql("select cnt,count(*) as cnt1 from foo group by cnt") + .setQueryContext(context) .setExpectedMSQSpec(MSQSpec.builder() .query(GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) @@ -1241,7 +1286,7 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio )) .setAggregatorSpecs(aggregators(new CountAggregatorFactory( "a0"))) - .setContext(DEFAULT_MSQ_CONTEXT) + .setContext(context) .build()) .columnMappings( new ColumnMappings(ImmutableList.of( @@ -1254,13 +1299,15 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) .verifyResults(); - File successFile = new File( - localFileStorageDir, - DurableStorageUtils.getSuccessFilePath("query-test-query", 0, 0) - ); + if (DURABLE_STORAGE.equals(contextName) || !FAULT_TOLERANCE.equals(contextName)) { + File successFile = new File( + localFileStorageDir, + DurableStorageUtils.getSuccessFilePath("query-test-query", 0, 0) + ); - Mockito.verify(localFileStorageConnector, Mockito.times(2)) - .write(ArgumentMatchers.endsWith("__success")); + Mockito.verify(localFileStorageConnector, Mockito.times(2)) + .write(ArgumentMatchers.endsWith("__success")); + } } @Test @@ -1318,6 +1365,7 @@ public void testMultiValueStringWithIncorrectType() throws IOException .build()) .setExpectedMSQFault(new CannotParseExternalDataFault( "Unable to add the row to the frame. Type conversion might be required.")) + .setQueryContext(context) .verifyResults(); } @@ -1354,4 +1402,13 @@ private List expectedMultiValueFooRowsGroupByList() )); return expected; } + + private static Map enableMultiValueUnnesting(Map context, boolean value) + { + Map localContext = ImmutableMap.builder() + .putAll(context) + .put("groupByEnableMultiValueUnnesting", value) + .build(); + return localContext; + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java index 7ec2766b5967..d80295dbfc5b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java @@ -335,7 +335,9 @@ public void testDefaultStrictMode() + " '[{\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n" + " )\n" + ") group by 1 PARTITIONED by day ") - .setQueryContext(ROLLUP_CONTEXT) + .setQueryContext(new ImmutableMap.Builder().putAll(DEFAULT_MSQ_CONTEXT) + .putAll(ROLLUP_CONTEXT_PARAMS) + .build()) .setExpectedRollUp(true) .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 9ed04aafa22d..c9e6bdb21dfb 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -72,6 +72,7 @@ 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.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.guice.MSQDurableStorageModule; @@ -216,25 +217,41 @@ public class MSQTestBase extends BaseCalciteQueryTest { public static final Map DEFAULT_MSQ_CONTEXT = ImmutableMap.builder() - .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, true) .put(QueryContexts.CTX_SQL_QUERY_ID, "test-query") .put(QueryContexts.FINALIZE_KEY, true) .build(); - public static final Map - ROLLUP_CONTEXT = ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) - .put(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false) - .put(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false) - .build(); + public static final Map DURABLE_STORAGE_MSQ_CONTEXT = + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, true).build(); + - public static final Map MSQ_CONTEXT_WITHOUT_DURABLE_STORAGE = + public static final Map FAULT_TOLERANCE_MSQ_CONTEXT = ImmutableMap.builder() - .put(QueryContexts.CTX_SQL_QUERY_ID, "test-query") - .put(QueryContexts.FINALIZE_KEY, true) - .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, false) + .putAll(DEFAULT_MSQ_CONTEXT) + .put(MultiStageQueryContext.CTX_FAULT_TOLERANCE, true).build(); + + public static final Map SEQUENTIAL_MERGE_MSQ_CONTEXT = + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put( + MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, + ClusterStatisticsMergeMode.SEQUENTIAL.toString() + ) .build(); + public static final Map + ROLLUP_CONTEXT_PARAMS = ImmutableMap.builder() + .put(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false) + .put(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false) + .build(); + + public static final String FAULT_TOLERANCE = "fault_tolerance"; + public static final String DURABLE_STORAGE = "durable_storage"; + public static final String DEFAULT = "default"; + public static final String SEQUENTIAL_MERGE = "sequential_merge"; + public final boolean useDefault = NullHandling.replaceWithDefault(); @@ -489,11 +506,11 @@ public String getFormatString() * Returns query context expected for a scan query. Same as {@link #DEFAULT_MSQ_CONTEXT}, but * includes {@link DruidQuery#CTX_SCAN_SIGNATURE}. */ - protected Map defaultScanQueryContext(final RowSignature signature) + protected Map defaultScanQueryContext(Map context, final RowSignature signature) { try { return ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) + .putAll(context) .put( DruidQuery.CTX_SCAN_SIGNATURE, queryFramework().queryJsonMapper().writeValueAsString(signature) diff --git a/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml b/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml index 2ce345063ac1..da658b25db35 100644 --- a/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml +++ b/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.yaml @@ -82,10 +82,10 @@ services: depends_on: - zookeeper - middlemanager: + indexer: extends: file: ../Common/druid.yaml - service: middlemanager + service: indexer environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - druid_msq_intermediate_storage_enable=true diff --git a/integration-tests-ex/cases/cluster/MultiStageQueryWithMM/docker-compose.yaml b/integration-tests-ex/cases/cluster/MultiStageQueryWithMM/docker-compose.yaml new file mode 100644 index 000000000000..2ce345063ac1 --- /dev/null +++ b/integration-tests-ex/cases/cluster/MultiStageQueryWithMM/docker-compose.yaml @@ -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. + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 + +services: + zookeeper: + extends: + file: ../Common/dependencies.yaml + service: zookeeper + + metadata: + extends: + file: ../Common/dependencies.yaml + service: metadata + + coordinator: + extends: + file: ../Common/druid.yaml + service: coordinator + container_name: coordinator + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_manager_segments_pollDuration=PT5S + - druid_coordinator_period=PT10S + depends_on: + - zookeeper + - metadata + + overlord: + extends: + file: ../Common/druid.yaml + service: overlord + container_name: overlord + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + - metadata + + broker: + extends: + file: ../Common/druid.yaml + service: broker + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + router: + extends: + file: ../Common/druid.yaml + service: router + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + historical: + extends: + file: ../Common/druid.yaml + service: historical + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + middlemanager: + extends: + file: ../Common/druid.yaml + service: middlemanager + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_msq_intermediate_storage_enable=true + - druid_msq_intermediate_storage_type=local + - druid_msq_intermediate_storage_basePath=/shared/durablestorage/ + volumes: + # Test data + - ../../resources:/resources + depends_on: + - zookeeper diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 01f56749b017..fecb0842df13 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -328,6 +328,15 @@ MultiStageQuery + + IT-MultiStageQueryWithMM + + false + + + MultiStageQueryWithMM + + IT-Catalog diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/MultiStageQueryWithMM.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/MultiStageQueryWithMM.java new file mode 100644 index 000000000000..97725c72255e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/MultiStageQueryWithMM.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.categories; + +public class MultiStageQueryWithMM +{ +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java similarity index 96% rename from integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java rename to integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java index b15ee58c38e9..e31083098874 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryHA.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java @@ -34,7 +34,7 @@ import org.apache.druid.testing.utils.DataLoaderHelper; import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.MsqTestQueryHelper; -import org.apache.druid.testsEx.categories.MultiStageQuery; +import org.apache.druid.testsEx.categories.MultiStageQueryWithMM; import org.apache.druid.testsEx.config.DruidTestRunner; import org.apache.druid.testsEx.utils.DruidClusterAdminClient; import org.junit.Assert; @@ -42,11 +42,14 @@ import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; +/** + * As we need to kill the PID of the launched task, these tests should be run with middle manager only. + */ @RunWith(DruidTestRunner.class) -@Category(MultiStageQuery.class) -public class ITMultiStageQueryHA +@Category(MultiStageQueryWithMM.class) +public class ITMultiStageQueryWorkerFaultTolerance { - private static final Logger LOG = new Logger(ITMultiStageQueryHA.class); + private static final Logger LOG = new Logger(ITMultiStageQueryWorkerFaultTolerance.class); @Inject private MsqTestQueryHelper msqHelper; From 87d39dcc22856c5fa3a2a412816990c3167ed8bf Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 9 Jan 2023 18:31:31 +0530 Subject: [PATCH 22/27] Adding missed files --- .../apache/druid/java/util/common/function/TriConsumer.java | 4 ++-- .../java/org/apache/druid/msq/exec/WorkerSketchFetcher.java | 1 - .../test/java/org/apache/druid/msq/exec/WorkerImplTest.java | 4 ++-- .../org/apache/druid/msq/indexing/WorkerChatHandlerTest.java | 2 +- 4 files changed, 5 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java b/core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java index 01d2ac5aebfb..318cad23addb 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java +++ b/core/src/main/java/org/apache/druid/java/util/common/function/TriConsumer.java @@ -37,14 +37,14 @@ public interface TriConsumer void accept(T t, U u, V v); /** - * Returns a composed {@code BiConsumer} that performs, in sequence, this + * Returns a composed {@code TriConsumer} that performs, in sequence, this * operation followed by the {@code after} operation. If performing either * operation throws an exception, it is relayed to the caller of the * composed operation. If performing this operation throws an exception, * the {@code after} operation will not be performed. * * @param after the operation to perform after this operation - * @return a composed {@code BiConsumer} that performs in sequence this + * @return a composed {@code TriConsumer} that performs in sequence this * operation followed by the {@code after} operation * @throws NullPointerException if {@code after} is null */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java index 0adf5bd48435..4a072e9bca93 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -40,7 +40,6 @@ import javax.annotation.Nullable; import java.util.Set; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReference; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java index 52231a116b6a..a6f67da0a9bc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java @@ -40,14 +40,14 @@ public class WorkerImplTest @Test public void testFetchStatsThrows() { - WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>()), workerContext); + WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0), workerContext); Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshot(new StageId("xx", 1))); } @Test public void testFetchStatsWithTimeChunkThrows() { - WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>()), workerContext); + WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0), workerContext); Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshotForTimeChunk(new StageId("xx", 1), 1L)); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java index 5b9d6e497aa0..9b148ac25489 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java @@ -158,7 +158,7 @@ public String id() @Override public MSQWorkerTask task() { - return new MSQWorkerTask("controller", "ds", 1, new HashMap<>()); + return new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0); } @Override From 2de05f88ea3cc8b7bb66f5b3e5cfe23a583ea4a2 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 9 Jan 2023 19:55:31 +0530 Subject: [PATCH 23/27] Review comments and fixing tests. --- .../src/main/java/org/apache/druid/msq/exec/Controller.java | 2 +- .../java/org/apache/druid/msq/exec/ControllerClient.java | 2 +- .../main/java/org/apache/druid/msq/exec/ControllerImpl.java | 5 ++--- .../src/main/java/org/apache/druid/msq/exec/MSQTasks.java | 6 ++++-- .../src/main/java/org/apache/druid/msq/exec/WorkerImpl.java | 2 +- .../apache/druid/msq/indexing/ControllerChatHandler.java | 5 +++-- .../apache/druid/msq/indexing/IndexerControllerClient.java | 5 +++-- .../druid/msq/kernel/controller/ControllerStageTracker.java | 5 +++++ .../test/java/org/apache/druid/msq/exec/MSQSelectTest.java | 2 +- .../test/java/org/apache/druid/msq/exec/MSQTasksTest.java | 1 + .../org/apache/druid/msq/test/MSQTestControllerClient.java | 4 ++-- .../apache/druid/msq/util/MultiStageQueryContextTest.java | 2 +- 12 files changed, 25 insertions(+), 16 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java index fe010b219e49..3552459f2359 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java @@ -104,7 +104,7 @@ public String getId() /** * Periodic update of {@link CounterSnapshots} from subtasks. */ - void updateCounters(CounterSnapshotsTree snapshotsTree); + void updateCounters(String taskId, CounterSnapshotsTree snapshotsTree); /** * Reports that results are ready for a subtask. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java index 9d197b76dd17..b3675f0e0476 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java @@ -47,7 +47,7 @@ void postPartialKeyStatistics( * Client-side method to update the controller with counters for a particular stage and worker. The controller uses * this to compile live reports, track warnings generated etc. */ - void postCounters(CounterSnapshotsTree snapshotsTree) throws IOException; + void postCounters(String workerId, CounterSnapshotsTree snapshotsTree) throws IOException; /** * Client side method to update the controller with the result object for a particular stage and worker. This also diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index a89fe4cf4060..ae89c9d7f477 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -709,7 +709,6 @@ public void updatePartialKeyStatisticsInformation( @Override public void workerError(MSQErrorReport errorReport) { - // move inside kernel if (workerTaskLauncher.isTaskCanceledByController(errorReport.getTaskId()) || !workerTaskLauncher.isTaskLatest(errorReport.getTaskId())) { log.info("Ignoring task %s", errorReport.getTaskId()); @@ -749,7 +748,7 @@ public void workerWarning(List errorReports) * Periodic update of {@link CounterSnapshots} from subtasks. */ @Override - public void updateCounters(CounterSnapshotsTree snapshotsTree) + public void updateCounters(String taskId, CounterSnapshotsTree snapshotsTree) { taskCountersForLiveReports.putAll(snapshotsTree); Optional> warningsExceeded = @@ -761,7 +760,7 @@ public void updateCounters(CounterSnapshotsTree snapshotsTree) Long limit = warningsExceeded.get().rhs; workerError(MSQErrorReport.fromFault( - id(), + taskId, selfDruidNode.getHost(), null, new TooManyWarningsFault(limit.intValue(), errorCode) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java index 890f5d0c60b1..e9bf8d92cb3b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java @@ -58,7 +58,9 @@ public class MSQTasks private static final String TASK_ID_PREFIX = "query-"; - private static final Pattern WORKER_PATTERN = Pattern.compile(".*-worker([0-9]+)_[0-9]+"); + private static final String WORKER_NUMBER = "workerNumber"; + // taskids are in the form 12dsa1-worker9_0. see method workerTaskId() for more details. + private static final Pattern WORKER_PATTERN = Pattern.compile(".*-worker(?<" + WORKER_NUMBER + ">[0-9]+)_[0-9]+"); /** * Returns a controller task ID given a SQL query id. @@ -83,7 +85,7 @@ public static int workerFromTaskId(final String taskId) { final Matcher matcher = WORKER_PATTERN.matcher(taskId); if (matcher.matches()) { - return Integer.parseInt(matcher.group(1)); + return Integer.parseInt(matcher.group(WORKER_NUMBER)); } else { throw new ISE( "Desired pattern %s to extract worker from task id %s did not match ", diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index c543e10c5e5c..b1fa1dfd6aaf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -736,7 +736,7 @@ private void postCountersToController() throws IOException final CounterSnapshotsTree snapshotsTree = getCounters(); if (controllerAlive && !snapshotsTree.isEmpty()) { - controllerClient.postCounters(snapshotsTree); + controllerClient.postCounters(id(), snapshotsTree); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java index 8fa04ce6d90b..5b36117d3f40 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java @@ -128,16 +128,17 @@ public Response httpPostWorkerWarning( * See {@link ControllerClient#postCounters} for the client-side code that calls this API. */ @POST - @Path("/counters") + @Path("/counters/{taskId}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) public Response httpPostCounters( + @PathParam("taskId") final String taskId, final CounterSnapshotsTree snapshotsTree, @Context final HttpServletRequest req ) { ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - controller.updateCounters(snapshotsTree); + controller.updateCounters(taskId, snapshotsTree); return Response.status(Response.Status.OK).build(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java index 1ff5952a8958..00302ebfc578 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java @@ -80,10 +80,11 @@ public void postPartialKeyStatistics( } @Override - public void postCounters(CounterSnapshotsTree snapshotsTree) throws IOException + public void postCounters(String workerId, CounterSnapshotsTree snapshotsTree) throws IOException { + final String path = StringUtils.format("/counters/%s", StringUtils.urlEncode(workerId)); doRequest( - new RequestBuilder(HttpMethod.POST, "/counters") + new RequestBuilder(HttpMethod.POST, path) .jsonContent(jsonMapper, snapshotsTree), IgnoreHttpResponseHandler.INSTANCE ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index 6494baa08feb..c57efd4cd47d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -77,6 +77,11 @@ class ControllerStageTracker private final WorkerInputs workerInputs; // worker-> workerStagePhase + // Controller keeps track of the stage with this map. + // Currently, we rely on the serial nature of the state machine to keep things in sync between the controller and the worker. + // So the worker state in the controller can go out of sync with the actual worker state. + + private final Int2ObjectMap workerToPhase = new Int2ObjectOpenHashMap<>(); // workers which have reported partial key information. diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 97d4df2eaa29..1cbab7decf7a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -1299,7 +1299,7 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio .setExpectedRowSignature(rowSignature) .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) .verifyResults(); - if (DURABLE_STORAGE.equals(contextName) || !FAULT_TOLERANCE.equals(contextName)) { + if (DURABLE_STORAGE.equals(contextName) || FAULT_TOLERANCE.equals(contextName)) { File successFile = new File( localFileStorageDir, DurableStorageUtils.getSuccessFilePath("query-test-query", 0, 0) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java index f84bafec7672..cc087092103e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java @@ -155,6 +155,7 @@ public void test_getWorkerFromTaskId() Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worker0-")); Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worr1_0")); Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xxxx-worker-1-0")); + Assert.assertThrows(ISE.class, () -> MSQTasks.workerFromTaskId("xx")); } @Test diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java index 06b0bae29690..3e78e477bda9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java @@ -55,10 +55,10 @@ public void postPartialKeyStatistics( } @Override - public void postCounters(CounterSnapshotsTree snapshotsTree) + public void postCounters(String workerId, CounterSnapshotsTree snapshotsTree) { if (snapshotsTree != null) { - controller.updateCounters(snapshotsTree); + controller.updateCounters(workerId, snapshotsTree); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index 916e120f3919..347bc47c5c49 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -75,7 +75,7 @@ public void isFaultToleranceEnabled_noParameterSetReturnsDefaultValue() public void isFaultToleranceEnabled_parameterSetReturnsCorrectValue() { Map propertyMap = ImmutableMap.of(CTX_FAULT_TOLERANCE, "true"); - Assert.assertTrue(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(propertyMap))); + Assert.assertTrue(MultiStageQueryContext.isFaultToleranceEnabled(QueryContext.of(propertyMap))); } @Test From 85843de727d3b9155dfaa1805d31bfaae087b1c2 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 9 Jan 2023 21:22:36 +0530 Subject: [PATCH 24/27] Documentation things. --- docs/multi-stage-query/reference.md | 6 ++++-- .../src/main/java/org/apache/druid/msq/exec/MSQTasks.java | 8 +++++++- .../org/apache/druid/msq/exec/WorkerSketchFetcher.java | 2 +- .../msq/kernel/controller/ControllerStageTracker.java | 4 ++-- .../java/org/apache/druid/msq/exec/MSQSelectTest.java | 2 +- 5 files changed, 15 insertions(+), 7 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index e593b1eea4c4..e8d6b31c398f 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -322,6 +322,8 @@ The following table lists the context parameters for the MSQ task engine: | `rowsPerSegment` | INSERT or REPLACE

The number of rows per segment to target. The actual number of rows per segment may be somewhat higher or lower than this number. In most cases, use the default. For general information about sizing rows per segment, see [Segment Size Optimization](../operations/segment-optimization.md). | 3,000,000 | | `indexSpec` | INSERT or REPLACE

An [`indexSpec`](../ingestion/ingestion-spec.md#indexspec) to use when generating segments. May be a JSON string or object. See [Front coding](../ingestion/ingestion-spec.md#front-coding) for details on configuring an `indexSpec` with front coding. | See [`indexSpec`](../ingestion/ingestion-spec.md#indexspec). | | `clusterStatisticsMergeMode` | Whether to use parallel or sequential mode for merging of the worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. See [Sketch Merging Mode](#sketch-merging-mode) for more information. | `PARALLEL` | +| `durableShuffleStorage` | SELECT, INSERT, REPLACE

Whether to use durable storage for shuffle mesh. To use this feature, configure the durable storage at the server level using `druid.msq.intermediate.storage.enable=true`). If these properties are not configured, any query with the context variable `durableShuffleStorage=true` fails with a configuration error.

| `false` | +| `faultTolerance` | SELECT, INSERT, REPLACE

Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` | ## Sketch Merging Mode This section details the advantages and performance of various Cluster By Statistics Merge Modes. @@ -375,7 +377,7 @@ The following table lists query limits: | Number of cluster by columns that can appear in a stage | 1,500 | [`TooManyClusteredByColumns`](#error_TooManyClusteredByColumns) | | Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | [`TooManyWorkers`](#error_TooManyWorkers) | | Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](concepts.md#memory-usage). | [`BroadcastTablesTooLarge`](#error_BroadcastTablesTooLarge) | -| Maximum relaunch attempts per worker. Initial run is not a relaunch. The worker will be spawned 1 + workerRelaunchLimit times before erroring out. | 2 | `TooManyAttemptsForWorker` | +| Maximum relaunch attempts per worker. Initial run is not a relaunch. The worker will be spawned 1 + `workerRelaunchLimit` times before the job fails. | 2 | `TooManyAttemptsForWorker` | | Maximum relaunch attempts for a job across all workers. | 100 | `TooManyAttemptsForJob` | @@ -402,7 +404,7 @@ The following table describes error codes you may encounter in the `multiStageQu | `RowTooLarge` | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for specific limits on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. | | `TaskStartTimeout` | Unable to launch all the worker tasks in time.

There might be insufficient available slots to start all the worker tasks simultaneously.

Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity. | `numTasks`: The number of tasks attempted to launch. | | `TooManyAttemptsForJob` | Total relaunch attempt count across all workers exceeded max relaunch attempt limit. See the [Limits](#limits) table for the specific limit. | `maxRelaunchCount`: Max number of relaunches across all the workers defined in the [Limits](#limits) section.

`currentRelaunchCount`: current relaunch counter for the job across all workers.

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| -| `TooManyAttemptsForWorker` | Worker exceeded maximum relaunch attempt count as definied in the [Limits](#limits) section. |`maxPerWorkerRelaunchCount`: Max number of relaunches allower per worker as defined in the [Limits](#limits) section.

`workerNumber`: the worker number for which the task failed

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| +| `TooManyAttemptsForWorker` | Worker exceeded maximum relaunch attempt count as defined in the [Limits](#limits) section. |`maxPerWorkerRelaunchCount`: Max number of relaunches allowed per worker as defined in the [Limits](#limits) section.

`workerNumber`: the worker number for which the task failed

`taskId`: Latest task id which failed

`rootErrorMessage`: Error message of the latest failed task.| | `TooManyBuckets` | Exceeded the maximum number of partition buckets for a stage (5,000 partition buckets).
< br />Partition buckets are created for each [`PARTITIONED BY`](#partitioned-by) time chunk for INSERT and REPLACE queries. The most common reason for this error is that your `PARTITIONED BY` is too narrow relative to your data. | `maxBuckets`: The limit on partition buckets. | | `TooManyInputFiles` | Exceeded the maximum number of input files or segments per worker (10,000 files or segments).

If you encounter this limit, consider adding more workers, or breaking up your query into smaller queries that process fewer files or segments per query. | `numInputFiles`: The total number of input files/segments for the stage.

`maxInputFiles`: The maximum number of input files/segments per worker per stage.

`minNumWorker`: The minimum number of workers required for a successful run. | | `TooManyPartitions` | Exceeded the maximum number of partitions for a stage (25,000 partitions).

This can occur with INSERT or REPLACE statements that generate large numbers of segments, since each segment is associated with a partition. If you encounter this limit, consider breaking up your INSERT or REPLACE statement into smaller statements that process less data per statement. | `maxPartitions`: The limit on partitions which was exceeded | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java index e9bf8d92cb3b..623d4c737f19 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java @@ -85,7 +85,13 @@ public static int workerFromTaskId(final String taskId) { final Matcher matcher = WORKER_PATTERN.matcher(taskId); if (matcher.matches()) { - return Integer.parseInt(matcher.group(WORKER_NUMBER)); + try { + String worker = matcher.group(WORKER_NUMBER); + return Integer.parseInt(worker); + } + catch (NumberFormatException e) { + throw new ISE(e, "Unable to parse worker out of task %s", taskId); + } } else { throw new ISE( "Desired pattern %s to extract worker from task id %s did not match ", diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java index 4a072e9bca93..5a8200c403d9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -115,7 +115,7 @@ public void inMemoryFullSketchMerging( throw rejectedExecutionException; } else { // throw worker error exception - throw new ISE("Unable to fetch partitions ", isError.get()); + throw new ISE("Unable to fetch partitions %s", isError.get()); } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index c57efd4cd47d..faee50fe0e95 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -236,7 +236,7 @@ IntSet getWorkersToSendPartitionBoundaries() throw new ISE("Result partition information is not relevant to this stage because it does not shuffle"); } IntAVLTreeSet workers = new IntAVLTreeSet(); - for (Integer worker : workerToPhase.keySet()) { + for (int worker : workerToPhase.keySet()) { if (ControllerWorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES.equals(workerToPhase.get(worker))) { workers.add(worker); } @@ -686,7 +686,7 @@ void mergeClusterByStatisticsCollectorForAllTimeChunks( if (allResultsStatsFetched()) { if (completeKeyStatisticsInformation == null || !completeKeyStatisticsInformation.isComplete()) { throw new ISE( - "Cannot generate partition boundaries until all the key information is received for stage[%d]", + "Cannot generate partition boundaries until all the key information is received for worker[%d] stage[%d]", workerNumber, stageDef.getStageNumber() ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 1cbab7decf7a..5c45fbcdf6a6 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -1300,7 +1300,7 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L})) .verifyResults(); if (DURABLE_STORAGE.equals(contextName) || FAULT_TOLERANCE.equals(contextName)) { - File successFile = new File( + new File( localFileStorageDir, DurableStorageUtils.getSuccessFilePath("query-test-query", 0, 0) ); From 791e2cb1266f92178bff01a6a6a92ba0a1ae1236 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 10 Jan 2023 09:37:32 +0530 Subject: [PATCH 25/27] Fixing IT --- .../controller/ControllerStageTracker.java | 3 -- .../ShufflingWorkersWithRetryKernelTest.java | 3 +- .../msq/ITKeyStatisticsSketchMergeMode.java | 4 +- .../druid/testsEx/msq/ITMultiStageQuery.java | 2 +- ...ITMultiStageQueryWorkerFaultTolerance.java | 4 +- .../cluster/MultiStageQueryWithMM/docker.yaml | 40 +++++++++++++++++++ .../testing/utils/MsqTestQueryHelper.java | 7 +++- 7 files changed, 52 insertions(+), 11 deletions(-) create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/MultiStageQueryWithMM/docker.yaml diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index faee50fe0e95..39b81b908103 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -382,9 +382,6 @@ ControllerStagePhase addPartialKeyInformationForWorker( final PartialKeyStatisticsInformation partialKeyStatisticsInformation ) { - if (phase != ControllerStagePhase.READING_INPUT) { - throw new ISE("Cannot add result key statistics from stage [%s]", phase); - } if (!stageDef.mustGatherResultKeyStatistics() || !stageDef.doesShuffle() || completeKeyStatisticsInformation == null) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java index 144033aebdeb..824c23b4fb11 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ShufflingWorkersWithRetryKernelTest.java @@ -96,13 +96,12 @@ public void testWorkerFailedBeforeAllWorkOrdersSent() controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.getRetriableWorkOrdersAndChangeState(0, RETRIABLE_FAULT); + controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RETRYING); controllerQueryKernelTester.sendWorkOrdersForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); controllerQueryKernelTester.addPartialKeyStatsInformation(0, 0); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addPartialKeyStatsInformation(0, 1); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); controllerQueryKernelTester.statsBeingFetchedForWorkers(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java index c0f22f2c762d..cd71e2765e15 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java @@ -128,7 +128,7 @@ public void testMsqIngestionParallelMerging() throws Exception )); } - msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); dataLoaderHelper.waitUntilDatasourceIsReady(datasource); msqHelper.testQueriesFromFile(QUERY_FILE, datasource); @@ -198,7 +198,7 @@ public void testMsqIngestionSequentialMerging() throws Exception )); } - msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); dataLoaderHelper.waitUntilDatasourceIsReady(datasource); msqHelper.testQueriesFromFile(QUERY_FILE, datasource); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java index e7ec1c0e3e4b..94d6a479cfd9 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java @@ -117,7 +117,7 @@ public void testMsqIngestionAndQuerying() throws Exception )); } - msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); dataLoaderHelper.waitUntilDatasourceIsReady(datasource); msqHelper.testQueriesFromFile(QUERY_FILE, datasource); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java index e31083098874..c91dc7b3fab7 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java @@ -125,7 +125,7 @@ public void testMsqIngestionAndQuerying() throws Exception SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask( queryLocal, ImmutableMap.of( - MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, + MultiStageQueryContext.CTX_FAULT_TOLERANCE, "true", MultiStageQueryContext.CTX_MAX_NUM_TASKS, 3 @@ -143,7 +143,7 @@ public void testMsqIngestionAndQuerying() throws Exception String taskIdToKill = sqlTaskStatus.getTaskId() + "-worker1_0"; killTaskAbruptly(taskIdToKill); - msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); dataLoaderHelper.waitUntilDatasourceIsReady(datasource); msqHelper.testQueriesFromFile(QUERY_FILE, datasource); diff --git a/integration-tests-ex/cases/src/test/resources/cluster/MultiStageQueryWithMM/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/MultiStageQueryWithMM/docker.yaml new file mode 100644 index 000000000000..2534f6a574ca --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/MultiStageQueryWithMM/docker.yaml @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# Definition of the multi stage query test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + middlemanager: + instances: + - port: 8091 diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java index 37647fde87d0..9051e1e138b7 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java @@ -168,6 +168,11 @@ public TaskState pollTaskIdForCompletion(String taskId) throws Exception ); } + public void pollTaskIdForSuccess(String taskId) throws Exception + { + Assert.assertEquals(pollTaskIdForCompletion(taskId), TaskState.SUCCESS); + } + /** * Fetches status reports for a given task */ @@ -255,7 +260,7 @@ public void testQueriesFromFile(String filePath, String fullDatasourcePath) thro ); } String taskId = sqlTaskStatus.getTaskId(); - pollTaskIdForCompletion(taskId); + pollTaskIdForSuccess(taskId); compareResults(taskId, queryWithResults); } } From 820b5b71cf1fed73991bebdd467442ab19804463 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 10 Jan 2023 12:59:36 +0530 Subject: [PATCH 26/27] Controller impl fix. --- .../main/java/org/apache/druid/msq/exec/ControllerImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index ae89c9d7f477..7bf8dffa8ecf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -2425,8 +2425,8 @@ private void sendPartitionBoundaries() && queryKernel.doesStageHaveResultPartitions(stageId)) { IntSet workersToSendPartitionBoundaries = queryKernel.getWorkersToSendPartitionBoundaries(stageId); if (workersToSendPartitionBoundaries.isEmpty()) { - log.debug("No workers for stage[%s] ready to recieve partition boundaries", stageId); - return; + log.debug("No workers for stage[%s] ready to receive partition boundaries", stageId); + continue; } final ClusterByPartitions partitions = queryKernel.getResultPartitionBoundariesForStage(stageId); From dd9a6a2c0992e6812fe4dba437b1fa03d183d570 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 10 Jan 2023 17:55:21 +0530 Subject: [PATCH 27/27] Fixing racy WorkerSketchFetcherTest.java exception handling. --- .../msq/exec/WorkerSketchFetcherTest.java | 104 +++++++++++------- 1 file changed, 62 insertions(+), 42 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java index 20f8503cf9be..bcfb692daf0b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -231,19 +231,24 @@ public void test_InMemoryRetryDisabled_multipleFailures() throws InterruptedExce workersWithFailedFetchParallel(ImmutableSet.of(TASK_1, TASK_0)); - target.inMemoryFullSketchMerging( - (kernelConsumer) -> kernelConsumer.accept(kernel), - stageDefinition.getId(), - ImmutableSet.copyOf(TASK_IDS), - ((queryKernel, integer, msqFault) -> { - throw new ISE("Should not be here"); - }) - ); + try { + target.inMemoryFullSketchMerging( + (kernelConsumer) -> kernelConsumer.accept(kernel), + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + } + catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Task fetch failed")); + } while (!target.executorService.isShutdown()) { Thread.sleep(100); } - Assert.assertNotNull(target.getError().getMessage().contains("Task fetch failed:")); + Assert.assertTrue((target.getError().getMessage().contains("Task fetch failed"))); } @@ -255,19 +260,24 @@ public void test_InMemoryRetryDisabled_singleFailure() throws InterruptedExcepti workersWithFailedFetchParallel(ImmutableSet.of(TASK_1)); - target.inMemoryFullSketchMerging( - (kernelConsumer) -> kernelConsumer.accept(kernel), - stageDefinition.getId(), - ImmutableSet.copyOf(TASK_IDS), - ((queryKernel, integer, msqFault) -> { - throw new ISE("Should not be here"); - }) - ); + try { + target.inMemoryFullSketchMerging( + (kernelConsumer) -> kernelConsumer.accept(kernel), + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + } + catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Task fetch failed")); + } while (!target.executorService.isShutdown()) { Thread.sleep(100); } - Assert.assertNotNull(target.getError().getMessage().contains(TASK_1)); + Assert.assertTrue((target.getError().getMessage().contains("Task fetch failed"))); } @@ -281,22 +291,27 @@ public void test_SequentialRetryDisabled_multipleFailures() throws InterruptedEx workersWithFailedFetchSequential(ImmutableSet.of(TASK_1, TASK_0)); - target.sequentialTimeChunkMerging( - (kernelConsumer) -> { - kernelConsumer.accept(kernel); - }, - completeKeyStatisticsInformation, - stageDefinition.getId(), - ImmutableSet.copyOf(TASK_IDS), - ((queryKernel, integer, msqFault) -> { - throw new ISE("Should not be here"); - }) - ); + try { + target.sequentialTimeChunkMerging( + (kernelConsumer) -> { + kernelConsumer.accept(kernel); + }, + completeKeyStatisticsInformation, + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + } + catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Task fetch failed")); + } while (!target.executorService.isShutdown()) { Thread.sleep(100); } - Assert.assertNotNull(target.getError().getMessage().contains("Task fetch failed:")); + Assert.assertTrue(target.getError().getMessage().contains("Task fetch failed")); } @@ -308,22 +323,27 @@ public void test_SequentialRetryDisabled_singleFailure() throws InterruptedExcep workersWithFailedFetchSequential(ImmutableSet.of(TASK_1)); - target.sequentialTimeChunkMerging( - (kernelConsumer) -> { - kernelConsumer.accept(kernel); - }, - completeKeyStatisticsInformation, - stageDefinition.getId(), - ImmutableSet.copyOf(TASK_IDS), - ((queryKernel, integer, msqFault) -> { - throw new ISE("Should not be here"); - }) - ); + try { + target.sequentialTimeChunkMerging( + (kernelConsumer) -> { + kernelConsumer.accept(kernel); + }, + completeKeyStatisticsInformation, + stageDefinition.getId(), + ImmutableSet.copyOf(TASK_IDS), + ((queryKernel, integer, msqFault) -> { + throw new ISE("Should not be here"); + }) + ); + } + catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Task fetch failed")); + } while (!target.executorService.isShutdown()) { Thread.sleep(100); } - Assert.assertNotNull(target.getError().getMessage().contains(TASK_1)); + Assert.assertTrue(target.getError().getMessage().contains(TASK_1)); }