From 681f8fbdea0ef321996049c2e431baba9d888a30 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 10 Oct 2022 20:49:56 +0530 Subject: [PATCH 01/31] Add sketch fetching framework --- .../org/apache/druid/msq/exec/Controller.java | 6 +- .../druid/msq/exec/ControllerClient.java | 10 +- .../apache/druid/msq/exec/ControllerImpl.java | 44 +++++++-- .../exec/ExceptionWrappingWorkerClient.java | 10 ++ .../org/apache/druid/msq/exec/Worker.java | 7 ++ .../apache/druid/msq/exec/WorkerClient.java | 9 ++ .../org/apache/druid/msq/exec/WorkerImpl.java | 17 +++- .../druid/msq/exec/WorkerSketchFetcher.java | 90 +++++++++++++++++ .../msq/indexing/ControllerChatHandler.java | 16 ++-- .../msq/indexing/IndexerControllerClient.java | 10 +- .../msq/indexing/IndexerWorkerClient.java | 22 +++++ .../druid/msq/indexing/WorkerChatHandler.java | 29 ++++++ .../controller/ControllerQueryKernel.java | 38 ++++++-- .../controller/ControllerStagePhase.java | 11 ++- .../controller/ControllerStageTracker.java | 96 +++++++++++-------- .../ClusterByStatisticsSnapshot.java | 6 ++ .../ClusterByStatisticsWorkerReport.java | 63 ++++++++++++ .../DelegateOrMinKeyCollectorSnapshot.java | 3 + .../msq/statistics/DistinctKeySnapshot.java | 3 + .../msq/statistics/KeyCollectorSnapshot.java | 9 ++ .../QuantilesSketchKeyCollectorSnapshot.java | 3 + .../BaseControllerQueryKernelTest.java | 25 ++++- .../ControllerQueryKernelTests.java | 55 +++++++++-- ...sterByStatisticsWorkerReportSerdeTest.java | 60 ++++++++++++ .../msq/test/MSQTestControllerClient.java | 10 +- .../druid/msq/test/MSQTestWorkerClient.java | 10 ++ 26 files changed, 569 insertions(+), 93 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java 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 07730de45e7a..97363aa7c4eb 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 @@ -27,7 +27,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.error.MSQErrorReport; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; import javax.annotation.Nullable; import java.util.List; @@ -81,9 +81,9 @@ public String getId() // Worker-to-controller messages /** - * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages. + * Accepts a {@link ClusterByStatisticsWorkerReport} for generating fetching sketches. */ - void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject); + void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerReport); /** * System error reported by a subtask. Note that the errors are organized by 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 f621133586c4..924bfc02c29c 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 @@ -22,7 +22,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; import javax.annotation.Nullable; import java.io.IOException; @@ -34,13 +34,13 @@ public interface ControllerClient extends AutoCloseable { /** - * Client side method to update the controller with key statistics for a particular stage and worker. - * Controller's implementation collates all the key statistics for a stage to generate the partition boundaries. + * Client side method to update the controller with worker reports for a particular stage and worker. + * Controller's implementation collates all the reports for a stage to fetch cluster by statistics from workers. */ - void postKeyStatistics( + void postWorkerReport( StageId stageId, int workerNumber, - ClusterByStatisticsSnapshot keyStatistics + ClusterByStatisticsWorkerReport workerReport ) throws IOException; /** 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 b6414c3db6e8..c4c8d67b3915 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 @@ -62,6 +62,7 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.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; @@ -104,6 +105,7 @@ 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.report.MSQResultsReport; @@ -146,7 +148,7 @@ import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory; import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory; import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; import org.apache.druid.msq.util.DimensionSchemaUtils; import org.apache.druid.msq.util.IntervalUtils; import org.apache.druid.msq.util.MSQFutureUtils; @@ -198,6 +200,7 @@ 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; @@ -256,6 +259,7 @@ public class ControllerImpl implements Controller // For live reports. Written by the main controller thread, read by HTTP threads. private final ConcurrentHashMap stagePartitionCountsForLiveReports = new ConcurrentHashMap<>(); + private WorkerSketchFetcher workerSketchFetcher; // Time at which the query started. // For live reports. Written by the main controller thread, read by HTTP threads. private volatile DateTime queryStartTime = null; @@ -515,6 +519,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); + this.workerSketchFetcher = new WorkerSketchFetcher(netClient); closer.register(netClient::close); final boolean isDurableStorageEnabled = @@ -558,10 +563,10 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) } /** - * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages. + * Provide a {@link ClusterByStatisticsWorkerReport} for shuffling stages. */ @Override - public void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject) + public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerReportObject) { addToKernelManipulationQueue( queryKernel -> { @@ -575,9 +580,9 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics stageDef.getShuffleSpec().get().doesAggregateByClusterKey() ); - final ClusterByStatisticsSnapshot keyStatistics; + final ClusterByStatisticsWorkerReport workerReport; try { - keyStatistics = mapper.convertValue(keyStatisticsObject, ClusterByStatisticsSnapshot.class); + workerReport = mapper.convertValue(workerReportObject, ClusterByStatisticsWorkerReport.class); } catch (IllegalArgumentException e) { throw new IAE( @@ -588,7 +593,34 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics ); } - queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics); + queryKernel.addResultStatisticsReportForStageAndWorker(stageId, workerNumber, workerReport); + + if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) { + List workerTaskIds = workerTaskLauncher.getTaskList(); + ClusterByStatisticsWorkerReport finalWorkerReport = queryKernel.getClusterByStatisticsWorkerReport(stageId); + + // Queue the sketch fetching task into the worker sketch fetcher. + CompletableFuture> clusterByPartitionsCompletableFuture = + workerSketchFetcher.submitFetcherTask( + finalWorkerReport, + workerTaskIds, + stageDef + ); + + // Add the listener to handle completion. + clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> { + if (throwable != null) { + queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable)); + } else if (clusterByPartitionsEither.isError()) { + queryKernel.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); + } else { + queryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow()); + } + kernelManipulationQueue.add(holder -> { + holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId)); + }); + }); + } } ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java index 1c3cc39987c7..879152255cc5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java @@ -31,9 +31,11 @@ import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import javax.annotation.Nullable; import java.io.IOException; +import java.util.concurrent.ExecutionException; /** * Wrapper around any {@link WorkerClient} that converts exceptions into {@link MSQException} @@ -55,6 +57,14 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO return wrap(workerTaskId, client, c -> c.postWorkOrder(workerTaskId, workOrder)); } + @Override + public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) + throws ExecutionException, InterruptedException + { + // TODO: check + return client.fetchClusterByStatisticsSnapshot(workerTaskId, queryId, stageNumber); + } + @Override public ListenableFuture postResultPartitionBoundaries( final String workerTaskId, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index f069c91e145a..82c4e07a7d74 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -25,10 +25,12 @@ import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import javax.annotation.Nullable; import java.io.IOException; import java.io.InputStream; +import java.util.concurrent.ExecutionException; public interface Worker { @@ -105,4 +107,9 @@ boolean postResultPartitionBoundaries( * Called when the work required for the query has been finished */ void postFinish(); + + /** + * Returns the statistics snapshot for the given stageId + */ + ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) throws ExecutionException, InterruptedException; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java index 72a0a81604d4..de71c8c3a001 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java @@ -25,8 +25,10 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import java.io.IOException; +import java.util.concurrent.ExecutionException; /** * Client for multi-stage query workers. Used by the controller task. @@ -38,6 +40,13 @@ public interface WorkerClient extends AutoCloseable */ ListenableFuture postWorkOrder(String workerId, WorkOrder workOrder); + /** + * Fetches the {@link ClusterByStatisticsSnapshot} from a worker. This is intended to be used by the + * {@link WorkerSketchFetcher}. + */ + ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) + throws ExecutionException, InterruptedException; + /** * Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the * controller after collating the result statistics from all the workers processing the query 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..61edd227e11c 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 @@ -129,8 +129,10 @@ import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -326,10 +328,10 @@ public Optional runTask(final Closer closer) throws Exception if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { if (controllerAlive) { - controllerClient.postKeyStatistics( + controllerClient.postWorkerReport( stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber(), - kernel.getResultKeyStatisticsSnapshot() + kernel.getResultKeyStatisticsSnapshot().workerReport(task().getWorkerNumber()) ); } kernel.startPreshuffleWaitingForResultPartitionBoundaries(); @@ -523,6 +525,17 @@ public void postFinish() kernelManipulationQueue.add(KernelHolder::setDone); } + @Override + public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) + throws ExecutionException, InterruptedException + { + CompletableFuture future = new CompletableFuture<>(); + kernelManipulationQueue.add(kernelHolder -> { + future.complete(kernelHolder.stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot()); + }); + return future.get(); + } + @Override public CounterSnapshotsTree getCounters() { 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 new file mode 100644 index 000000000000..4dca9641bba6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * Queues up fetching sketches from workers and progressively generates partitions boundaries. + * TODO: still in progress. Some cleanup required as things will likely move around with sequantial merging. + */ +public class WorkerSketchFetcher +{ + private static final int DEFAULT_THREAD_COUNT = 10; + + private final WorkerClient workerClient; + private final ExecutorService executorService; + + public WorkerSketchFetcher(WorkerClient workerClient) + { + this.workerClient = workerClient; + this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); + } + + public CompletableFuture> submitFetcherTask( + ClusterByStatisticsWorkerReport workerReport, + List workerTaskIds, + StageDefinition stageDefinition) + { + CompletableFuture> partitionFuture = new CompletableFuture<>(); + ClusterByStatisticsCollector finalClusterByStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); + + int workerNumber = stageDefinition.getMaxWorkerCount(); + IntSet finishedWorkers = new IntAVLTreeSet(); + + for (int i = 0; i < workerNumber; i++) { + final int workerNo = i; + executorService.submit(() -> { + try { + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = workerClient.fetchClusterByStatisticsSnapshot( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber() + ); + // If the future already failed for some reason, skip ahead. + if (!partitionFuture.isDone()) { + finalClusterByStatisticsCollector.addAll(clusterByStatisticsSnapshot); + finishedWorkers.add(workerNo); + if (finishedWorkers.size() == workerNumber) { + partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(finalClusterByStatisticsCollector)); + } + } + } + catch (Exception e) { + finalClusterByStatisticsCollector.clear(); + partitionFuture.completeExceptionally(e); + } + }); + } + return partitionFuture; + } +} 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 5561a98a07bb..a2ef9588f1f1 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 @@ -26,7 +26,8 @@ import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; @@ -58,16 +59,17 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller) } /** - * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages. + * Used by subtasks to post {@link ClusterByStatisticsWorkerReport} for shuffling stages. * - * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API. + * See {@link ControllerClient#postWorkerReport(StageId, int, ClusterByStatisticsWorkerReport)} + * for the client-side code that calls this API. */ @POST - @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}") + @Path("/statisticsWorkerReport/{queryId}/{stageNumber}/{workerNumber}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) - public Response httpPostKeyStatistics( - final Object keyStatisticsObject, + public Response httpPostStatisticsWorkerReport( + final Object workerReport, @PathParam("queryId") final String queryId, @PathParam("stageNumber") final int stageNumber, @PathParam("workerNumber") final int workerNumber, @@ -75,7 +77,7 @@ public Response httpPostKeyStatistics( ) { ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - controller.updateStatus(stageNumber, workerNumber, keyStatisticsObject); + controller.updateWorkerReportStatus(stageNumber, workerNumber, workerReport); return Response.status(Response.Status.ACCEPTED).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 2c249e26398d..db533842c435 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 @@ -29,7 +29,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -59,14 +59,14 @@ public IndexerControllerClient( } @Override - public void postKeyStatistics( + public void postWorkerReport( StageId stageId, int workerNumber, - ClusterByStatisticsSnapshot keyStatistics + ClusterByStatisticsWorkerReport workerReport ) throws IOException { final String path = StringUtils.format( - "/keyStatistics/%s/%s/%d", + "/statisticsWorkerReport/%s/%d/%d", StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), workerNumber @@ -74,7 +74,7 @@ public void postKeyStatistics( doRequest( new RequestBuilder(HttpMethod.POST, path) - .jsonContent(jsonMapper, keyStatistics), + .jsonContent(jsonMapper, workerReport), IgnoreHttpResponseHandler.INSTANCE ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java index 430c264291f8..bc6bfce700d7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java @@ -41,6 +41,7 @@ import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -59,6 +60,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; public class IndexerWorkerClient implements WorkerClient @@ -103,6 +105,26 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO ); } + @Override + public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber + ) throws ExecutionException, InterruptedException + { + String path = StringUtils.format("/keyStatistics/%s/%d", + StringUtils.urlEncode(queryId), + stageNumber); + + return deserialize( + getClient(workerTaskId).request( + new RequestBuilder(HttpMethod.POST, path), + new BytesFullResponseHandler() + ), + new TypeReference() {} + ); + } + @Override public ListenableFuture postResultPartitionBoundaries( String workerTaskId, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java index 06fd8f36caf7..5da7223da962 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java @@ -28,6 +28,7 @@ import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; @@ -47,6 +48,7 @@ import javax.ws.rs.core.StreamingOutput; import java.io.IOException; import java.io.InputStream; +import java.util.concurrent.ExecutionException; public class WorkerChatHandler implements ChatHandler { @@ -179,6 +181,33 @@ public Response httpPostResultPartitionBoundaries( } } + @POST + @Path("/keyStatistics/{queryId}/{stageNumber}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpFetchKeyStatistics( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot; + try { + StageId stageId = new StageId(queryId, stageNumber); + clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); + } + catch (ExecutionException | InterruptedException e) { + return Response + .status(Response.Status.INTERNAL_SERVER_ERROR) + .build(); + + } + return Response.status(Response.Status.ACCEPTED) + .entity(clusterByStatisticsSnapshot) + .build(); + } + /** * See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API. */ 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..2d6a23ce7533 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 @@ -41,7 +41,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.ClusterByStatisticsWorkerReport; import javax.annotation.Nullable; import java.util.HashMap; @@ -331,6 +331,22 @@ public ClusterByPartitions getResultPartitionBoundariesForStage(final StageId st return getStageKernelOrThrow(stageId).getResultPartitionBoundaries(); } + /** + * Delegates call to {@link ControllerStageTracker#getResultPartitionBoundaries()} + */ + public ClusterByStatisticsWorkerReport getClusterByStatisticsWorkerReport(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getClusterByStatisticsWorkerReport(); + } + + /** + * 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()} */ @@ -387,22 +403,24 @@ public WorkerInputs getWorkerInputsForStage(final StageId stageId) } /** - * Delegates call to {@link ControllerStageTracker#addResultKeyStatisticsForWorker(int, ClusterByStatisticsSnapshot)}. + * Delegates call to {@link ControllerStageTracker#addStatisticsReportForWorker}. * If calling this causes transition for the stage kernel, then this gets registered in this query kernel */ - public void addResultKeyStatisticsForStageAndWorker( + public void addResultStatisticsReportForStageAndWorker( final StageId stageId, final int workerNumber, - final ClusterByStatisticsSnapshot snapshot + final ClusterByStatisticsWorkerReport workerReport ) { - ControllerStagePhase newPhase = getStageKernelOrThrow(stageId).addResultKeyStatisticsForWorker( + ControllerStageTracker stageKernel = getStageKernelOrThrow(stageId); + ControllerStagePhase newPhase = stageKernel.addStatisticsReportForWorker( workerNumber, - snapshot + workerReport ); - // If the phase is POST_READING or FAILED, that implies the kernel has transitioned. We need to account for that + // If the kernel phase has transitioned, we need to account for that. switch (newPhase) { + case MERGING_STATISTICS: case POST_READING: case FAILED: transitionStageKernel(stageId, newPhase); @@ -433,6 +451,12 @@ public MSQFault getFailureReasonForStage(final StageId stageId) return getStageKernelOrThrow(stageId).getFailureReason(); } + public void failStageForReason(final StageId stageId, MSQFault fault) + { + getStageKernelOrThrow(stageId).failForReason(fault); + transitionStageKernel(stageId, ControllerStagePhase.FAILED); + } + /** * Delegates call to {@link ControllerStageTracker#fail()} and registers this transition to FAILED in this query kernel */ 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..332138914eeb 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 @@ -48,6 +48,15 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) } }, + // Waiting to fetch key statistics in the background from the workers and incrementally generate partitions. + MERGING_STATISTICS { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == READING_INPUT; + } + }, + // Post the inputs have been read and mapped to frames, in the `POST_READING` stage, we pre-shuffle and determing the partition boundaries. // This step for a stage spits out the statistics of the data as a whole (and not just the individual records). This // phase is not required in non-pre shuffle contexts. @@ -55,7 +64,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return priorPhase == READING_INPUT; + return priorPhase == MERGING_STATISTICS; } }, 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..7b5cde6117ec 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 @@ -38,11 +38,11 @@ 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.ClusterByStatisticsWorkerReport; import javax.annotation.Nullable; import java.util.List; +import java.util.TreeSet; /** * Controller-side state machine for each stage. Used by {@link ControllerQueryKernel} to form the overall state @@ -57,13 +57,13 @@ class ControllerStageTracker private final int workerCount; private final WorkerInputs workerInputs; - private final IntSet workersWithResultKeyStatistics = new IntAVLTreeSet(); + private final IntSet workersWithFinishedReport = new IntAVLTreeSet(); private final IntSet workersWithResultsComplete = new IntAVLTreeSet(); private ControllerStagePhase phase = ControllerStagePhase.NEW; @Nullable - private final ClusterByStatisticsCollector resultKeyStatisticsCollector; + public final ClusterByStatisticsWorkerReport clusterByStatisticsWorkerReport; // Result partitions and where they can be read from. @Nullable @@ -89,10 +89,10 @@ private ControllerStageTracker( this.workerInputs = workerInputs; if (stageDef.mustGatherResultKeyStatistics()) { - this.resultKeyStatisticsCollector = stageDef.createResultKeyStatisticsCollector(); + this.clusterByStatisticsWorkerReport = new ClusterByStatisticsWorkerReport(new TreeSet<>(), false); } else { - this.resultKeyStatisticsCollector = null; - generateResultPartitionsAndBoundaries(); + this.clusterByStatisticsWorkerReport = null; + generateResultPartitionsAndBoundariesWithoutKeyStatistics(); } } @@ -172,18 +172,12 @@ ClusterByPartitions getResultPartitionBoundaries() */ boolean collectorEncounteredAnyMultiValueField() { - if (resultKeyStatisticsCollector == null) { + if (clusterByStatisticsWorkerReport == null) { throw new ISE("Stage does not gather result key statistics"); - } else if (resultPartitions == null) { + } else if (workersWithFinishedReport.size() != workerCount) { throw new ISE("Result key statistics are not ready"); } else { - for (int i = 0; i < resultKeyStatisticsCollector.getClusterBy().getColumns().size(); i++) { - if (resultKeyStatisticsCollector.hasMultipleValues(i)) { - return true; - } - } - - return false; + return clusterByStatisticsWorkerReport.isHasMultipleValues(); } } @@ -227,19 +221,28 @@ WorkerInputs getWorkerInputs() return workerInputs; } + /** + * Returns the merged worker report. + */ + @Nullable + public ClusterByStatisticsWorkerReport getClusterByStatisticsWorkerReport() + { + return clusterByStatisticsWorkerReport; + } + /** * Adds result key statistics for a particular worker number. If statistics have already been added for this worker, * then this call ignores the new ones and does nothing. * * @param workerNumber the worker - * @param snapshot worker statistics + * @param workerReport worker report */ - ControllerStagePhase addResultKeyStatisticsForWorker( + ControllerStagePhase addStatisticsReportForWorker( final int workerNumber, - final ClusterByStatisticsSnapshot snapshot + final ClusterByStatisticsWorkerReport workerReport ) { - if (resultKeyStatisticsCollector == null) { + if (clusterByStatisticsWorkerReport == null) { throw new ISE("Stage does not gather result key statistics"); } @@ -252,14 +255,17 @@ ControllerStagePhase addResultKeyStatisticsForWorker( } try { - if (workersWithResultKeyStatistics.add(workerNumber)) { - resultKeyStatisticsCollector.addAll(snapshot); + if (workersWithFinishedReport.add(workerNumber)) { + clusterByStatisticsWorkerReport.addAll(workerReport); - if (workersWithResultKeyStatistics.size() == workerCount) { - generateResultPartitionsAndBoundaries(); + if (workersWithFinishedReport.size() == workerCount) { + // All workers have sent the report. + // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers. + transitionTo(ControllerStagePhase.MERGING_STATISTICS); - // Phase can become FAILED after generateResultPartitionsAndBoundaries, if there were too many partitions. - if (phase != ControllerStagePhase.FAILED) { + if (!stageDef.doesShuffle()) { + // We don't need to wait for key statistics in this case. We can generate parititions and skip to the next phase. + generateResultPartitionsAndBoundariesWithoutKeyStatistics(); transitionTo(ControllerStagePhase.POST_READING); } } @@ -273,6 +279,25 @@ ControllerStagePhase addResultKeyStatisticsForWorker( return getPhase(); } + /** + * Sets the {@link #resultPartitions} and {@link #resultPartitionBoundaries} and transitions the phase to POST_READING. + */ + void setClusterByPartitionBoundaries(ClusterByPartitions clusterByPartitions) + { + if (resultPartitions != null) { + throw new ISE("Result partitions have already been generated"); + } + + this.resultPartitionBoundaries = clusterByPartitions; + this.resultPartitions = ReadablePartitions.striped( + stageDef.getStageNumber(), + workerCount, + clusterByPartitions.size() + ); + + transitionTo(ControllerStagePhase.POST_READING); + } + /** * Accepts and sets the results that each worker produces for this particular stage * @@ -332,12 +357,11 @@ void fail() } /** - * Sets {@link #resultPartitions} (always) and {@link #resultPartitionBoundaries}. + * Sets {@link #resultPartitions} (always) and {@link #resultPartitionBoundaries} without using key statistics. * - * If {@link StageDefinition#mustGatherResultKeyStatistics()} is true, this method cannot be called until after - * statistics have been provided to {@link #addResultKeyStatisticsForWorker} for all workers. + * If {@link StageDefinition#mustGatherResultKeyStatistics()} is true, this method should not be called. */ - private void generateResultPartitionsAndBoundaries() + private void generateResultPartitionsAndBoundariesWithoutKeyStatistics() { if (resultPartitions != null) { throw new ISE("Result partitions have already been generated"); @@ -346,12 +370,12 @@ private void generateResultPartitionsAndBoundaries() final int stageNumber = stageDef.getStageNumber(); if (stageDef.doesShuffle()) { - if (stageDef.mustGatherResultKeyStatistics() && workersWithResultKeyStatistics.size() != workerCount) { - throw new ISE("Cannot generate result partitions without all worker statistics"); + if (stageDef.mustGatherResultKeyStatistics()) { + throw new ISE("Cannot generate result partitions without key statistics"); } final Either maybeResultPartitionBoundaries = - stageDef.generatePartitionsForShuffle(resultKeyStatisticsCollector); + stageDef.generatePartitionsForShuffle(null); if (maybeResultPartitionBoundaries.isError()) { failForReason(new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); @@ -390,15 +414,11 @@ private void generateResultPartitionsAndBoundaries() * * @param fault reason why this stage has failed */ - private void failForReason(final MSQFault fault) + void failForReason(final MSQFault fault) { transitionTo(ControllerStagePhase.FAILED); this.failureReason = fault; - - if (resultKeyStatisticsCollector != null) { - resultKeyStatisticsCollector.clear(); - } } void transitionTo(final ControllerStagePhase newPhase) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index 6b16890254a0..4d06ea7f5412 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; import org.apache.druid.frame.key.RowKey; import javax.annotation.Nullable; @@ -64,6 +65,11 @@ Set getHasMultipleValues() return hasMultipleValues; } + public ClusterByStatisticsWorkerReport workerReport(int workerNumber) + { + return new ClusterByStatisticsWorkerReport(ImmutableSet.of(workerNumber), !getHasMultipleValues().isEmpty()); + } + @Override public boolean equals(Object o) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java new file mode 100644 index 000000000000..37c6ca11782e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.statistics; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Set; + +/** + * Class sent by worker to controller after reading input to generate partition boundries. + */ +public class ClusterByStatisticsWorkerReport +{ + private final Set workerIds; // TODO: change to be a map of rowkey vs worker id. Currently this isn't very useful. + + private Boolean hasMultipleValues; + + @JsonCreator + public ClusterByStatisticsWorkerReport( + @JsonProperty("workerIds") final Set timeChunks, + @JsonProperty("hasMultipleValues") boolean hasMultipleValues + ) + { + this.workerIds = timeChunks; + this.hasMultipleValues = hasMultipleValues; + } + + public void addAll(ClusterByStatisticsWorkerReport other) + { + workerIds.addAll(other.getWorkerIds()); + hasMultipleValues = hasMultipleValues || other.hasMultipleValues; + } + + @JsonProperty("workerIds") + public Set getWorkerIds() + { + return workerIds; + } + + @JsonProperty("hasMultipleValues") + public boolean isHasMultipleValues() + { + return hasMultipleValues; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java index f0191da126b2..5e8da8df0c22 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java @@ -22,16 +22,19 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.frame.key.RowKey; import org.apache.druid.java.util.common.ISE; import javax.annotation.Nullable; import java.util.Objects; +@JsonTypeName(DelegateOrMinKeyCollectorSnapshot.TYPE) public class DelegateOrMinKeyCollectorSnapshot implements KeyCollectorSnapshot { static final String FIELD_SNAPSHOT = "snapshot"; static final String FIELD_MIN_KEY = "minKey"; + static final String TYPE = "delegate"; private final T snapshot; private final RowKey minKey; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java index 44b95d3874f6..47d46f72cabb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import org.apache.druid.collections.SerializablePair; import org.apache.druid.frame.key.RowKey; @@ -31,8 +32,10 @@ import java.util.Map; import java.util.Objects; +@JsonTypeName(DistinctKeySnapshot.TYPE) public class DistinctKeySnapshot implements KeyCollectorSnapshot { + static final String TYPE = "distinct"; private final List> keys; private final int spaceReductionFactor; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java index ad359cb1e60e..d1e717c37ec9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java @@ -19,9 +19,18 @@ package org.apache.druid.msq.statistics; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + /** * Marker interface for deserialization. */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "collectorType") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = DelegateOrMinKeyCollectorSnapshot.TYPE, value = DelegateOrMinKeyCollectorSnapshot.class), + @JsonSubTypes.Type(name = QuantilesSketchKeyCollectorSnapshot.TYPE, value = QuantilesSketchKeyCollectorSnapshot.class), + @JsonSubTypes.Type(name = DistinctKeySnapshot.TYPE, value = DistinctKeySnapshot.class), +}) public interface KeyCollectorSnapshot { } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java index 1b555ac3f944..5784afd49a0f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java @@ -21,11 +21,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.Objects; +@JsonTypeName(QuantilesSketchKeyCollectorSnapshot.TYPE) public class QuantilesSketchKeyCollectorSnapshot implements KeyCollectorSnapshot { + static final String TYPE = "quantile"; private final String encodedSketch; private final double averageKeyLength; 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..bd5157a4d346 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,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.key.KeyTestUtils; import org.apache.druid.frame.key.RowKey; import org.apache.druid.java.util.common.IAE; @@ -31,6 +32,7 @@ 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.WorkerAssignmentStrategy; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; @@ -121,10 +123,10 @@ public ControllerQueryKernelTester setupStage( if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { for (int i = 0; i < numWorkers; ++i) { - controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( + controllerQueryKernel.addResultStatisticsReportForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), i, - ClusterByStatisticsSnapshot.empty() + ClusterByStatisticsSnapshot.empty().workerReport(i) ); } } else { @@ -244,7 +246,7 @@ public void finishStage(int stageNumber, boolean strict) controllerQueryKernel.finishStage(new StageId(queryDefinition.getQueryId(), stageNumber), strict); } - public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerNumber) + public ClusterByStatisticsCollector addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerNumber) { Preconditions.checkArgument(initialized); @@ -260,11 +262,12 @@ public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerN keyStatsCollector.add(key, 1); } - controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( + controllerQueryKernel.addResultStatisticsReportForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), workerNumber, - keyStatsCollector.snapshot() + keyStatsCollector.snapshot().workerReport(workerNumber) ); + return keyStatsCollector; } public void setResultsCompleteForStageAndWorker(int stageNumber, int workerNumber) @@ -277,6 +280,18 @@ public void setResultsCompleteForStageAndWorker(int stageNumber, int workerNumbe ); } + public void setPartitionBoundaries(int stageNumber, ClusterByStatisticsCollector clusterByStatisticsCollector) + { + 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); + } + public void failStage(int stageNumber) { Preconditions.checkArgument(initialized); 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..f2b6139e9985 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,7 @@ package org.apache.druid.msq.kernel.controller; import com.google.common.collect.ImmutableSet; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.junit.Assert; import org.junit.Test; @@ -146,8 +147,13 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(0), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + ClusterByStatisticsCollector clusterByStatisticsCollector = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 0, + 0 + ); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(0, clusterByStatisticsCollector); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); @@ -156,9 +162,20 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(1), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 0); + clusterByStatisticsCollector = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 1, + 0 + ); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 1); + clusterByStatisticsCollector.addAll( + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 1, + 1 + ) + ); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(1, clusterByStatisticsCollector); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); @@ -182,9 +199,19 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(1), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(3); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 0); + ClusterByStatisticsCollector clusterByStatisticsCollector3 = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 3, + 0 + ); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 1); + ClusterByStatisticsCollector clusterByStatisticsCollector4 = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 3, + 1 + ); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(3, clusterByStatisticsCollector3.addAll(clusterByStatisticsCollector4)); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 0); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); @@ -217,11 +244,21 @@ public void testTransitionsInShufflingStagesAndMultipleWorkers() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + ClusterByStatisticsCollector clusterByStatisticsCollector = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 0, + 0 + ); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + clusterByStatisticsCollector.addAll( + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 0, + 1 + ) + ); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(0, clusterByStatisticsCollector); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java new file mode 100644 index 000000000000..cad1147ecdac --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.statistics; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ClusterByStatisticsWorkerReportSerdeTest +{ + private ObjectMapper objectMapper; + + @Before + public void setUp() + { + objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerModules(new MSQIndexingModule().getJacksonModules()); + objectMapper.enable(JsonParser.Feature.STRICT_DUPLICATE_DETECTION); + } + + @Test + public void testSerde() throws JsonProcessingException + { + ClusterByStatisticsWorkerReport workerReport = new ClusterByStatisticsWorkerReport( + ImmutableSet.of(1, 3, 4), + false + ); + + final String json = objectMapper.writeValueAsString(workerReport); + final ClusterByStatisticsWorkerReport deserializedWorkerReport = objectMapper.readValue( + json, + ClusterByStatisticsWorkerReport.class + ); + Assert.assertEquals(json, workerReport.getWorkerIds(), deserializedWorkerReport.getWorkerIds()); + Assert.assertEquals(json, workerReport.isHasMultipleValues(), deserializedWorkerReport.isHasMultipleValues()); + } +} 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 0e00a8e3edf8..0db3c2cff014 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 @@ -25,7 +25,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; import javax.annotation.Nullable; import java.util.List; @@ -40,17 +40,17 @@ public MSQTestControllerClient(Controller controller) } @Override - public void postKeyStatistics( + public void postWorkerReport( StageId stageId, int workerNumber, - ClusterByStatisticsSnapshot keyStatistics + ClusterByStatisticsWorkerReport workerReport ) { try { - controller.updateStatus(stageId.getStageNumber(), workerNumber, keyStatistics); + controller.updateWorkerReportStatus(stageId.getStageNumber(), workerNumber, workerReport); } catch (Exception e) { - throw new ISE(e, "unable to post key statistics"); + throw new ISE(e, "unable to post worker report"); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java index 094c1bd16cf7..751bf07963c2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java @@ -29,10 +29,12 @@ import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import java.io.InputStream; import java.util.Arrays; import java.util.Map; +import java.util.concurrent.ExecutionException; public class MSQTestWorkerClient implements WorkerClient { @@ -50,6 +52,14 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO return Futures.immediateFuture(null); } + @Override + public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) + throws ExecutionException, InterruptedException + { + StageId stageId = new StageId(queryId, stageNumber); + return inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId); + } + @Override public ListenableFuture postResultPartitionBoundaries( String workerTaskId, From b375d591b9d5d20f88b4169b69b7bb023b0807a8 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 13 Oct 2022 11:43:45 +0530 Subject: [PATCH 02/31] Refactor code to support sequential merge --- .../exec/ExceptionWrappingWorkerClient.java | 12 ++++++++- .../org/apache/druid/msq/exec/Worker.java | 6 +++++ .../apache/druid/msq/exec/WorkerClient.java | 11 ++++++++ .../org/apache/druid/msq/exec/WorkerImpl.java | 20 +++++++++++++- .../msq/indexing/IndexerWorkerClient.java | 22 +++++++++++++++ .../druid/msq/indexing/WorkerChatHandler.java | 27 +++++++++++++++++++ .../controller/ControllerStageTracker.java | 4 +-- .../ClusterByStatisticsCollectorImpl.java | 12 ++++++--- .../ClusterByStatisticsSnapshot.java | 27 ++++++++++++++----- .../ClusterByStatisticsWorkerReport.java | 19 ++++++++----- ...sterByStatisticsWorkerReportSerdeTest.java | 7 +++-- .../druid/msq/test/MSQTestWorkerClient.java | 12 +++++++++ .../apache/druid/frame/key/RowKeyReader.java | 22 +++++++++++++++ 13 files changed, 179 insertions(+), 22 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java index 879152255cc5..1f8cd10a76c7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java @@ -61,10 +61,20 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) throws ExecutionException, InterruptedException { - // TODO: check return client.fetchClusterByStatisticsSnapshot(workerTaskId, queryId, stageNumber); } + @Override + public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ) throws ExecutionException, InterruptedException + { + return client.fetchSingletonStatisticsSnapshot(workerTaskId, queryId, stageNumber, timeChunk); + } + @Override public ListenableFuture postResultPartitionBoundaries( final String workerTaskId, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index 82c4e07a7d74..5c1aab736f16 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -112,4 +112,10 @@ boolean postResultPartitionBoundaries( * Returns the statistics snapshot for the given stageId */ ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) throws ExecutionException, InterruptedException; + + /** + * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk + */ + ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(StageId stageId, long timeChunk) + throws ExecutionException, InterruptedException; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java index de71c8c3a001..ee8085850b71 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java @@ -47,6 +47,17 @@ public interface WorkerClient extends AutoCloseable ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) throws ExecutionException, InterruptedException; + /** + * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk. + * This is intended to be used by the {@link WorkerSketchFetcher}. + */ + ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ) throws ExecutionException, InterruptedException; + /** * Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the * controller after collating the result statistics from all the workers processing the query 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 61edd227e11c..5255f09dd0f4 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 @@ -102,6 +102,7 @@ import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; import org.apache.druid.msq.util.DecoratedExecutorService; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.PrioritizedCallable; @@ -328,10 +329,14 @@ public Optional runTask(final Closer closer) throws Exception if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { if (controllerAlive) { + ClusterByStatisticsWorkerReport workerReport = + kernel.getResultKeyStatisticsSnapshot() + .workerReport(task().getWorkerNumber()); + controllerClient.postWorkerReport( stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber(), - kernel.getResultKeyStatisticsSnapshot().workerReport(task().getWorkerNumber()) + workerReport ); } kernel.startPreshuffleWaitingForResultPartitionBoundaries(); @@ -536,6 +541,19 @@ public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) return future.get(); } + @Override + public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(StageId stageId, long timeChunk) + throws ExecutionException, InterruptedException + { + CompletableFuture future = new CompletableFuture<>(); + kernelManipulationQueue.add(kernelHolder -> { + ClusterByStatisticsSnapshot snapshot = kernelHolder.stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); + ClusterByStatisticsSnapshot singletonSnapshot = snapshot.getSingletonSnapshot(timeChunk); + future.complete(singletonSnapshot); + }); + return future.get(); + } + @Override public CounterSnapshotsTree getCounters() { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java index bc6bfce700d7..d0e4e06d2c2f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java @@ -125,6 +125,28 @@ public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot( ); } + @Override + public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ) throws ExecutionException, InterruptedException + { + String path = StringUtils.format("/singletonKeyStatistics/%s/%d/%d", + StringUtils.urlEncode(queryId), + stageNumber, + timeChunk); + + return deserialize( + getClient(workerTaskId).request( + new RequestBuilder(HttpMethod.POST, path), + new BytesFullResponseHandler() + ), + new TypeReference() {} + ); + } + @Override public ListenableFuture postResultPartitionBoundaries( String workerTaskId, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java index 5da7223da962..09e713d5225f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java @@ -208,6 +208,33 @@ public Response httpFetchKeyStatistics( .build(); } + @POST + @Path("/singletonKeyStatistics/{queryId}/{stageNumber}/{timeChunk}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpSketch( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("timeChunk") final long timeChunk, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); + ClusterByStatisticsSnapshot singletonSnapshot; + try { + StageId stageId = new StageId(queryId, stageNumber); + singletonSnapshot = worker.fetchSingletonStatisticsSnapshot(stageId, timeChunk); + } + catch (ExecutionException | InterruptedException e) { + return Response + .status(Response.Status.INTERNAL_SERVER_ERROR) + .build(); + } + return Response.status(Response.Status.ACCEPTED) + .entity(singletonSnapshot) + .build(); + } + /** * See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API. */ 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 7b5cde6117ec..6886933a4586 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 @@ -42,7 +42,7 @@ import javax.annotation.Nullable; import java.util.List; -import java.util.TreeSet; +import java.util.TreeMap; /** * Controller-side state machine for each stage. Used by {@link ControllerQueryKernel} to form the overall state @@ -89,7 +89,7 @@ private ControllerStageTracker( this.workerInputs = workerInputs; if (stageDef.mustGatherResultKeyStatistics()) { - this.clusterByStatisticsWorkerReport = new ClusterByStatisticsWorkerReport(new TreeSet<>(), false); + this.clusterByStatisticsWorkerReport = new ClusterByStatisticsWorkerReport(new TreeMap<>(), false); } else { this.clusterByStatisticsWorkerReport = null; generateResultPartitionsAndBoundariesWithoutKeyStatistics(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java index 9e033c87498d..d30df2fe6ab2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.SortedMap; @@ -167,7 +168,7 @@ public ClusterByStatisticsCollector addAll(final ClusterByStatisticsCollector ot public ClusterByStatisticsCollector addAll(final ClusterByStatisticsSnapshot snapshot) { // Add all key collectors from the other collector. - for (ClusterByStatisticsSnapshot.Bucket otherBucket : snapshot.getBuckets()) { + for (ClusterByStatisticsSnapshot.Bucket otherBucket : snapshot.getBuckets().values()) { //noinspection rawtypes, unchecked final KeyCollector otherKeyCollector = ((KeyCollectorFactory) keyCollectorFactory).fromSnapshot(otherBucket.getKeyCollectorSnapshot()); @@ -315,13 +316,18 @@ public ClusterByStatisticsSnapshot snapshot() { assertRetainedByteCountsAreTrackedCorrectly(); - final List bucketSnapshots = new ArrayList<>(); + final Map bucketSnapshots = new HashMap<>(); + final RowKeyReader trimmedRowReader = keyReader.trimmedKeyReader(clusterBy.getBucketByCount()); for (final Map.Entry bucketEntry : buckets.entrySet()) { //noinspection rawtypes, unchecked final KeyCollectorSnapshot keyCollectorSnapshot = ((KeyCollectorFactory) keyCollectorFactory).toSnapshot(bucketEntry.getValue().keyCollector); - bucketSnapshots.add(new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot)); + Long bucketKey = Long.MIN_VALUE; + if (clusterBy.getBucketByCount() == 1) { + bucketKey = (Long) trimmedRowReader.read(bucketEntry.getKey(), 0); + } + bucketSnapshots.put(bucketKey, new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot)); } final IntSet hasMultipleValuesSet; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index 4d06ea7f5412..048787dd254f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -23,23 +23,26 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.frame.key.RowKey; import javax.annotation.Nullable; import java.util.Collections; -import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; public class ClusterByStatisticsSnapshot { - private final List buckets; + private final Map buckets; private final Set hasMultipleValues; @JsonCreator ClusterByStatisticsSnapshot( - @JsonProperty("buckets") final List buckets, + @JsonProperty("buckets") final Map buckets, @JsonProperty("hasMultipleValues") @Nullable final Set hasMultipleValues ) { @@ -49,15 +52,21 @@ public class ClusterByStatisticsSnapshot public static ClusterByStatisticsSnapshot empty() { - return new ClusterByStatisticsSnapshot(Collections.emptyList(), null); + return new ClusterByStatisticsSnapshot(Collections.emptyMap(), null); } @JsonProperty("buckets") - List getBuckets() + Map getBuckets() { return buckets; } + public ClusterByStatisticsSnapshot getSingletonSnapshot(long timeChunk) + { + Bucket bucket = buckets.get(timeChunk); + return new ClusterByStatisticsSnapshot(ImmutableMap.of(timeChunk, bucket), null); + } + @JsonProperty("hasMultipleValues") @JsonInclude(JsonInclude.Include.NON_EMPTY) Set getHasMultipleValues() @@ -67,7 +76,13 @@ Set getHasMultipleValues() public ClusterByStatisticsWorkerReport workerReport(int workerNumber) { - return new ClusterByStatisticsWorkerReport(ImmutableSet.of(workerNumber), !getHasMultipleValues().isEmpty()); + SortedMap> rowKeyIntSetTreeSet = new TreeMap<>(); + + for (Long bucketKey : buckets.keySet()) { + rowKeyIntSetTreeSet.put(bucketKey, ImmutableSet.of(workerNumber)); + } + + return new ClusterByStatisticsWorkerReport(rowKeyIntSetTreeSet, !getHasMultipleValues().isEmpty()); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java index 37c6ca11782e..ee1201a67d2f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java @@ -22,37 +22,42 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.HashSet; import java.util.Set; +import java.util.SortedMap; /** * Class sent by worker to controller after reading input to generate partition boundries. */ public class ClusterByStatisticsWorkerReport { - private final Set workerIds; // TODO: change to be a map of rowkey vs worker id. Currently this isn't very useful. + private final SortedMap> timeSegmentVsWorkerIdMap; private Boolean hasMultipleValues; @JsonCreator public ClusterByStatisticsWorkerReport( - @JsonProperty("workerIds") final Set timeChunks, + @JsonProperty("timeSegmentVsWorkerIdMap") final SortedMap> timeChunks, @JsonProperty("hasMultipleValues") boolean hasMultipleValues ) { - this.workerIds = timeChunks; + this.timeSegmentVsWorkerIdMap = timeChunks; this.hasMultipleValues = hasMultipleValues; } public void addAll(ClusterByStatisticsWorkerReport other) { - workerIds.addAll(other.getWorkerIds()); + for (Long timeChunk : other.timeSegmentVsWorkerIdMap.keySet()) { + timeSegmentVsWorkerIdMap.computeIfAbsent(timeChunk, key -> new HashSet<>()) + .addAll(other.timeSegmentVsWorkerIdMap.get(timeChunk)); + } hasMultipleValues = hasMultipleValues || other.hasMultipleValues; } - @JsonProperty("workerIds") - public Set getWorkerIds() + @JsonProperty("timeSegmentVsWorkerIdMap") + public SortedMap> getTimeSegmentVsWorkerIdMap() { - return workerIds; + return timeSegmentVsWorkerIdMap; } @JsonProperty("hasMultipleValues") diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java index cad1147ecdac..e2197f923ba5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.segment.TestHelper; @@ -29,6 +30,8 @@ import org.junit.Before; import org.junit.Test; +import java.util.TreeMap; + public class ClusterByStatisticsWorkerReportSerdeTest { private ObjectMapper objectMapper; @@ -45,7 +48,7 @@ public void setUp() public void testSerde() throws JsonProcessingException { ClusterByStatisticsWorkerReport workerReport = new ClusterByStatisticsWorkerReport( - ImmutableSet.of(1, 3, 4), + new TreeMap<>(ImmutableMap.of(2L, ImmutableSet.of(2, 3))), false ); @@ -54,7 +57,7 @@ public void testSerde() throws JsonProcessingException json, ClusterByStatisticsWorkerReport.class ); - Assert.assertEquals(json, workerReport.getWorkerIds(), deserializedWorkerReport.getWorkerIds()); + Assert.assertEquals(json, workerReport.getTimeSegmentVsWorkerIdMap(), deserializedWorkerReport.getTimeSegmentVsWorkerIdMap()); Assert.assertEquals(json, workerReport.isHasMultipleValues(), deserializedWorkerReport.isHasMultipleValues()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java index 751bf07963c2..f7d092f21a62 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java @@ -60,6 +60,18 @@ public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String worke return inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId); } + @Override + public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ) throws ExecutionException, InterruptedException + { + StageId stageId = new StageId(queryId, stageNumber); + return inMemoryWorkers.get(workerTaskId).fetchSingletonStatisticsSnapshot(stageId, timeChunk); + } + @Override public ListenableFuture postResultPartitionBoundaries( String workerTaskId, diff --git a/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java b/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java index d3ca6d035c49..3102b103c9c7 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java +++ b/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java @@ -130,6 +130,28 @@ public boolean hasMultipleValues(final RowKey key, final int fieldNumber) } } + /** + * Trims the key reader to a particular fieldCount. Used to read keys trimmed by {@link #trim(RowKey, int)}. + */ + public RowKeyReader trimmedKeyReader(int trimmedFieldCount) + { + final RowSignature.Builder newSignature = RowSignature.builder(); + + if (trimmedFieldCount > signature.size()) { + throw new IAE("Cannot trim to [%,d] fields, only have [%,d] fields", trimmedFieldCount, signature); + } + + for (int i = 0; i < trimmedFieldCount; i++) { + final String columnName = signature.getColumnName(i); + final ColumnType columnType = + Preconditions.checkNotNull(signature.getColumnType(i).orElse(null), "Type for column [%s]", columnName); + + newSignature.add(columnName, columnType); + } + + return RowKeyReader.create(newSignature.build()); + } + /** * Trim a key to a particular fieldCount. The returned key may be a copy, but is not guaranteed to be. */ From 113a3d9b93bbfc151ad8d880dd47f9e7872faa42 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 13 Oct 2022 17:58:12 +0530 Subject: [PATCH 03/31] Update worker sketch fetcher --- .../druid/msq/exec/WorkerSketchFetcher.java | 170 +++++++++++++++--- 1 file changed, 146 insertions(+), 24 deletions(-) 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 4dca9641bba6..93ff065c871a 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,8 +19,8 @@ package org.apache.druid.msq.exec; -import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; -import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.java.util.common.Either; import org.apache.druid.msq.kernel.StageDefinition; @@ -28,18 +28,22 @@ import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.SortedMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; /** * Queues up fetching sketches from workers and progressively generates partitions boundaries. - * TODO: still in progress. Some cleanup required as things will likely move around with sequantial merging. */ public class WorkerSketchFetcher { private static final int DEFAULT_THREAD_COUNT = 10; + private static final boolean SEQUENTIAL_MERGING = true; private final WorkerClient workerClient; private final ExecutorService executorService; @@ -50,38 +54,156 @@ public WorkerSketchFetcher(WorkerClient workerClient) this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); } + /** + * Submits a request to fetch and generate partitions for the given worker report and returns a future for it. It + * decides based on the report if it should fetch sketches one by one or together. + */ public CompletableFuture> submitFetcherTask( ClusterByStatisticsWorkerReport workerReport, List workerTaskIds, StageDefinition stageDefinition) + { + ClusterBy clusterBy = stageDefinition.getClusterBy(); + + if (clusterBy.getBucketByCount() == 0 || !SEQUENTIAL_MERGING) { + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + } else { + return sequentialTimeChunkMerging(workerReport, stageDefinition, workerTaskIds); + } + } + + /** + * 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. + */ + private CompletableFuture> sequentialTimeChunkMerging( + ClusterByStatisticsWorkerReport workerReport, + StageDefinition stageDefinition, + List workerTaskIds + ) + { + CompletableFuture> partitionFuture = new CompletableFuture<>(); + List finalRanges = new ArrayList<>(); + + SortedMap> timeSegmentVsWorkerIdMap = workerReport.getTimeSegmentVsWorkerIdMap(); + submitFetchingTasksForTimeChunk(partitionFuture, timeSegmentVsWorkerIdMap, stageDefinition, workerTaskIds, finalRanges); + return partitionFuture; + } + + private void submitFetchingTasksForTimeChunk( + CompletableFuture> partitionFuture, + SortedMap> timeSegmentVsWorkerIdMap, + StageDefinition stageDefinition, + List workerTaskIds, + List finalRanges + ) + { + if (!timeSegmentVsWorkerIdMap.isEmpty()) { + final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); + final Set finishedWorkers = new HashSet<>(); + + Long timeChunk = timeSegmentVsWorkerIdMap.firstKey(); + Set workerIdsWithTimeChunk = timeSegmentVsWorkerIdMap.get(timeChunk); + timeSegmentVsWorkerIdMap.remove(timeChunk); + + for (int workerNo : workerIdsWithTimeChunk) { + executorService.submit(new Runnable() + { + @Override + public void run() + { + try { + ClusterByStatisticsSnapshot singletonStatisticsSnapshot = + workerClient.fetchSingletonStatisticsSnapshot( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber(), + timeChunk + ); + // If the future already failed for some reason, stop the task. + if (partitionFuture.isDone()) { + return; + } + + synchronized (mergedStatisticsCollector) { + mergedStatisticsCollector.addAll(singletonStatisticsSnapshot); + finishedWorkers.add(workerNo); + + if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) { + Either longClusterByPartitionsEither = stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector); + if (longClusterByPartitionsEither.isError()) { + partitionFuture.complete(longClusterByPartitionsEither); + } + List partitions = stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector).valueOrThrow().ranges(); + if (!finalRanges.isEmpty()) { + ClusterByPartition clusterByPartition = finalRanges.get(finalRanges.size() - 1); + finalRanges.remove(finalRanges.size() - 1); + finalRanges.add(new ClusterByPartition(clusterByPartition.getStart(), partitions.get(0).getStart())); + } + finalRanges.addAll(partitions); + submitFetchingTasksForTimeChunk(partitionFuture, timeSegmentVsWorkerIdMap, stageDefinition, workerTaskIds, finalRanges); + } + } + } + catch (Exception e) { + partitionFuture.completeExceptionally(e); + } + } + }); + } + } else { + partitionFuture.complete(Either.value(new ClusterByPartitions(finalRanges))); + } + } + + /** + * 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. + */ + private CompletableFuture> inMemoryFullSketchMerging( + StageDefinition stageDefinition, + List workerTaskIds + ) { CompletableFuture> partitionFuture = new CompletableFuture<>(); - ClusterByStatisticsCollector finalClusterByStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); - int workerNumber = stageDefinition.getMaxWorkerCount(); - IntSet finishedWorkers = new IntAVLTreeSet(); + final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); + final int workerCount = stageDefinition.getMaxWorkerCount(); + final Set finishedWorkers = new HashSet<>(); - for (int i = 0; i < workerNumber; i++) { + for (int i = 0; i < workerCount; i++) { final int workerNo = i; - executorService.submit(() -> { - try { - ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = workerClient.fetchClusterByStatisticsSnapshot( - workerTaskIds.get(workerNo), - stageDefinition.getId().getQueryId(), - stageDefinition.getStageNumber() - ); - // If the future already failed for some reason, skip ahead. - if (!partitionFuture.isDone()) { - finalClusterByStatisticsCollector.addAll(clusterByStatisticsSnapshot); - finishedWorkers.add(workerNo); - if (finishedWorkers.size() == workerNumber) { - partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(finalClusterByStatisticsCollector)); + executorService.submit(new Runnable() + { + @Override + public void run() + { + try { + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = workerClient.fetchClusterByStatisticsSnapshot( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber() + ); + + // If the future already failed for some reason, stop the task. + if (partitionFuture.isDone()) { + return; + } + + synchronized (mergedStatisticsCollector) { + mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); + finishedWorkers.add(workerNo); + + if (finishedWorkers.size() == workerCount) { + partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); + } } } - } - catch (Exception e) { - finalClusterByStatisticsCollector.clear(); - partitionFuture.completeExceptionally(e); + catch (Exception e) { + partitionFuture.completeExceptionally(e); + } } }); } From 9fce02a9f7e2620e4bbb19b9220866044f566dc3 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 13 Oct 2022 21:48:06 +0530 Subject: [PATCH 04/31] Refactor sketch fetcher --- .../druid/msq/exec/WorkerSketchFetcher.java | 189 ++++++++++-------- 1 file changed, 102 insertions(+), 87 deletions(-) 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 93ff065c871a..92b693731e2b 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 @@ -73,46 +73,107 @@ public CompletableFuture> submitFetcherTask( } /** - * 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. + * 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. */ - private CompletableFuture> sequentialTimeChunkMerging( - ClusterByStatisticsWorkerReport workerReport, + private CompletableFuture> inMemoryFullSketchMerging( StageDefinition stageDefinition, List workerTaskIds ) { CompletableFuture> partitionFuture = new CompletableFuture<>(); - List finalRanges = new ArrayList<>(); - SortedMap> timeSegmentVsWorkerIdMap = workerReport.getTimeSegmentVsWorkerIdMap(); - submitFetchingTasksForTimeChunk(partitionFuture, timeSegmentVsWorkerIdMap, stageDefinition, workerTaskIds, finalRanges); + final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); + final int workerCount = stageDefinition.getMaxWorkerCount(); + final Set finishedWorkers = new HashSet<>(); + + for (int i = 0; i < workerCount; i++) { + final int workerNo = i; + executorService.submit(() -> { + try { + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = workerClient.fetchClusterByStatisticsSnapshot( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber() + ); + + // If the future already failed for some reason, stop the task. + if (partitionFuture.isDone()) { + return; + } + + synchronized (mergedStatisticsCollector) { + mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); + finishedWorkers.add(workerNo); + + if (finishedWorkers.size() == workerCount) { + partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); + } + } + } + catch (Exception e) { + partitionFuture.completeExceptionally(e); + } + }); + } return partitionFuture; } - private void submitFetchingTasksForTimeChunk( - CompletableFuture> partitionFuture, - SortedMap> timeSegmentVsWorkerIdMap, + /** + * 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. + */ + private CompletableFuture> sequentialTimeChunkMerging( + ClusterByStatisticsWorkerReport workerReport, StageDefinition stageDefinition, - List workerTaskIds, - List finalRanges + List workerTaskIds ) { - if (!timeSegmentVsWorkerIdMap.isEmpty()) { - final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); - final Set finishedWorkers = new HashSet<>(); - - Long timeChunk = timeSegmentVsWorkerIdMap.firstKey(); - Set workerIdsWithTimeChunk = timeSegmentVsWorkerIdMap.get(timeChunk); - timeSegmentVsWorkerIdMap.remove(timeChunk); - - for (int workerNo : workerIdsWithTimeChunk) { - executorService.submit(new Runnable() - { - @Override - public void run() - { + SequentialFetchStage sequentialFetchStage = new SequentialFetchStage( + stageDefinition, + workerTaskIds, + workerReport.getTimeSegmentVsWorkerIdMap() + ); + sequentialFetchStage.submitFetchingTasksForNextTimeChunk(); + return sequentialFetchStage.getPartitionFuture(); + } + + private class SequentialFetchStage + { + private final StageDefinition stageDefinition; + private final List workerTaskIds; + private final SortedMap> timeSegmentVsWorkerIdMap; + private final CompletableFuture> partitionFuture; + private final List finalRanges; + + public SequentialFetchStage( + StageDefinition stageDefinition, + List workerTaskIds, + SortedMap> timeSegmentVsWorkerIdMap + ) + { + this.finalRanges = new ArrayList<>(); + this.stageDefinition = stageDefinition; + this.workerTaskIds = workerTaskIds; + this.timeSegmentVsWorkerIdMap = timeSegmentVsWorkerIdMap; + this.partitionFuture = new CompletableFuture<>(); + } + + public void submitFetchingTasksForNextTimeChunk() + { + if (timeSegmentVsWorkerIdMap.isEmpty()) { + partitionFuture.complete(Either.value(new ClusterByPartitions(finalRanges))); + } else { + Long timeChunk = timeSegmentVsWorkerIdMap.firstKey(); + Set workerIdsWithTimeChunk = timeSegmentVsWorkerIdMap.get(timeChunk); + timeSegmentVsWorkerIdMap.remove(timeChunk); + ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); + Set finishedWorkers = new HashSet<>(); + + for (int workerNo : workerIdsWithTimeChunk) { + executorService.submit(() -> { try { ClusterByStatisticsSnapshot singletonStatisticsSnapshot = workerClient.fetchSingletonStatisticsSnapshot( @@ -125,88 +186,42 @@ public void run() if (partitionFuture.isDone()) { return; } - synchronized (mergedStatisticsCollector) { mergedStatisticsCollector.addAll(singletonStatisticsSnapshot); finishedWorkers.add(workerNo); if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) { - Either longClusterByPartitionsEither = stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector); + Either longClusterByPartitionsEither = stageDefinition.generatePartitionsForShuffle( + mergedStatisticsCollector); if (longClusterByPartitionsEither.isError()) { partitionFuture.complete(longClusterByPartitionsEither); } - List partitions = stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector).valueOrThrow().ranges(); + List partitions = stageDefinition.generatePartitionsForShuffle( + mergedStatisticsCollector).valueOrThrow().ranges(); if (!finalRanges.isEmpty()) { ClusterByPartition clusterByPartition = finalRanges.get(finalRanges.size() - 1); finalRanges.remove(finalRanges.size() - 1); - finalRanges.add(new ClusterByPartition(clusterByPartition.getStart(), partitions.get(0).getStart())); + finalRanges.add(new ClusterByPartition( + clusterByPartition.getStart(), + partitions.get(0).getStart() + )); } finalRanges.addAll(partitions); - submitFetchingTasksForTimeChunk(partitionFuture, timeSegmentVsWorkerIdMap, stageDefinition, workerTaskIds, finalRanges); + submitFetchingTasksForNextTimeChunk(); } } } catch (Exception e) { partitionFuture.completeExceptionally(e); } - } - }); + }); + } } - } else { - partitionFuture.complete(Either.value(new ClusterByPartitions(finalRanges))); } - } - /** - * 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. - */ - private CompletableFuture> inMemoryFullSketchMerging( - StageDefinition stageDefinition, - List workerTaskIds - ) - { - CompletableFuture> partitionFuture = new CompletableFuture<>(); - - final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); - final int workerCount = stageDefinition.getMaxWorkerCount(); - final Set finishedWorkers = new HashSet<>(); - - for (int i = 0; i < workerCount; i++) { - final int workerNo = i; - executorService.submit(new Runnable() - { - @Override - public void run() - { - try { - ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = workerClient.fetchClusterByStatisticsSnapshot( - workerTaskIds.get(workerNo), - stageDefinition.getId().getQueryId(), - stageDefinition.getStageNumber() - ); - - // If the future already failed for some reason, stop the task. - if (partitionFuture.isDone()) { - return; - } - - synchronized (mergedStatisticsCollector) { - mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); - finishedWorkers.add(workerNo); - - if (finishedWorkers.size() == workerCount) { - partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); - } - } - } - catch (Exception e) { - partitionFuture.completeExceptionally(e); - } - } - }); + public CompletableFuture> getPartitionFuture() + { + return partitionFuture; } - return partitionFuture; } } From 1657ee095ff0cfdf009ef9c074e6f861f7edbd83 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 13 Oct 2022 22:51:14 +0530 Subject: [PATCH 05/31] Refactor sketch fetcher --- .../druid/msq/exec/WorkerSketchFetcher.java | 60 +++++++++++-------- 1 file changed, 35 insertions(+), 25 deletions(-) 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 92b693731e2b..7e23b15749e7 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 @@ -30,9 +30,10 @@ import java.util.ArrayList; import java.util.HashSet; +import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.SortedMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -134,7 +135,7 @@ private CompletableFuture> sequentialTimeChunk SequentialFetchStage sequentialFetchStage = new SequentialFetchStage( stageDefinition, workerTaskIds, - workerReport.getTimeSegmentVsWorkerIdMap() + workerReport.getTimeSegmentVsWorkerIdMap().entrySet().iterator() ); sequentialFetchStage.submitFetchingTasksForNextTimeChunk(); return sequentialFetchStage.getPartitionFuture(); @@ -144,31 +145,31 @@ private class SequentialFetchStage { private final StageDefinition stageDefinition; private final List workerTaskIds; - private final SortedMap> timeSegmentVsWorkerIdMap; + private final Iterator>> timeSegmentVsWorkerIdIterator; private final CompletableFuture> partitionFuture; - private final List finalRanges; + private final List finalPartitionBoundries; public SequentialFetchStage( StageDefinition stageDefinition, List workerTaskIds, - SortedMap> timeSegmentVsWorkerIdMap + Iterator>> timeSegmentVsWorkerIdIterator ) { - this.finalRanges = new ArrayList<>(); + this.finalPartitionBoundries = new ArrayList<>(); this.stageDefinition = stageDefinition; this.workerTaskIds = workerTaskIds; - this.timeSegmentVsWorkerIdMap = timeSegmentVsWorkerIdMap; + this.timeSegmentVsWorkerIdIterator = timeSegmentVsWorkerIdIterator; this.partitionFuture = new CompletableFuture<>(); } public void submitFetchingTasksForNextTimeChunk() { - if (timeSegmentVsWorkerIdMap.isEmpty()) { - partitionFuture.complete(Either.value(new ClusterByPartitions(finalRanges))); + if (!timeSegmentVsWorkerIdIterator.hasNext()) { + partitionFuture.complete(Either.value(new ClusterByPartitions(finalPartitionBoundries))); } else { - Long timeChunk = timeSegmentVsWorkerIdMap.firstKey(); - Set workerIdsWithTimeChunk = timeSegmentVsWorkerIdMap.get(timeChunk); - timeSegmentVsWorkerIdMap.remove(timeChunk); + Map.Entry> entry = timeSegmentVsWorkerIdIterator.next(); + Long timeChunk = entry.getKey(); + Set workerIdsWithTimeChunk = entry.getValue(); ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); Set finishedWorkers = new HashSet<>(); @@ -191,22 +192,19 @@ public void submitFetchingTasksForNextTimeChunk() finishedWorkers.add(workerNo); if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) { - Either longClusterByPartitionsEither = stageDefinition.generatePartitionsForShuffle( - mergedStatisticsCollector); + Either longClusterByPartitionsEither = + stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector); + if (longClusterByPartitionsEither.isError()) { partitionFuture.complete(longClusterByPartitionsEither); } - List partitions = stageDefinition.generatePartitionsForShuffle( - mergedStatisticsCollector).valueOrThrow().ranges(); - if (!finalRanges.isEmpty()) { - ClusterByPartition clusterByPartition = finalRanges.get(finalRanges.size() - 1); - finalRanges.remove(finalRanges.size() - 1); - finalRanges.add(new ClusterByPartition( - clusterByPartition.getStart(), - partitions.get(0).getStart() - )); - } - finalRanges.addAll(partitions); + + List timeSketchpartitions = + stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector) + .valueOrThrow() + .ranges(); + abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchpartitions); + submitFetchingTasksForNextTimeChunk(); } } @@ -219,6 +217,18 @@ public void submitFetchingTasksForNextTimeChunk() } } + private void abutAndAppendPartitionBoundries( + List finalPartitionBoundries, + List timeSketchpartitions + ) + { + if (!finalPartitionBoundries.isEmpty()) { + 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; From 313e9a849a7203bc58a9732ed72cefb63d63bc02 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 17 Oct 2022 16:37:14 +0530 Subject: [PATCH 06/31] Add context parameter and threshold to trigger sequential merge --- .../apache/druid/msq/exec/ControllerImpl.java | 5 +++- .../druid/msq/exec/WorkerSketchFetcher.java | 24 ++++++++++++------- .../druid/msq/kernel/StageDefinition.java | 2 +- .../controller/ControllerStageTracker.java | 2 +- .../ClusterByStatisticsCollectorImpl.java | 8 +++---- .../ClusterByStatisticsSnapshot.java | 9 +++++-- .../ClusterByStatisticsWorkerReport.java | 20 ++++++++++++---- ...sterByStatisticsWorkerReportSerdeTest.java | 4 +++- 8 files changed, 51 insertions(+), 23 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 c4c8d67b3915..51dea9d82bf3 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 @@ -519,7 +519,10 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); - this.workerSketchFetcher = new WorkerSketchFetcher(netClient); + Boolean forceNonSequentialMerging = (Boolean) task.getSqlQueryContext() + .getOrDefault("msqForceNonSequentialMerging", false); + this.workerSketchFetcher = new WorkerSketchFetcher(netClient, forceNonSequentialMerging); + closer.register(netClient::close); final boolean isDurableStorageEnabled = 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 7e23b15749e7..d2919aa7d55c 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 @@ -44,14 +44,17 @@ public class WorkerSketchFetcher { private static final int DEFAULT_THREAD_COUNT = 10; - private static final boolean SEQUENTIAL_MERGING = true; + private static final long BYTES_THRESHOLD = 1_000_000_000L; + private static final long WORKER_THRESHOLD = 100; + private final boolean forceNonSequentialMerging; private final WorkerClient workerClient; private final ExecutorService executorService; - public WorkerSketchFetcher(WorkerClient workerClient) + public WorkerSketchFetcher(WorkerClient workerClient, boolean forceNonSequentialMerging) { this.workerClient = workerClient; + this.forceNonSequentialMerging = forceNonSequentialMerging; this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); } @@ -62,11 +65,14 @@ public WorkerSketchFetcher(WorkerClient workerClient) public CompletableFuture> submitFetcherTask( ClusterByStatisticsWorkerReport workerReport, List workerTaskIds, - StageDefinition stageDefinition) + StageDefinition stageDefinition + ) { ClusterBy clusterBy = stageDefinition.getClusterBy(); - if (clusterBy.getBucketByCount() == 0 || !SEQUENTIAL_MERGING) { + if (forceNonSequentialMerging || clusterBy.getBucketByCount() == 0) { + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || workerReport.getBytesRetained() > BYTES_THRESHOLD) { return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); } else { return sequentialTimeChunkMerging(workerReport, stageDefinition, workerTaskIds); @@ -199,11 +205,11 @@ public void submitFetchingTasksForNextTimeChunk() partitionFuture.complete(longClusterByPartitionsEither); } - List timeSketchpartitions = + List timeSketchPartitions = stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector) .valueOrThrow() .ranges(); - abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchpartitions); + abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchPartitions); submitFetchingTasksForNextTimeChunk(); } @@ -219,14 +225,14 @@ public void submitFetchingTasksForNextTimeChunk() private void abutAndAppendPartitionBoundries( List finalPartitionBoundries, - List timeSketchpartitions + List timeSketchPartitions ) { if (!finalPartitionBoundries.isEmpty()) { ClusterByPartition clusterByPartition = finalPartitionBoundries.remove(finalPartitionBoundries.size() - 1); - finalPartitionBoundries.add(new ClusterByPartition(clusterByPartition.getStart(), timeSketchpartitions.get(0).getStart())); + finalPartitionBoundries.add(new ClusterByPartition(clusterByPartition.getStart(), timeSketchPartitions.get(0).getStart())); } - finalPartitionBoundries.addAll(timeSketchpartitions); + finalPartitionBoundries.addAll(timeSketchPartitions); } public CompletableFuture> getPartitionFuture() 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 c01506054d09..a70ec870d912 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 @@ -74,7 +74,7 @@ */ public class StageDefinition { - private static final int PARTITION_STATS_MAX_BYTES = 300_000_000; // Avoid immediate downsample of single-bucket collectors + private static final long PARTITION_STATS_MAX_BYTES = Runtime.getRuntime().totalMemory() / 10; // 10% of heap. private static final int PARTITION_STATS_MAX_BUCKETS = 5_000; // Limit for TooManyBuckets private static final int MAX_PARTITIONS = 25_000; // Limit for TooManyPartitions 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 6886933a4586..9fce2a0e925b 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 @@ -89,7 +89,7 @@ private ControllerStageTracker( this.workerInputs = workerInputs; if (stageDef.mustGatherResultKeyStatistics()) { - this.clusterByStatisticsWorkerReport = new ClusterByStatisticsWorkerReport(new TreeMap<>(), false); + this.clusterByStatisticsWorkerReport = new ClusterByStatisticsWorkerReport(new TreeMap<>(), false, 0); } else { this.clusterByStatisticsWorkerReport = null; generateResultPartitionsAndBoundariesWithoutKeyStatistics(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java index d30df2fe6ab2..65fd04ee29e9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java @@ -57,7 +57,7 @@ public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsColl private final boolean[] hasMultipleValues; - private final int maxRetainedBytes; + private final long maxRetainedBytes; private final int maxBuckets; private double totalRetainedBytes; @@ -65,7 +65,7 @@ private ClusterByStatisticsCollectorImpl( final ClusterBy clusterBy, final RowKeyReader keyReader, final KeyCollectorFactory keyCollectorFactory, - final int maxRetainedBytes, + final long maxRetainedBytes, final int maxBuckets, final boolean checkHasMultipleValues ) @@ -87,7 +87,7 @@ private ClusterByStatisticsCollectorImpl( public static ClusterByStatisticsCollector create( final ClusterBy clusterBy, final RowSignature signature, - final int maxRetainedBytes, + final long maxRetainedBytes, final int maxBuckets, final boolean aggregate, final boolean checkHasMultipleValues @@ -327,7 +327,7 @@ public ClusterByStatisticsSnapshot snapshot() if (clusterBy.getBucketByCount() == 1) { bucketKey = (Long) trimmedRowReader.read(bucketEntry.getKey(), 0); } - bucketSnapshots.put(bucketKey, new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot)); + bucketSnapshots.put(bucketKey, new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot, totalRetainedBytes)); } final IntSet hasMultipleValuesSet; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index 048787dd254f..74221585cdc7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -77,12 +77,14 @@ Set getHasMultipleValues() public ClusterByStatisticsWorkerReport workerReport(int workerNumber) { SortedMap> rowKeyIntSetTreeSet = new TreeMap<>(); + double bytesRetained = 0; for (Long bucketKey : buckets.keySet()) { rowKeyIntSetTreeSet.put(bucketKey, ImmutableSet.of(workerNumber)); + bytesRetained += buckets.get(bucketKey).bytesRetained; } - return new ClusterByStatisticsWorkerReport(rowKeyIntSetTreeSet, !getHasMultipleValues().isEmpty()); + return new ClusterByStatisticsWorkerReport(rowKeyIntSetTreeSet, !getHasMultipleValues().isEmpty(), bytesRetained); } @Override @@ -107,16 +109,19 @@ public int hashCode() static class Bucket { private final RowKey bucketKey; + private final double bytesRetained; private final KeyCollectorSnapshot keyCollectorSnapshot; @JsonCreator Bucket( @JsonProperty("bucketKey") RowKey bucketKey, - @JsonProperty("data") KeyCollectorSnapshot keyCollectorSnapshot + @JsonProperty("data") KeyCollectorSnapshot keyCollectorSnapshot, + @JsonProperty("bytesRetained") double bytesRetained ) { this.bucketKey = Preconditions.checkNotNull(bucketKey, "bucketKey"); this.keyCollectorSnapshot = Preconditions.checkNotNull(keyCollectorSnapshot, "data"); + this.bytesRetained = bytesRetained; } @JsonProperty diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java index ee1201a67d2f..c8cb89dc171d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java @@ -35,23 +35,29 @@ public class ClusterByStatisticsWorkerReport private Boolean hasMultipleValues; + private double bytesRetained; + @JsonCreator public ClusterByStatisticsWorkerReport( @JsonProperty("timeSegmentVsWorkerIdMap") final SortedMap> timeChunks, - @JsonProperty("hasMultipleValues") boolean hasMultipleValues + @JsonProperty("hasMultipleValues") boolean hasMultipleValues, + @JsonProperty("bytesRetained") double bytesRetained ) { this.timeSegmentVsWorkerIdMap = timeChunks; this.hasMultipleValues = hasMultipleValues; + this.bytesRetained = bytesRetained; } public void addAll(ClusterByStatisticsWorkerReport other) { for (Long timeChunk : other.timeSegmentVsWorkerIdMap.keySet()) { - timeSegmentVsWorkerIdMap.computeIfAbsent(timeChunk, key -> new HashSet<>()) - .addAll(other.timeSegmentVsWorkerIdMap.get(timeChunk)); + this.timeSegmentVsWorkerIdMap + .computeIfAbsent(timeChunk, key -> new HashSet<>()) + .addAll(other.timeSegmentVsWorkerIdMap.get(timeChunk)); } - hasMultipleValues = hasMultipleValues || other.hasMultipleValues; + this.hasMultipleValues = this.hasMultipleValues || other.hasMultipleValues; + this.bytesRetained += bytesRetained; } @JsonProperty("timeSegmentVsWorkerIdMap") @@ -65,4 +71,10 @@ public boolean isHasMultipleValues() { return hasMultipleValues; } + + @JsonProperty("bytesRetained") + public double getBytesRetained() + { + return bytesRetained; + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java index e2197f923ba5..11f4b37cb0f3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java @@ -49,7 +49,8 @@ public void testSerde() throws JsonProcessingException { ClusterByStatisticsWorkerReport workerReport = new ClusterByStatisticsWorkerReport( new TreeMap<>(ImmutableMap.of(2L, ImmutableSet.of(2, 3))), - false + false, + 0.0 ); final String json = objectMapper.writeValueAsString(workerReport); @@ -59,5 +60,6 @@ public void testSerde() throws JsonProcessingException ); Assert.assertEquals(json, workerReport.getTimeSegmentVsWorkerIdMap(), deserializedWorkerReport.getTimeSegmentVsWorkerIdMap()); Assert.assertEquals(json, workerReport.isHasMultipleValues(), deserializedWorkerReport.isHasMultipleValues()); + Assert.assertEquals(json, workerReport.getBytesRetained(), deserializedWorkerReport.getBytesRetained()); } } From 0be3d4f294218dab2dd223f2b4a152608ab11be7 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 17 Oct 2022 16:44:06 +0530 Subject: [PATCH 07/31] Fix test --- .../statistics/ClusterByStatisticsWorkerReportSerdeTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java index 11f4b37cb0f3..d658f54e5528 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java @@ -60,6 +60,6 @@ public void testSerde() throws JsonProcessingException ); Assert.assertEquals(json, workerReport.getTimeSegmentVsWorkerIdMap(), deserializedWorkerReport.getTimeSegmentVsWorkerIdMap()); Assert.assertEquals(json, workerReport.isHasMultipleValues(), deserializedWorkerReport.isHasMultipleValues()); - Assert.assertEquals(json, workerReport.getBytesRetained(), deserializedWorkerReport.getBytesRetained()); + Assert.assertEquals(json, workerReport.getBytesRetained(), deserializedWorkerReport.getBytesRetained(), 0); } } From a312f208ffdbfde3740615761a057578caa121fd Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 18 Oct 2022 16:17:15 +0530 Subject: [PATCH 08/31] Add integration test for non sequential merge --- .../apache/druid/msq/exec/ControllerImpl.java | 3 +- .../msq/util/MultiStageQueryContext.java | 15 +++++ integration-tests-ex/README.md | 2 +- .../druid/testsEx/msq/ITMultiStageQuery.java | 67 +++++++++++++++++++ 4 files changed, 84 insertions(+), 3 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 51dea9d82bf3..abcbf03c2f2b 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 @@ -519,8 +519,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); - Boolean forceNonSequentialMerging = (Boolean) task.getSqlQueryContext() - .getOrDefault("msqForceNonSequentialMerging", false); + boolean forceNonSequentialMerging = MultiStageQueryContext.isForceSequentialMerge(task.getSqlQueryContext()); this.workerSketchFetcher = new WorkerSketchFetcher(netClient, forceNonSequentialMerging); closer.register(netClient::close); 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 7a6b576e68d9..2ab068e21e80 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 @@ -59,6 +59,7 @@ public class MultiStageQueryContext private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; + public static final String CTX_FORCE_NON_SEQUENTIAL_MERGE = "forceNonSequentialMerging"; private static final String DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = "false"; public static final String CTX_DESTINATION = "destination"; @@ -101,6 +102,20 @@ public static boolean isDurableStorageEnabled(Map propertyMap) ); } + public static boolean isForceSequentialMerge(Map propertyMap) + { + return Boolean.parseBoolean( + String.valueOf( + getValueFromPropertyMap( + propertyMap, + CTX_FORCE_NON_SEQUENTIAL_MERGE, + null, + false + ) + ) + ); + } + public static boolean isFinalizeAggregations(final QueryContext queryContext) { return Numbers.parseBoolean( diff --git a/integration-tests-ex/README.md b/integration-tests-ex/README.md index 9c29ec101fe9..3a17c6e7c700 100644 --- a/integration-tests-ex/README.md +++ b/integration-tests-ex/README.md @@ -71,7 +71,7 @@ Start the cluster: ```bash cd $DRUID_DEV/integration-tests-ex/cases -./cluster.sh up +./cluster.sh up ``` Where `` is one of the test categories. Then launch the 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..8d88fa15e893 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 @@ -20,9 +20,11 @@ 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.StringUtils; import org.apache.druid.msq.sql.SqlTaskStatus; +import org.apache.druid.sql.http.SqlQuery; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; import org.apache.druid.testing.clients.SqlResourceTestClient; @@ -122,4 +124,69 @@ public void testMsqIngestionAndQuerying() throws Exception msqHelper.testQueriesFromFile(QUERY_FILE, datasource); } + + @Test + public void testMsqIngestionWithoutSequentialMerging() 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\"]}',\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 + SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, ImmutableMap.of("msqForceNonSequentialMerging", true), null); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } } From c74da2cdb53eaf0639dcdb53902a4f329ca04239 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 26 Oct 2022 09:25:40 +0530 Subject: [PATCH 09/31] Address review comments --- .../org/apache/druid/msq/exec/Controller.java | 8 ++++--- .../druid/msq/exec/ControllerClient.java | 4 ++-- .../apache/druid/msq/exec/ControllerImpl.java | 14 ++++++------ .../org/apache/druid/msq/exec/Worker.java | 22 +++++++++---------- .../org/apache/druid/msq/exec/WorkerImpl.java | 8 +++---- .../druid/msq/exec/WorkerSketchFetcher.java | 12 +++++----- .../msq/indexing/ControllerChatHandler.java | 6 ++--- .../msq/indexing/IndexerControllerClient.java | 6 ++--- .../controller/ControllerQueryKernel.java | 8 +++---- .../controller/ControllerStageTracker.java | 22 +++++++++---------- .../ClusterByStatisticsCollectorImpl.java | 2 ++ .../ClusterByStatisticsSnapshot.java | 4 ++-- ...ava => WorkerAggregatedKeyStatistics.java} | 8 +++---- .../BaseControllerQueryKernelTest.java | 4 ++-- ...sterByStatisticsWorkerReportSerdeTest.java | 12 +++++----- .../msq/test/MSQTestControllerClient.java | 6 ++--- 16 files changed, 75 insertions(+), 71 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/{ClusterByStatisticsWorkerReport.java => WorkerAggregatedKeyStatistics.java} (92%) 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 97363aa7c4eb..583afed2e13c 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 @@ -27,7 +27,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.error.MSQErrorReport; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import javax.annotation.Nullable; import java.util.List; @@ -81,9 +81,11 @@ public String getId() // Worker-to-controller messages /** - * Accepts a {@link ClusterByStatisticsWorkerReport} for generating fetching sketches. + * Accepts a {@link WorkerAggregatedKeyStatistics} and updates the controller aggregated key statistics. If all key + * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate key statistics. + * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ - void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerReport); + void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerStatisticsReport); /** * System error reported by a subtask. Note that the errors are organized by 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 924bfc02c29c..89ea3979d9c4 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 @@ -22,7 +22,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import javax.annotation.Nullable; import java.io.IOException; @@ -40,7 +40,7 @@ public interface ControllerClient extends AutoCloseable void postWorkerReport( StageId stageId, int workerNumber, - ClusterByStatisticsWorkerReport workerReport + WorkerAggregatedKeyStatistics workerReport ) throws IOException; /** 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 83623e73adb1..d070d7d7b9a5 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 @@ -149,7 +149,7 @@ import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory; import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory; import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import org.apache.druid.msq.util.DimensionSchemaUtils; import org.apache.druid.msq.util.IntervalUtils; import org.apache.druid.msq.util.MSQFutureUtils; @@ -567,10 +567,10 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) } /** - * Provide a {@link ClusterByStatisticsWorkerReport} for shuffling stages. + * Provide a {@link WorkerAggregatedKeyStatistics} for shuffling stages. */ @Override - public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerReportObject) + public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerStatisticsReport) { addToKernelManipulationQueue( queryKernel -> { @@ -584,9 +584,9 @@ public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object w stageDef.getShuffleSpec().get().doesAggregateByClusterKey() ); - final ClusterByStatisticsWorkerReport workerReport; + final WorkerAggregatedKeyStatistics workerReport; try { - workerReport = mapper.convertValue(workerReportObject, ClusterByStatisticsWorkerReport.class); + workerReport = mapper.convertValue(workerStatisticsReport, WorkerAggregatedKeyStatistics.class); } catch (IllegalArgumentException e) { throw new IAE( @@ -601,12 +601,12 @@ public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object w if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) { List workerTaskIds = workerTaskLauncher.getTaskList(); - ClusterByStatisticsWorkerReport finalWorkerReport = queryKernel.getClusterByStatisticsWorkerReport(stageId); + WorkerAggregatedKeyStatistics aggregatedKeyStatistics = queryKernel.getClusterByStatisticsWorkerReport(stageId); // Queue the sketch fetching task into the worker sketch fetcher. CompletableFuture> clusterByPartitionsCompletableFuture = workerSketchFetcher.submitFetcherTask( - finalWorkerReport, + aggregatedKeyStatistics, workerTaskIds, stageDef ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index 5c1aab736f16..9081f5e7ee6f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -69,6 +69,17 @@ public interface Worker */ void postWorkOrder(WorkOrder workOrder); + /** + * Returns the statistics snapshot for the given stageId + */ + ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) throws ExecutionException, InterruptedException; + + /** + * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk + */ + ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(StageId stageId, long timeChunk) + throws ExecutionException, InterruptedException; + /** * Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber * and queryId @@ -107,15 +118,4 @@ boolean postResultPartitionBoundaries( * Called when the work required for the query has been finished */ void postFinish(); - - /** - * Returns the statistics snapshot for the given stageId - */ - ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) throws ExecutionException, InterruptedException; - - /** - * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk - */ - ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(StageId stageId, long timeChunk) - throws ExecutionException, InterruptedException; } 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 b0f36550ed0b..1a060c3c9def 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 @@ -102,7 +102,7 @@ import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import org.apache.druid.msq.util.DecoratedExecutorService; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.PrioritizedCallable; @@ -332,14 +332,14 @@ public Optional runTask(final Closer closer) throws Exception if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { if (controllerAlive) { - ClusterByStatisticsWorkerReport workerReport = + WorkerAggregatedKeyStatistics aggregatedKeyStatistics = kernel.getResultKeyStatisticsSnapshot() - .workerReport(task().getWorkerNumber()); + .aggregatedKeyStatistics(task().getWorkerNumber()); controllerClient.postWorkerReport( stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber(), - workerReport + aggregatedKeyStatistics ); } kernel.startPreshuffleWaitingForResultPartitionBoundaries(); 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 d2919aa7d55c..8954387bb3be 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 @@ -26,7 +26,7 @@ import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import java.util.ArrayList; import java.util.HashSet; @@ -63,7 +63,7 @@ public WorkerSketchFetcher(WorkerClient workerClient, boolean forceNonSequential * decides based on the report if it should fetch sketches one by one or together. */ public CompletableFuture> submitFetcherTask( - ClusterByStatisticsWorkerReport workerReport, + WorkerAggregatedKeyStatistics aggregatedKeyStatistics, List workerTaskIds, StageDefinition stageDefinition ) @@ -72,10 +72,10 @@ public CompletableFuture> submitFetcherTask( if (forceNonSequentialMerging || clusterBy.getBucketByCount() == 0) { return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); - } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || workerReport.getBytesRetained() > BYTES_THRESHOLD) { - return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || aggregatedKeyStatistics.getBytesRetained() > BYTES_THRESHOLD) { + return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); } else { - return sequentialTimeChunkMerging(workerReport, stageDefinition, workerTaskIds); + return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); } } @@ -133,7 +133,7 @@ private CompletableFuture> inMemoryFullSketchM * downsampling on the controller. */ private CompletableFuture> sequentialTimeChunkMerging( - ClusterByStatisticsWorkerReport workerReport, + WorkerAggregatedKeyStatistics workerReport, StageDefinition stageDefinition, List workerTaskIds ) 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 a2ef9588f1f1..0a96b5bb80d5 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 @@ -27,7 +27,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; @@ -59,9 +59,9 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller) } /** - * Used by subtasks to post {@link ClusterByStatisticsWorkerReport} for shuffling stages. + * Used by subtasks to post {@link WorkerAggregatedKeyStatistics} for shuffling stages. * - * See {@link ControllerClient#postWorkerReport(StageId, int, ClusterByStatisticsWorkerReport)} + * See {@link ControllerClient#postWorkerReport(StageId, int, WorkerAggregatedKeyStatistics)} * for the client-side code that calls this API. */ @POST 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 db533842c435..f702954a6a74 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 @@ -29,7 +29,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -62,7 +62,7 @@ public IndexerControllerClient( public void postWorkerReport( StageId stageId, int workerNumber, - ClusterByStatisticsWorkerReport workerReport + WorkerAggregatedKeyStatistics aggregatedKeyStatistics ) throws IOException { final String path = StringUtils.format( @@ -74,7 +74,7 @@ public void postWorkerReport( doRequest( new RequestBuilder(HttpMethod.POST, path) - .jsonContent(jsonMapper, workerReport), + .jsonContent(jsonMapper, aggregatedKeyStatistics), IgnoreHttpResponseHandler.INSTANCE ); } 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 2d6a23ce7533..3142791d3be9 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 @@ -41,7 +41,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.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import javax.annotation.Nullable; import java.util.HashMap; @@ -334,9 +334,9 @@ public ClusterByPartitions getResultPartitionBoundariesForStage(final StageId st /** * Delegates call to {@link ControllerStageTracker#getResultPartitionBoundaries()} */ - public ClusterByStatisticsWorkerReport getClusterByStatisticsWorkerReport(final StageId stageId) + public WorkerAggregatedKeyStatistics getClusterByStatisticsWorkerReport(final StageId stageId) { - return getStageKernelOrThrow(stageId).getClusterByStatisticsWorkerReport(); + return getStageKernelOrThrow(stageId).getAggregatedKeyStatistics(); } /** @@ -409,7 +409,7 @@ public WorkerInputs getWorkerInputsForStage(final StageId stageId) public void addResultStatisticsReportForStageAndWorker( final StageId stageId, final int workerNumber, - final ClusterByStatisticsWorkerReport workerReport + final WorkerAggregatedKeyStatistics workerReport ) { ControllerStageTracker stageKernel = getStageKernelOrThrow(stageId); 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 9fce2a0e925b..b51d1b9fb1f1 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 @@ -38,7 +38,7 @@ 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.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import javax.annotation.Nullable; import java.util.List; @@ -63,7 +63,7 @@ class ControllerStageTracker private ControllerStagePhase phase = ControllerStagePhase.NEW; @Nullable - public final ClusterByStatisticsWorkerReport clusterByStatisticsWorkerReport; + public final WorkerAggregatedKeyStatistics aggregatedKeyStatistics; // Result partitions and where they can be read from. @Nullable @@ -89,9 +89,9 @@ private ControllerStageTracker( this.workerInputs = workerInputs; if (stageDef.mustGatherResultKeyStatistics()) { - this.clusterByStatisticsWorkerReport = new ClusterByStatisticsWorkerReport(new TreeMap<>(), false, 0); + this.aggregatedKeyStatistics = new WorkerAggregatedKeyStatistics(new TreeMap<>(), false, 0); } else { - this.clusterByStatisticsWorkerReport = null; + this.aggregatedKeyStatistics = null; generateResultPartitionsAndBoundariesWithoutKeyStatistics(); } } @@ -172,12 +172,12 @@ ClusterByPartitions getResultPartitionBoundaries() */ boolean collectorEncounteredAnyMultiValueField() { - if (clusterByStatisticsWorkerReport == null) { + if (aggregatedKeyStatistics == null) { throw new ISE("Stage does not gather result key statistics"); } else if (workersWithFinishedReport.size() != workerCount) { throw new ISE("Result key statistics are not ready"); } else { - return clusterByStatisticsWorkerReport.isHasMultipleValues(); + return aggregatedKeyStatistics.isHasMultipleValues(); } } @@ -225,9 +225,9 @@ WorkerInputs getWorkerInputs() * Returns the merged worker report. */ @Nullable - public ClusterByStatisticsWorkerReport getClusterByStatisticsWorkerReport() + public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics() { - return clusterByStatisticsWorkerReport; + return aggregatedKeyStatistics; } /** @@ -239,10 +239,10 @@ public ClusterByStatisticsWorkerReport getClusterByStatisticsWorkerReport() */ ControllerStagePhase addStatisticsReportForWorker( final int workerNumber, - final ClusterByStatisticsWorkerReport workerReport + final WorkerAggregatedKeyStatistics workerReport ) { - if (clusterByStatisticsWorkerReport == null) { + if (aggregatedKeyStatistics == null) { throw new ISE("Stage does not gather result key statistics"); } @@ -256,7 +256,7 @@ ControllerStagePhase addStatisticsReportForWorker( try { if (workersWithFinishedReport.add(workerNumber)) { - clusterByStatisticsWorkerReport.addAll(workerReport); + aggregatedKeyStatistics.addAll(workerReport); if (workersWithFinishedReport.size() == workerCount) { // All workers have sent the report. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java index 65fd04ee29e9..b5a76607123f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java @@ -324,6 +324,8 @@ public ClusterByStatisticsSnapshot snapshot() final KeyCollectorSnapshot keyCollectorSnapshot = ((KeyCollectorFactory) keyCollectorFactory).toSnapshot(bucketEntry.getValue().keyCollector); Long bucketKey = Long.MIN_VALUE; + + // If there is a clustering on time, read the first field from each bucket and add it to the snapshots. if (clusterBy.getBucketByCount() == 1) { bucketKey = (Long) trimmedRowReader.read(bucketEntry.getKey(), 0); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index 74221585cdc7..bb7c4002a93c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -74,7 +74,7 @@ Set getHasMultipleValues() return hasMultipleValues; } - public ClusterByStatisticsWorkerReport workerReport(int workerNumber) + public WorkerAggregatedKeyStatistics aggregatedKeyStatistics(int workerNumber) { SortedMap> rowKeyIntSetTreeSet = new TreeMap<>(); double bytesRetained = 0; @@ -84,7 +84,7 @@ public ClusterByStatisticsWorkerReport workerReport(int workerNumber) bytesRetained += buckets.get(bucketKey).bytesRetained; } - return new ClusterByStatisticsWorkerReport(rowKeyIntSetTreeSet, !getHasMultipleValues().isEmpty(), bytesRetained); + return new WorkerAggregatedKeyStatistics(rowKeyIntSetTreeSet, !getHasMultipleValues().isEmpty(), bytesRetained); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/WorkerAggregatedKeyStatistics.java similarity index 92% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/WorkerAggregatedKeyStatistics.java index c8cb89dc171d..a145a772c976 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/WorkerAggregatedKeyStatistics.java @@ -29,16 +29,16 @@ /** * Class sent by worker to controller after reading input to generate partition boundries. */ -public class ClusterByStatisticsWorkerReport +public class WorkerAggregatedKeyStatistics { private final SortedMap> timeSegmentVsWorkerIdMap; - private Boolean hasMultipleValues; + private boolean hasMultipleValues; private double bytesRetained; @JsonCreator - public ClusterByStatisticsWorkerReport( + public WorkerAggregatedKeyStatistics( @JsonProperty("timeSegmentVsWorkerIdMap") final SortedMap> timeChunks, @JsonProperty("hasMultipleValues") boolean hasMultipleValues, @JsonProperty("bytesRetained") double bytesRetained @@ -49,7 +49,7 @@ public ClusterByStatisticsWorkerReport( this.bytesRetained = bytesRetained; } - public void addAll(ClusterByStatisticsWorkerReport other) + public void addAll(WorkerAggregatedKeyStatistics other) { for (Long timeChunk : other.timeSegmentVsWorkerIdMap.keySet()) { this.timeSegmentVsWorkerIdMap 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 bd5157a4d346..9ebe4e622746 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 @@ -126,7 +126,7 @@ public ControllerQueryKernelTester setupStage( controllerQueryKernel.addResultStatisticsReportForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), i, - ClusterByStatisticsSnapshot.empty().workerReport(i) + ClusterByStatisticsSnapshot.empty().aggregatedKeyStatistics(i) ); } } else { @@ -265,7 +265,7 @@ public ClusterByStatisticsCollector addResultKeyStatisticsForStageAndWorker(int controllerQueryKernel.addResultStatisticsReportForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), workerNumber, - keyStatsCollector.snapshot().workerReport(workerNumber) + keyStatsCollector.snapshot().aggregatedKeyStatistics(workerNumber) ); return keyStatsCollector; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java index d658f54e5528..d9604ad1d633 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java @@ -47,19 +47,19 @@ public void setUp() @Test public void testSerde() throws JsonProcessingException { - ClusterByStatisticsWorkerReport workerReport = new ClusterByStatisticsWorkerReport( + WorkerAggregatedKeyStatistics workerReport = new WorkerAggregatedKeyStatistics( new TreeMap<>(ImmutableMap.of(2L, ImmutableSet.of(2, 3))), false, 0.0 ); final String json = objectMapper.writeValueAsString(workerReport); - final ClusterByStatisticsWorkerReport deserializedWorkerReport = objectMapper.readValue( + final WorkerAggregatedKeyStatistics deserializedKeyStatistics = objectMapper.readValue( json, - ClusterByStatisticsWorkerReport.class + WorkerAggregatedKeyStatistics.class ); - Assert.assertEquals(json, workerReport.getTimeSegmentVsWorkerIdMap(), deserializedWorkerReport.getTimeSegmentVsWorkerIdMap()); - Assert.assertEquals(json, workerReport.isHasMultipleValues(), deserializedWorkerReport.isHasMultipleValues()); - Assert.assertEquals(json, workerReport.getBytesRetained(), deserializedWorkerReport.getBytesRetained(), 0); + Assert.assertEquals(json, workerReport.getTimeSegmentVsWorkerIdMap(), deserializedKeyStatistics.getTimeSegmentVsWorkerIdMap()); + Assert.assertEquals(json, workerReport.isHasMultipleValues(), deserializedKeyStatistics.isHasMultipleValues()); + Assert.assertEquals(json, workerReport.getBytesRetained(), deserializedKeyStatistics.getBytesRetained(), 0); } } 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 0db3c2cff014..61e12ea96d49 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 @@ -25,7 +25,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsWorkerReport; +import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; import javax.annotation.Nullable; import java.util.List; @@ -43,11 +43,11 @@ public MSQTestControllerClient(Controller controller) public void postWorkerReport( StageId stageId, int workerNumber, - ClusterByStatisticsWorkerReport workerReport + WorkerAggregatedKeyStatistics aggregatedKeyStatistics ) { try { - controller.updateWorkerReportStatus(stageId.getStageNumber(), workerNumber, workerReport); + controller.updateWorkerReportStatus(stageId.getStageNumber(), workerNumber, aggregatedKeyStatistics); } catch (Exception e) { throw new ISE(e, "unable to post worker report"); From 5ec8e391072ad8f16b8a01abc3305d90da926376 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 26 Oct 2022 12:13:35 +0530 Subject: [PATCH 10/31] Address review comments --- .../java/org/apache/druid/msq/exec/WorkerSketchFetcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8954387bb3be..57e16719c356 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 @@ -92,7 +92,7 @@ private CompletableFuture> inMemoryFullSketchM CompletableFuture> partitionFuture = new CompletableFuture<>(); final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); - final int workerCount = stageDefinition.getMaxWorkerCount(); + final int workerCount = workerTaskIds.size(); final Set finishedWorkers = new HashSet<>(); for (int i = 0; i < workerCount; i++) { From 0264bb0c4cb0a0580c080e450dd751f386774d2b Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 26 Oct 2022 12:57:46 +0530 Subject: [PATCH 11/31] Address review comments --- docs/multi-stage-query/reference.md | 1 + .../msq/exec/ClusterStatisticsMergeMode.java | 41 ++++++++++ .../org/apache/druid/msq/exec/Controller.java | 2 +- .../druid/msq/exec/ControllerClient.java | 8 +- .../apache/druid/msq/exec/ControllerImpl.java | 20 ++--- .../org/apache/druid/msq/exec/WorkerImpl.java | 2 +- .../druid/msq/exec/WorkerSketchFetcher.java | 36 ++++++--- .../msq/indexing/ControllerChatHandler.java | 10 +-- .../msq/indexing/IndexerControllerClient.java | 4 +- .../controller/ControllerQueryKernel.java | 12 +-- .../controller/ControllerStageTracker.java | 10 +-- .../msq/util/MultiStageQueryContext.java | 12 +-- .../msq/test/MSQTestControllerClient.java | 6 +- .../druid/testsEx/msq/ITMultiStageQuery.java | 81 ++++++++++++++++++- 14 files changed, 189 insertions(+), 56 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index eb70f143d4c5..1cf4c15073a1 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -204,6 +204,7 @@ 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 | | sqlTimeZone | Sets the time zone for this connection, which affects how time functions and timestamp literals behave. Use a time zone name like "America/Los_Angeles" or offset like "-08:00".| `druid.sql.planner.sqlTimeZone` on the Broker (default: UTC)| | useApproximateCountDistinct | Whether to use an approximate cardinality algorithm for `COUNT(DISTINCT foo)`.| `druid.sql.planner.useApproximateCountDistinct` on the Broker (default: true)| +| clusterStatisticsMergeMode | Whether to parallel or sequential merging of worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. On `AUTO` tries to find the best approach based on number of workers and size of input rows. | `AUTO` | ## Limits diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java new file mode 100644 index 000000000000..027affc852ee --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +/** + * Mode which dictates how {@link WorkerSketchFetcher} gets sketches from workers. + */ +public enum ClusterStatisticsMergeMode +{ + /** + * Fetches sketch in sequential order based on time. Slower due to overhead, but more accurate. + */ + SEQUENTIAL, + + /** + * Fetch all sketches from the worker at once. Faster to generate partitions, but less accurate. + */ + PARALLEL, + + /** + * Tries to decide between sequential and parallel modes based on the number of workers and size of the input + */ + AUTO +} 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 583afed2e13c..fb306057899f 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 @@ -85,7 +85,7 @@ public String getId() * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate key statistics. * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ - void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerStatisticsReport); + void updateAggregatedKeyStatistics(int stageNumber, int workerNumber, Object aggregatedKeyStatisticsObject); /** * System error reported by a subtask. Note that the errors are organized by 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 89ea3979d9c4..778a1781deca 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 @@ -34,13 +34,13 @@ public interface ControllerClient extends AutoCloseable { /** - * Client side method to update the controller with worker reports for a particular stage and worker. - * Controller's implementation collates all the reports for a stage to fetch cluster by statistics from workers. + * Client side method to update the controller with aggregated key statistics for a particular stage and worker. + * Controller's implementation collates all the aggregations for a stage to fetch cluster by statistics from workers. */ - void postWorkerReport( + void postAggregatedKeyStatistics( StageId stageId, int workerNumber, - WorkerAggregatedKeyStatistics workerReport + WorkerAggregatedKeyStatistics aggregatedKeyStatistics ) throws IOException; /** 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 d070d7d7b9a5..10c394d77d94 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 @@ -520,8 +520,8 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); - boolean forceNonSequentialMerging = MultiStageQueryContext.isForceSequentialMerge(task.getSqlQueryContext()); - this.workerSketchFetcher = new WorkerSketchFetcher(netClient, forceNonSequentialMerging); + ClusterStatisticsMergeMode clusterStatisticsMergeMode = MultiStageQueryContext.getClusterStatisticsMergeMode(task.getSqlQueryContext()); + this.workerSketchFetcher = new WorkerSketchFetcher(netClient, clusterStatisticsMergeMode); closer.register(netClient::close); @@ -567,10 +567,12 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) } /** - * Provide a {@link WorkerAggregatedKeyStatistics} for shuffling stages. + * Accepts a {@link WorkerAggregatedKeyStatistics} and updates the controller aggregated key statistics. If all key + * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate key statistics. + * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ @Override - public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerStatisticsReport) + public void updateAggregatedKeyStatistics(int stageNumber, int workerNumber, Object aggregatedKeyStatisticsObject) { addToKernelManipulationQueue( queryKernel -> { @@ -584,9 +586,9 @@ public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object w stageDef.getShuffleSpec().get().doesAggregateByClusterKey() ); - final WorkerAggregatedKeyStatistics workerReport; + final WorkerAggregatedKeyStatistics aggregatedKeyStatistics; try { - workerReport = mapper.convertValue(workerStatisticsReport, WorkerAggregatedKeyStatistics.class); + aggregatedKeyStatistics = mapper.convertValue(aggregatedKeyStatisticsObject, WorkerAggregatedKeyStatistics.class); } catch (IllegalArgumentException e) { throw new IAE( @@ -597,16 +599,16 @@ public void updateWorkerReportStatus(int stageNumber, int workerNumber, Object w ); } - queryKernel.addResultStatisticsReportForStageAndWorker(stageId, workerNumber, workerReport); + queryKernel.addResultStatisticsReportForStageAndWorker(stageId, workerNumber, aggregatedKeyStatistics); if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) { List workerTaskIds = workerTaskLauncher.getTaskList(); - WorkerAggregatedKeyStatistics aggregatedKeyStatistics = queryKernel.getClusterByStatisticsWorkerReport(stageId); + WorkerAggregatedKeyStatistics mergedKeyStatistics = queryKernel.getAggregatedKeyStatistics(stageId); // Queue the sketch fetching task into the worker sketch fetcher. CompletableFuture> clusterByPartitionsCompletableFuture = workerSketchFetcher.submitFetcherTask( - aggregatedKeyStatistics, + mergedKeyStatistics, workerTaskIds, stageDef ); 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 1a060c3c9def..7261ef144e48 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 @@ -336,7 +336,7 @@ public Optional runTask(final Closer closer) throws Exception kernel.getResultKeyStatisticsSnapshot() .aggregatedKeyStatistics(task().getWorkerNumber()); - controllerClient.postWorkerReport( + controllerClient.postAggregatedKeyStatistics( stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber(), aggregatedKeyStatistics 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 57e16719c356..d292d289dc37 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 @@ -47,20 +47,20 @@ public class WorkerSketchFetcher private static final long BYTES_THRESHOLD = 1_000_000_000L; private static final long WORKER_THRESHOLD = 100; - private final boolean forceNonSequentialMerging; + private final ClusterStatisticsMergeMode clusterStatisticsMergeMode; private final WorkerClient workerClient; private final ExecutorService executorService; - public WorkerSketchFetcher(WorkerClient workerClient, boolean forceNonSequentialMerging) + public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode clusterStatisticsMergeMode) { this.workerClient = workerClient; - this.forceNonSequentialMerging = forceNonSequentialMerging; + this.clusterStatisticsMergeMode = clusterStatisticsMergeMode; this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); } /** - * Submits a request to fetch and generate partitions for the given worker report and returns a future for it. It - * decides based on the report if it should fetch sketches one by one or together. + * 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. */ public CompletableFuture> submitFetcherTask( WorkerAggregatedKeyStatistics aggregatedKeyStatistics, @@ -70,12 +70,22 @@ public CompletableFuture> submitFetcherTask( { ClusterBy clusterBy = stageDefinition.getClusterBy(); - if (forceNonSequentialMerging || clusterBy.getBucketByCount() == 0) { - return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); - } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || aggregatedKeyStatistics.getBytesRetained() > BYTES_THRESHOLD) { - return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); - } else { - return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); + switch (clusterStatisticsMergeMode) { + case SEQUENTIAL: + return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); + case PARALLEL: + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + case AUTO: + if (clusterBy.getBucketByCount() == 0) { + // If there is no time cluserting, there is no scope for sequential merge + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || aggregatedKeyStatistics.getBytesRetained() > BYTES_THRESHOLD) { + return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); + } else { + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + } + default: + throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode); } } @@ -133,7 +143,7 @@ private CompletableFuture> inMemoryFullSketchM * downsampling on the controller. */ private CompletableFuture> sequentialTimeChunkMerging( - WorkerAggregatedKeyStatistics workerReport, + WorkerAggregatedKeyStatistics aggregatedKeyStatistics, StageDefinition stageDefinition, List workerTaskIds ) @@ -141,7 +151,7 @@ private CompletableFuture> sequentialTimeChunk SequentialFetchStage sequentialFetchStage = new SequentialFetchStage( stageDefinition, workerTaskIds, - workerReport.getTimeSegmentVsWorkerIdMap().entrySet().iterator() + aggregatedKeyStatistics.getTimeSegmentVsWorkerIdMap().entrySet().iterator() ); sequentialFetchStage.submitFetchingTasksForNextTimeChunk(); return sequentialFetchStage.getPartitionFuture(); 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 0a96b5bb80d5..48fccee8fe7c 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 @@ -61,15 +61,15 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller) /** * Used by subtasks to post {@link WorkerAggregatedKeyStatistics} for shuffling stages. * - * See {@link ControllerClient#postWorkerReport(StageId, int, WorkerAggregatedKeyStatistics)} + * See {@link ControllerClient#postAggregatedKeyStatistics(StageId, int, WorkerAggregatedKeyStatistics)} * for the client-side code that calls this API. */ @POST - @Path("/statisticsWorkerReport/{queryId}/{stageNumber}/{workerNumber}") + @Path("/aggregatedKeyStatistics/{queryId}/{stageNumber}/{workerNumber}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) - public Response httpPostStatisticsWorkerReport( - final Object workerReport, + public Response httpPostAggregatedKeyStatistics( + final Object aggregatedKeyStatisticsObject, @PathParam("queryId") final String queryId, @PathParam("stageNumber") final int stageNumber, @PathParam("workerNumber") final int workerNumber, @@ -77,7 +77,7 @@ public Response httpPostStatisticsWorkerReport( ) { ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - controller.updateWorkerReportStatus(stageNumber, workerNumber, workerReport); + controller.updateAggregatedKeyStatistics(stageNumber, workerNumber, aggregatedKeyStatisticsObject); return Response.status(Response.Status.ACCEPTED).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 f702954a6a74..647ca826b12e 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 @@ -59,14 +59,14 @@ public IndexerControllerClient( } @Override - public void postWorkerReport( + public void postAggregatedKeyStatistics( StageId stageId, int workerNumber, WorkerAggregatedKeyStatistics aggregatedKeyStatistics ) throws IOException { final String path = StringUtils.format( - "/statisticsWorkerReport/%s/%d/%d", + "/aggregatedKeyStatistics/%s/%d/%d", StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), 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 3142791d3be9..785f212e3c26 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 @@ -332,9 +332,9 @@ public ClusterByPartitions getResultPartitionBoundariesForStage(final StageId st } /** - * Delegates call to {@link ControllerStageTracker#getResultPartitionBoundaries()} + * Delegates call to {@link ControllerStageTracker#getAggregatedKeyStatistics()} */ - public WorkerAggregatedKeyStatistics getClusterByStatisticsWorkerReport(final StageId stageId) + public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics(final StageId stageId) { return getStageKernelOrThrow(stageId).getAggregatedKeyStatistics(); } @@ -403,19 +403,19 @@ public WorkerInputs getWorkerInputsForStage(final StageId stageId) } /** - * Delegates call to {@link ControllerStageTracker#addStatisticsReportForWorker}. + * Delegates call to {@link ControllerStageTracker#addAggregatedStatisticsForWorker}. * If calling this causes transition for the stage kernel, then this gets registered in this query kernel */ public void addResultStatisticsReportForStageAndWorker( final StageId stageId, final int workerNumber, - final WorkerAggregatedKeyStatistics workerReport + final WorkerAggregatedKeyStatistics aggregatedKeyStatistics ) { ControllerStageTracker stageKernel = getStageKernelOrThrow(stageId); - ControllerStagePhase newPhase = stageKernel.addStatisticsReportForWorker( + ControllerStagePhase newPhase = stageKernel.addAggregatedStatisticsForWorker( workerNumber, - workerReport + aggregatedKeyStatistics ); // If the kernel phase has transitioned, we need to account for that. 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 b51d1b9fb1f1..365a00c8875a 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 @@ -222,7 +222,7 @@ WorkerInputs getWorkerInputs() } /** - * Returns the merged worker report. + * Returns the merged key statistics. */ @Nullable public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics() @@ -235,11 +235,11 @@ public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics() * then this call ignores the new ones and does nothing. * * @param workerNumber the worker - * @param workerReport worker report + * @param aggregatedKeyStatistics aggregated key statistics */ - ControllerStagePhase addStatisticsReportForWorker( + ControllerStagePhase addAggregatedStatisticsForWorker( final int workerNumber, - final WorkerAggregatedKeyStatistics workerReport + final WorkerAggregatedKeyStatistics aggregatedKeyStatistics ) { if (aggregatedKeyStatistics == null) { @@ -256,7 +256,7 @@ ControllerStagePhase addStatisticsReportForWorker( try { if (workersWithFinishedReport.add(workerNumber)) { - aggregatedKeyStatistics.addAll(workerReport); + aggregatedKeyStatistics.addAll(aggregatedKeyStatistics); if (workersWithFinishedReport.size() == workerCount) { // All workers have sent the report. 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 441a18178842..35595de7ca7f 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 @@ -26,6 +26,7 @@ import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.sql.MSQMode; import org.apache.druid.query.QueryContext; @@ -59,7 +60,8 @@ public class MultiStageQueryContext private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; - public static final String CTX_FORCE_NON_SEQUENTIAL_MERGE = "forceNonSequentialMerging"; + public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode"; + public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.AUTO.toString(); private static final boolean DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = false; public static final String CTX_DESTINATION = "destination"; @@ -94,15 +96,15 @@ public static boolean isDurableStorageEnabled(final QueryContext queryContext) ); } - public static boolean isForceSequentialMerge(Map propertyMap) + public static ClusterStatisticsMergeMode getClusterStatisticsMergeMode(Map propertyMap) { - return Boolean.parseBoolean( + return ClusterStatisticsMergeMode.valueOf( String.valueOf( getValueFromPropertyMap( propertyMap, - CTX_FORCE_NON_SEQUENTIAL_MERGE, + CTX_CLUSTER_STATISTICS_MERGE_MODE, null, - false + DEFAULT_CLUSTER_STATISTICS_MERGE_MODE ) ) ); 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 61e12ea96d49..5e5be2c1d01a 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 @@ -40,17 +40,17 @@ public MSQTestControllerClient(Controller controller) } @Override - public void postWorkerReport( + public void postAggregatedKeyStatistics( StageId stageId, int workerNumber, WorkerAggregatedKeyStatistics aggregatedKeyStatistics ) { try { - controller.updateWorkerReportStatus(stageId.getStageNumber(), workerNumber, aggregatedKeyStatistics); + controller.updateAggregatedKeyStatistics(stageId.getStageNumber(), workerNumber, aggregatedKeyStatistics); } catch (Exception e) { - throw new ISE(e, "unable to post worker report"); + throw new ISE(e, "unable to post aggregated key statistics"); } } 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 8d88fa15e893..78a23019039d 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 @@ -23,7 +23,9 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.sql.SqlTaskStatus; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.sql.http.SqlQuery; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; @@ -126,7 +128,7 @@ public void testMsqIngestionAndQuerying() throws Exception } @Test - public void testMsqIngestionWithoutSequentialMerging() throws Exception + public void testMsqIngestionParallelMerging() throws Exception { String datasource = "dst"; @@ -173,8 +175,83 @@ public void testMsqIngestionWithoutSequentialMerging() throws Exception datasource ); + ImmutableMap context = ImmutableMap.of( + MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, + ClusterStatisticsMergeMode.PARALLEL + ); + + // Submit the task and wait for the datasource to get loaded + SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } + + @Test + public void testMsqIngestionSequentialMerging() 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\"]}',\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 + ); + + ImmutableMap context = ImmutableMap.of( + MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, + ClusterStatisticsMergeMode.SEQUENTIAL + ); + // Submit the task and wait for the datasource to get loaded - SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, ImmutableMap.of("msqForceNonSequentialMerging", true), null); + SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); if (sqlTaskStatus.getState().isFailure()) { From 4656f47e86600040237bb56d24a73af1c5b18dd1 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 14 Nov 2022 21:57:39 +0530 Subject: [PATCH 12/31] Resolve maxRetainedBytes --- .../java/org/apache/druid/msq/exec/ControllerImpl.java | 10 ++++------ .../org/apache/druid/msq/exec/WorkerSketchFetcher.java | 8 +++++--- .../msq/kernel/controller/ControllerQueryKernel.java | 7 ++----- .../msq/kernel/controller/ControllerStageTracker.java | 8 +++----- .../controller/BaseControllerQueryKernelTest.java | 2 +- 5 files changed, 15 insertions(+), 20 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 eedd2e451129..0ed40847ca47 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 @@ -525,7 +525,9 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); ClusterStatisticsMergeMode clusterStatisticsMergeMode = MultiStageQueryContext.getClusterStatisticsMergeMode(task.getQuerySpec().getQuery().context()); - this.workerSketchFetcher = new WorkerSketchFetcher(netClient, clusterStatisticsMergeMode); + int statisticsMaxRetainedBytes = WorkerMemoryParameters.createProductionInstanceForController(context.injector()) + .getPartitionStatisticsMaxRetainedBytes(); + this.workerSketchFetcher = new WorkerSketchFetcher(netClient, clusterStatisticsMergeMode, statisticsMaxRetainedBytes); closer.register(netClient::close); @@ -1996,11 +1998,7 @@ public RunQueryUntilDone( this.queryDef = queryDef; this.inputSpecSlicerFactory = inputSpecSlicerFactory; this.closer = closer; - this.queryKernel = new ControllerQueryKernel( - queryDef, - WorkerMemoryParameters.createProductionInstanceForController(context.injector()) - .getPartitionStatisticsMaxRetainedBytes() - ); + this.queryKernel = new ControllerQueryKernel(queryDef); } /** 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 d292d289dc37..56c2440610fc 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 @@ -48,14 +48,16 @@ public class WorkerSketchFetcher private static final long WORKER_THRESHOLD = 100; private final ClusterStatisticsMergeMode clusterStatisticsMergeMode; + private final int statisticsMaxRetainedBytes; private final WorkerClient workerClient; private final ExecutorService executorService; - public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode clusterStatisticsMergeMode) + public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode clusterStatisticsMergeMode, int statisticsMaxRetainedBytes) { this.workerClient = workerClient; this.clusterStatisticsMergeMode = clusterStatisticsMergeMode; this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); + this.statisticsMaxRetainedBytes = statisticsMaxRetainedBytes; } /** @@ -101,7 +103,7 @@ private CompletableFuture> inMemoryFullSketchM { CompletableFuture> partitionFuture = new CompletableFuture<>(); - final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); + final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); final int workerCount = workerTaskIds.size(); final Set finishedWorkers = new HashSet<>(); @@ -186,7 +188,7 @@ public void submitFetchingTasksForNextTimeChunk() Map.Entry> entry = timeSegmentVsWorkerIdIterator.next(); Long timeChunk = entry.getKey(); Set workerIdsWithTimeChunk = entry.getValue(); - ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(); + ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); Set finishedWorkers = new HashSet<>(); for (int workerNo : workerIdsWithTimeChunk) { 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 1e32adda8277..785f212e3c26 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 @@ -65,7 +65,6 @@ public class ControllerQueryKernel { private final QueryDefinition queryDef; - private final int partitionStatisticsMaxRetainedBytes; /** * Stage ID -> tracker for that stage. An extension of the state of this kernel. @@ -107,10 +106,9 @@ public class ControllerQueryKernel */ private final Set effectivelyFinishedStages = new HashSet<>(); - public ControllerQueryKernel(final QueryDefinition queryDef, final int partitionStatisticsMaxRetainedBytes) + public ControllerQueryKernel(final QueryDefinition queryDef) { this.queryDef = queryDef; - this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes; this.inflowMap = ImmutableMap.copyOf(computeStageInflowMap(queryDef)); this.outflowMap = ImmutableMap.copyOf(computeStageOutflowMap(queryDef)); @@ -266,8 +264,7 @@ private void createNewKernels( stageDef, stageWorkerCountMap, slicer, - assignmentStrategy, - partitionStatisticsMaxRetainedBytes + assignmentStrategy ); stageTracker.put(nextStage, stageKernel); } 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 e5186a348bba..d80ed41fc3cf 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 @@ -81,8 +81,7 @@ class ControllerStageTracker private ControllerStageTracker( final StageDefinition stageDef, - final WorkerInputs workerInputs, - final int partitionStatisticsMaxRetainedBytes + final WorkerInputs workerInputs ) { this.stageDef = stageDef; @@ -106,12 +105,11 @@ static ControllerStageTracker create( final StageDefinition stageDef, final Int2IntMap stageWorkerCountMap, final InputSpecSlicer slicer, - final WorkerAssignmentStrategy assignmentStrategy, - final int partitionStatisticsMaxRetainedBytes + final WorkerAssignmentStrategy assignmentStrategy ) { final WorkerInputs workerInputs = WorkerInputs.create(stageDef, stageWorkerCountMap, slicer, assignmentStrategy); - return new ControllerStageTracker(stageDef, workerInputs, partitionStatisticsMaxRetainedBytes); + return new ControllerStageTracker(stageDef, workerInputs); } /** 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 d3c9764cc458..efe4d959578d 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 @@ -82,7 +82,7 @@ private ControllerQueryKernelTester(int numWorkers) public ControllerQueryKernelTester queryDefinition(QueryDefinition queryDefinition) { this.queryDefinition = Preconditions.checkNotNull(queryDefinition); - this.controllerQueryKernel = new ControllerQueryKernel(queryDefinition, 10_000_000); + this.controllerQueryKernel = new ControllerQueryKernel(queryDefinition); return this; } From 07876f56ad116a0c6174028da0aa0940b36fa9d9 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 15 Nov 2022 10:09:52 +0530 Subject: [PATCH 13/31] Add new classes --- .../CompleteKeyStatisticsInformation.java | 54 +++++++++++++++++++ .../PartialKeyStatisticsInformation.java | 48 +++++++++++++++++ 2 files changed, 102 insertions(+) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java 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 new file mode 100644 index 000000000000..5d3ba037abae --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java @@ -0,0 +1,54 @@ +package org.apache.druid.msq.statistics; + +import java.util.HashSet; +import java.util.Set; +import java.util.SortedMap; + +/** + * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker. + */ +public class CompleteKeyStatisticsInformation +{ + private final SortedMap> timeSegmentVsWorkerIdMap; + + private boolean hasMultipleValues; + + private double bytesRetained; + + public CompleteKeyStatisticsInformation( + final SortedMap> timeChunks, + boolean hasMultipleValues, + double bytesRetained + ) + { + this.timeSegmentVsWorkerIdMap = timeChunks; + this.hasMultipleValues = hasMultipleValues; + this.bytesRetained = bytesRetained; + } + + public void mergePartialInformation(int workerId, PartialKeyStatisticsInformation partialKeyStatisticsInformation) + { + for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) { + this.timeSegmentVsWorkerIdMap + .computeIfAbsent(timeSegment, key -> new HashSet<>()) + .add(workerId); + } + this.hasMultipleValues = this.hasMultipleValues || partialKeyStatisticsInformation.isHasMultipleValues(); + this.bytesRetained += bytesRetained; + } + + public SortedMap> getTimeSegmentVsWorkerIdMap() + { + return timeSegmentVsWorkerIdMap; + } + + public boolean isHasMultipleValues() + { + return hasMultipleValues; + } + + public double getBytesRetained() + { + return bytesRetained; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java new file mode 100644 index 000000000000..a249d682876e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java @@ -0,0 +1,48 @@ +package org.apache.druid.msq.statistics; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Set; + +/** + * Class sent by worker to controller after reading input to generate partition boundries. + */ +public class PartialKeyStatisticsInformation +{ + private final Set timeSegments; + + private final boolean hasMultipleValues; + + private final double bytesRetained; + + @JsonCreator + public PartialKeyStatisticsInformation( + @JsonProperty("timeSegments") Set timeSegments, + @JsonProperty("hasMultipleValues") boolean hasMultipleValues, + @JsonProperty("bytesRetained") double bytesRetained + ) + { + this.timeSegments = timeSegments; + this.hasMultipleValues = hasMultipleValues; + this.bytesRetained = bytesRetained; + } + + @JsonProperty("timeSegments") + public Set getTimeSegments() + { + return timeSegments; + } + + @JsonProperty("hasMultipleValues") + public boolean isHasMultipleValues() + { + return hasMultipleValues; + } + + @JsonProperty("bytesRetained") + public double getBytesRetained() + { + return bytesRetained; + } +} From bb86c1969a26da970208c168293dd8c5a3473120 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 15 Nov 2022 12:00:51 +0530 Subject: [PATCH 14/31] Renamed key statistics information class --- .../org/apache/druid/msq/exec/Controller.java | 8 +- .../druid/msq/exec/ControllerClient.java | 10 +-- .../apache/druid/msq/exec/ControllerImpl.java | 22 ++--- .../org/apache/druid/msq/exec/WorkerImpl.java | 10 +-- .../druid/msq/exec/WorkerSketchFetcher.java | 14 ++-- .../msq/indexing/ControllerChatHandler.java | 14 ++-- .../msq/indexing/IndexerControllerClient.java | 10 +-- .../controller/ControllerQueryKernel.java | 19 ++--- .../controller/ControllerStageTracker.java | 36 +++++---- .../ClusterByStatisticsSnapshot.java | 11 +-- .../CompleteKeyStatisticsInformation.java | 14 ++-- .../WorkerAggregatedKeyStatistics.java | 80 ------------------- .../BaseControllerQueryKernelTest.java | 8 +- ...ialKeyStatisticsInformationSerdeTest.java} | 21 +++-- .../msq/test/MSQTestControllerClient.java | 10 +-- 15 files changed, 101 insertions(+), 186 deletions(-) delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/WorkerAggregatedKeyStatistics.java rename extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/{ClusterByStatisticsWorkerReportSerdeTest.java => PartialKeyStatisticsInformationSerdeTest.java} (65%) 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 fb306057899f..4b3808dee505 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 @@ -27,7 +27,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.error.MSQErrorReport; -import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.List; @@ -81,11 +81,11 @@ public String getId() // Worker-to-controller messages /** - * Accepts a {@link WorkerAggregatedKeyStatistics} and updates the controller aggregated key statistics. If all key - * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate key statistics. + * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key + * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate partiton boundaries. * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ - void updateAggregatedKeyStatistics(int stageNumber, int workerNumber, Object aggregatedKeyStatisticsObject); + void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject); /** * System error reported by a subtask. Note that the errors are organized by 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 26b6e41a9b2b..9d197b76dd17 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 @@ -22,7 +22,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.io.IOException; @@ -34,13 +34,13 @@ public interface ControllerClient extends AutoCloseable { /** - * Client side method to update the controller with aggregated key statistics for a particular stage and worker. - * Controller's implementation collates all the aggregations for a stage to fetch cluster by statistics from workers. + * Client side method to update the controller with partial key statistics information for a particular stage and worker. + * Controller's implementation collates all the information for a stage to fetch key statistics from workers. */ - void postAggregatedKeyStatistics( + void postPartialKeyStatistics( StageId stageId, int workerNumber, - WorkerAggregatedKeyStatistics aggregatedKeyStatistics + PartialKeyStatisticsInformation partialKeyStatisticsInformation ) throws IOException; /** 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 0ed40847ca47..f365df729f76 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 @@ -151,7 +151,8 @@ 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.WorkerAggregatedKeyStatistics; +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; import org.apache.druid.msq.util.MSQFutureUtils; @@ -575,12 +576,12 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) } /** - * Accepts a {@link WorkerAggregatedKeyStatistics} and updates the controller aggregated key statistics. If all key - * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate key statistics. - * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. + * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key + * statistics information has been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate + * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ @Override - public void updateAggregatedKeyStatistics(int stageNumber, int workerNumber, Object aggregatedKeyStatisticsObject) + public void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject) { addToKernelManipulationQueue( queryKernel -> { @@ -594,9 +595,9 @@ public void updateAggregatedKeyStatistics(int stageNumber, int workerNumber, Obj stageDef.getShuffleSpec().get().doesAggregateByClusterKey() ); - final WorkerAggregatedKeyStatistics aggregatedKeyStatistics; + final PartialKeyStatisticsInformation partialKeyStatisticsInformation; try { - aggregatedKeyStatistics = mapper.convertValue(aggregatedKeyStatisticsObject, WorkerAggregatedKeyStatistics.class); + partialKeyStatisticsInformation = mapper.convertValue(partialKeyStatisticsObject, PartialKeyStatisticsInformation.class); } catch (IllegalArgumentException e) { throw new IAE( @@ -607,16 +608,17 @@ public void updateAggregatedKeyStatistics(int stageNumber, int workerNumber, Obj ); } - queryKernel.addResultStatisticsReportForStageAndWorker(stageId, workerNumber, aggregatedKeyStatistics); + queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation); if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) { List workerTaskIds = workerTaskLauncher.getTaskList(); - WorkerAggregatedKeyStatistics mergedKeyStatistics = queryKernel.getAggregatedKeyStatistics(stageId); + CompleteKeyStatisticsInformation completeKeyStatisticsInformation = + queryKernel.getCompleteKeyStatisticsInformation(stageId); // Queue the sketch fetching task into the worker sketch fetcher. CompletableFuture> clusterByPartitionsCompletableFuture = workerSketchFetcher.submitFetcherTask( - mergedKeyStatistics, + completeKeyStatisticsInformation, workerTaskIds, stageDef ); 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 a359c8af7e7d..397cbd69e814 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 @@ -106,7 +106,7 @@ import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.msq.util.DecoratedExecutorService; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.PrioritizedCallable; @@ -368,14 +368,14 @@ public Optional runTask(final Closer closer) throws Exception if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { if (controllerAlive) { - WorkerAggregatedKeyStatistics aggregatedKeyStatistics = + PartialKeyStatisticsInformation partialKeyStatisticsInformation = kernel.getResultKeyStatisticsSnapshot() - .aggregatedKeyStatistics(task().getWorkerNumber()); + .partialKeyStatistics(); - controllerClient.postAggregatedKeyStatistics( + controllerClient.postPartialKeyStatistics( stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber(), - aggregatedKeyStatistics + partialKeyStatisticsInformation ); } kernel.startPreshuffleWaitingForResultPartitionBoundaries(); 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 56c2440610fc..be8f70baa7e6 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 @@ -26,7 +26,7 @@ import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; import java.util.ArrayList; import java.util.HashSet; @@ -65,7 +65,7 @@ public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode * decides based on the statistics if it should fetch sketches one by one or together. */ public CompletableFuture> submitFetcherTask( - WorkerAggregatedKeyStatistics aggregatedKeyStatistics, + CompleteKeyStatisticsInformation completeKeyStatisticsInformation, List workerTaskIds, StageDefinition stageDefinition ) @@ -74,15 +74,15 @@ public CompletableFuture> submitFetcherTask( switch (clusterStatisticsMergeMode) { case SEQUENTIAL: - return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); + return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); case PARALLEL: return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); case AUTO: if (clusterBy.getBucketByCount() == 0) { // If there is no time cluserting, there is no scope for sequential merge return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); - } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || aggregatedKeyStatistics.getBytesRetained() > BYTES_THRESHOLD) { - return sequentialTimeChunkMerging(aggregatedKeyStatistics, stageDefinition, workerTaskIds); + } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || completeKeyStatisticsInformation.getBytesRetained() > BYTES_THRESHOLD) { + return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); } else { return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); } @@ -145,7 +145,7 @@ private CompletableFuture> inMemoryFullSketchM * downsampling on the controller. */ private CompletableFuture> sequentialTimeChunkMerging( - WorkerAggregatedKeyStatistics aggregatedKeyStatistics, + CompleteKeyStatisticsInformation completeKeyStatisticsInformation, StageDefinition stageDefinition, List workerTaskIds ) @@ -153,7 +153,7 @@ private CompletableFuture> sequentialTimeChunk SequentialFetchStage sequentialFetchStage = new SequentialFetchStage( stageDefinition, workerTaskIds, - aggregatedKeyStatistics.getTimeSegmentVsWorkerIdMap().entrySet().iterator() + completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().entrySet().iterator() ); sequentialFetchStage.submitFetchingTasksForNextTimeChunk(); return sequentialFetchStage.getPartitionFuture(); 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 2ff76360fdb4..8557a8ca3f24 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 @@ -27,7 +27,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; @@ -59,17 +59,17 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller) } /** - * Used by subtasks to post {@link WorkerAggregatedKeyStatistics} for shuffling stages. + * Used by subtasks to post {@link PartialKeyStatisticsInformation} for shuffling stages. * - * See {@link ControllerClient#postAggregatedKeyStatistics(StageId, int, WorkerAggregatedKeyStatistics)} + * See {@link ControllerClient#postPartialKeyStatistics(StageId, int, PartialKeyStatisticsInformation)} * for the client-side code that calls this API. */ @POST - @Path("/aggregatedKeyStatistics/{queryId}/{stageNumber}/{workerNumber}") + @Path("/partialKeyStatistics/{queryId}/{stageNumber}/{workerNumber}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) - public Response httpPostAggregatedKeyStatistics( - final Object aggregatedKeyStatisticsObject, + public Response httpPostPartialKeyStatistics( + final Object partialKeyStatisticsObject, @PathParam("queryId") final String queryId, @PathParam("stageNumber") final int stageNumber, @PathParam("workerNumber") final int workerNumber, @@ -77,7 +77,7 @@ public Response httpPostAggregatedKeyStatistics( ) { ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - controller.updateAggregatedKeyStatistics(stageNumber, workerNumber, aggregatedKeyStatisticsObject); + controller.updatePartialKeyStatistics(stageNumber, workerNumber, partialKeyStatisticsObject); return Response.status(Response.Status.ACCEPTED).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 30ea32cf8b9e..2b130c7ab599 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 @@ -29,7 +29,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -59,14 +59,14 @@ public IndexerControllerClient( } @Override - public void postAggregatedKeyStatistics( + public void postPartialKeyStatistics( StageId stageId, int workerNumber, - WorkerAggregatedKeyStatistics aggregatedKeyStatistics + PartialKeyStatisticsInformation partialKeyStatisticsInformation ) throws IOException { final String path = StringUtils.format( - "/aggregatedKeyStatistics/%s/%d/%d", + "/partialKeyStatistics/%s/%d/%d", StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), workerNumber @@ -74,7 +74,7 @@ public void postAggregatedKeyStatistics( doRequest( new RequestBuilder(HttpMethod.POST, path) - .jsonContent(jsonMapper, aggregatedKeyStatistics), + .jsonContent(jsonMapper, partialKeyStatisticsInformation), IgnoreHttpResponseHandler.INSTANCE ); } 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 785f212e3c26..d2e9f198f04b 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 @@ -41,7 +41,8 @@ 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.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.HashMap; @@ -332,11 +333,11 @@ public ClusterByPartitions getResultPartitionBoundariesForStage(final StageId st } /** - * Delegates call to {@link ControllerStageTracker#getAggregatedKeyStatistics()} + * Delegates call to {@link ControllerStageTracker#getCompleteKeyStatisticsInformation()} */ - public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics(final StageId stageId) + public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation(final StageId stageId) { - return getStageKernelOrThrow(stageId).getAggregatedKeyStatistics(); + return getStageKernelOrThrow(stageId).getCompleteKeyStatisticsInformation(); } /** @@ -403,19 +404,19 @@ public WorkerInputs getWorkerInputsForStage(final StageId stageId) } /** - * Delegates call to {@link ControllerStageTracker#addAggregatedStatisticsForWorker}. + * Delegates call to {@link ControllerStageTracker#addPartialKeyStatisticsForWorker(int, PartialKeyStatisticsInformation)}. * If calling this causes transition for the stage kernel, then this gets registered in this query kernel */ - public void addResultStatisticsReportForStageAndWorker( + public void addPartialKeyStatisticsForStageAndWorker( final StageId stageId, final int workerNumber, - final WorkerAggregatedKeyStatistics aggregatedKeyStatistics + final PartialKeyStatisticsInformation partialKeyStatisticsInformation ) { ControllerStageTracker stageKernel = getStageKernelOrThrow(stageId); - ControllerStagePhase newPhase = stageKernel.addAggregatedStatisticsForWorker( + ControllerStagePhase newPhase = stageKernel.addPartialKeyStatisticsForWorker( workerNumber, - aggregatedKeyStatistics + partialKeyStatisticsInformation ); // If the kernel phase has transitioned, we need to account for that. 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 d80ed41fc3cf..ada67e5b67c6 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 @@ -38,7 +38,8 @@ 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.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.List; @@ -57,13 +58,13 @@ class ControllerStageTracker private final int workerCount; private final WorkerInputs workerInputs; - private final IntSet workersWithFinishedReport = new IntAVLTreeSet(); + private final IntSet workersWithReportedKeyStatistics = new IntAVLTreeSet(); private final IntSet workersWithResultsComplete = new IntAVLTreeSet(); private ControllerStagePhase phase = ControllerStagePhase.NEW; @Nullable - public final WorkerAggregatedKeyStatistics aggregatedKeyStatistics; + public final CompleteKeyStatisticsInformation completeKeyStatisticsInformation; // Result partitions and where they can be read from. @Nullable @@ -89,9 +90,10 @@ private ControllerStageTracker( this.workerInputs = workerInputs; if (stageDef.mustGatherResultKeyStatistics()) { - this.aggregatedKeyStatistics = new WorkerAggregatedKeyStatistics(new TreeMap<>(), false, 0); + this.completeKeyStatisticsInformation = + new CompleteKeyStatisticsInformation(new TreeMap<>(), false, 0); } else { - this.aggregatedKeyStatistics = null; + this.completeKeyStatisticsInformation = null; generateResultPartitionsAndBoundariesWithoutKeyStatistics(); } } @@ -172,12 +174,12 @@ ClusterByPartitions getResultPartitionBoundaries() */ boolean collectorEncounteredAnyMultiValueField() { - if (aggregatedKeyStatistics == null) { + if (completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); - } else if (workersWithFinishedReport.size() != workerCount) { + } else if (workersWithReportedKeyStatistics.size() != workerCount) { throw new ISE("Result key statistics are not ready"); } else { - return aggregatedKeyStatistics.isHasMultipleValues(); + return completeKeyStatisticsInformation.isHasMultipleValues(); } } @@ -225,9 +227,9 @@ WorkerInputs getWorkerInputs() * Returns the merged key statistics. */ @Nullable - public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics() + public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation() { - return aggregatedKeyStatistics; + return completeKeyStatisticsInformation; } /** @@ -235,17 +237,17 @@ public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics() * then this call ignores the new ones and does nothing. * * @param workerNumber the worker - * @param aggregatedKeyStatistics aggregated key statistics + * @param partialKeyStatisticsInformation partial key statistics */ - ControllerStagePhase addAggregatedStatisticsForWorker( + ControllerStagePhase addPartialKeyStatisticsForWorker( final int workerNumber, - final WorkerAggregatedKeyStatistics aggregatedKeyStatistics + final PartialKeyStatisticsInformation partialKeyStatisticsInformation ) { if (phase != ControllerStagePhase.READING_INPUT) { throw new ISE("Cannot add result key statistics from stage [%s]", phase); } - if (aggregatedKeyStatistics == null) { + if (completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); } @@ -254,10 +256,10 @@ ControllerStagePhase addAggregatedStatisticsForWorker( } try { - if (workersWithFinishedReport.add(workerNumber)) { - aggregatedKeyStatistics.addAll(aggregatedKeyStatistics); + if (workersWithReportedKeyStatistics.add(workerNumber)) { + completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation); - if (workersWithFinishedReport.size() == workerCount) { + if (workersWithReportedKeyStatistics.size() == workerCount) { // All workers have sent the report. // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers. transitionTo(ControllerStagePhase.MERGING_STATISTICS); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index bb7c4002a93c..2aa4b959dbff 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import org.apache.druid.frame.key.RowKey; import javax.annotation.Nullable; @@ -32,8 +31,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; public class ClusterByStatisticsSnapshot { @@ -74,17 +71,13 @@ Set getHasMultipleValues() return hasMultipleValues; } - public WorkerAggregatedKeyStatistics aggregatedKeyStatistics(int workerNumber) + public PartialKeyStatisticsInformation partialKeyStatistics() { - SortedMap> rowKeyIntSetTreeSet = new TreeMap<>(); double bytesRetained = 0; - for (Long bucketKey : buckets.keySet()) { - rowKeyIntSetTreeSet.put(bucketKey, ImmutableSet.of(workerNumber)); bytesRetained += buckets.get(bucketKey).bytesRetained; } - - return new WorkerAggregatedKeyStatistics(rowKeyIntSetTreeSet, !getHasMultipleValues().isEmpty(), bytesRetained); + return new PartialKeyStatisticsInformation(buckets.keySet(), !getHasMultipleValues().isEmpty(), bytesRetained); } @Override 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 5d3ba037abae..0206a73ac997 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 @@ -9,7 +9,7 @@ */ public class CompleteKeyStatisticsInformation { - private final SortedMap> timeSegmentVsWorkerIdMap; + private final SortedMap> timeSegmentVsWorkerMap; private boolean hasMultipleValues; @@ -21,25 +21,25 @@ public CompleteKeyStatisticsInformation( double bytesRetained ) { - this.timeSegmentVsWorkerIdMap = timeChunks; + this.timeSegmentVsWorkerMap = timeChunks; this.hasMultipleValues = hasMultipleValues; this.bytesRetained = bytesRetained; } - public void mergePartialInformation(int workerId, PartialKeyStatisticsInformation partialKeyStatisticsInformation) + public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation) { for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) { - this.timeSegmentVsWorkerIdMap + this.timeSegmentVsWorkerMap .computeIfAbsent(timeSegment, key -> new HashSet<>()) - .add(workerId); + .add(workerNumber); } this.hasMultipleValues = this.hasMultipleValues || partialKeyStatisticsInformation.isHasMultipleValues(); this.bytesRetained += bytesRetained; } - public SortedMap> getTimeSegmentVsWorkerIdMap() + public SortedMap> getTimeSegmentVsWorkerMap() { - return timeSegmentVsWorkerIdMap; + return timeSegmentVsWorkerMap; } public boolean isHasMultipleValues() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/WorkerAggregatedKeyStatistics.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/WorkerAggregatedKeyStatistics.java deleted file mode 100644 index a145a772c976..000000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/WorkerAggregatedKeyStatistics.java +++ /dev/null @@ -1,80 +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.statistics; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.HashSet; -import java.util.Set; -import java.util.SortedMap; - -/** - * Class sent by worker to controller after reading input to generate partition boundries. - */ -public class WorkerAggregatedKeyStatistics -{ - private final SortedMap> timeSegmentVsWorkerIdMap; - - private boolean hasMultipleValues; - - private double bytesRetained; - - @JsonCreator - public WorkerAggregatedKeyStatistics( - @JsonProperty("timeSegmentVsWorkerIdMap") final SortedMap> timeChunks, - @JsonProperty("hasMultipleValues") boolean hasMultipleValues, - @JsonProperty("bytesRetained") double bytesRetained - ) - { - this.timeSegmentVsWorkerIdMap = timeChunks; - this.hasMultipleValues = hasMultipleValues; - this.bytesRetained = bytesRetained; - } - - public void addAll(WorkerAggregatedKeyStatistics other) - { - for (Long timeChunk : other.timeSegmentVsWorkerIdMap.keySet()) { - this.timeSegmentVsWorkerIdMap - .computeIfAbsent(timeChunk, key -> new HashSet<>()) - .addAll(other.timeSegmentVsWorkerIdMap.get(timeChunk)); - } - this.hasMultipleValues = this.hasMultipleValues || other.hasMultipleValues; - this.bytesRetained += bytesRetained; - } - - @JsonProperty("timeSegmentVsWorkerIdMap") - public SortedMap> getTimeSegmentVsWorkerIdMap() - { - return timeSegmentVsWorkerIdMap; - } - - @JsonProperty("hasMultipleValues") - public boolean isHasMultipleValues() - { - return hasMultipleValues; - } - - @JsonProperty("bytesRetained") - public double getBytesRetained() - { - return bytesRetained; - } -} 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 efe4d959578d..f6b0ea9f08bb 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 @@ -123,10 +123,10 @@ public ControllerQueryKernelTester setupStage( if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { for (int i = 0; i < numWorkers; ++i) { - controllerQueryKernel.addResultStatisticsReportForStageAndWorker( + controllerQueryKernel.addPartialKeyStatisticsForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), i, - ClusterByStatisticsSnapshot.empty().aggregatedKeyStatistics(i) + ClusterByStatisticsSnapshot.empty().partialKeyStatistics() ); } } else { @@ -256,10 +256,10 @@ public ClusterByStatisticsCollector addResultKeyStatisticsForStageAndWorker(int keyStatsCollector.add(key, 1); } - controllerQueryKernel.addResultStatisticsReportForStageAndWorker( + controllerQueryKernel.addPartialKeyStatisticsForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), workerNumber, - keyStatsCollector.snapshot().aggregatedKeyStatistics(workerNumber) + keyStatsCollector.snapshot().partialKeyStatistics() ); return keyStatsCollector; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java similarity index 65% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java index d9604ad1d633..213ababd231f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReportSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.segment.TestHelper; @@ -30,9 +29,7 @@ import org.junit.Before; import org.junit.Test; -import java.util.TreeMap; - -public class ClusterByStatisticsWorkerReportSerdeTest +public class PartialKeyStatisticsInformationSerdeTest { private ObjectMapper objectMapper; @@ -47,19 +44,19 @@ public void setUp() @Test public void testSerde() throws JsonProcessingException { - WorkerAggregatedKeyStatistics workerReport = new WorkerAggregatedKeyStatistics( - new TreeMap<>(ImmutableMap.of(2L, ImmutableSet.of(2, 3))), + PartialKeyStatisticsInformation partialInformation = new PartialKeyStatisticsInformation( + ImmutableSet.of(2L, 3L), false, 0.0 ); - final String json = objectMapper.writeValueAsString(workerReport); - final WorkerAggregatedKeyStatistics deserializedKeyStatistics = objectMapper.readValue( + final String json = objectMapper.writeValueAsString(partialInformation); + final PartialKeyStatisticsInformation deserializedKeyStatistics = objectMapper.readValue( json, - WorkerAggregatedKeyStatistics.class + PartialKeyStatisticsInformation.class ); - Assert.assertEquals(json, workerReport.getTimeSegmentVsWorkerIdMap(), deserializedKeyStatistics.getTimeSegmentVsWorkerIdMap()); - Assert.assertEquals(json, workerReport.isHasMultipleValues(), deserializedKeyStatistics.isHasMultipleValues()); - Assert.assertEquals(json, workerReport.getBytesRetained(), deserializedKeyStatistics.getBytesRetained(), 0); + Assert.assertEquals(json, partialInformation.getTimeSegments(), deserializedKeyStatistics.getTimeSegments()); + Assert.assertEquals(json, partialInformation.isHasMultipleValues(), deserializedKeyStatistics.isHasMultipleValues()); + Assert.assertEquals(json, partialInformation.getBytesRetained(), deserializedKeyStatistics.getBytesRetained(), 0); } } 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 b697d5d9d1c0..0f474b381195 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 @@ -25,7 +25,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.List; @@ -40,17 +40,17 @@ public MSQTestControllerClient(Controller controller) } @Override - public void postAggregatedKeyStatistics( + public void postPartialKeyStatistics( StageId stageId, int workerNumber, - WorkerAggregatedKeyStatistics aggregatedKeyStatistics + PartialKeyStatisticsInformation partialKeyStatisticsInformation ) { try { - controller.updateAggregatedKeyStatistics(stageId.getStageNumber(), workerNumber, aggregatedKeyStatistics); + controller.updatePartialKeyStatistics(stageId.getStageNumber(), workerNumber, partialKeyStatisticsInformation); } catch (Exception e) { - throw new ISE(e, "unable to post aggregated key statistics"); + throw new ISE(e, "unable to post partial key statistics"); } } From 35d8e719fb1c97e171b76c9d6c416ec86029b2c1 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 15 Nov 2022 12:28:32 +0530 Subject: [PATCH 15/31] Rename fetchStatisticsSnapshotForTimeChunk function --- .../druid/msq/exec/ExceptionWrappingWorkerClient.java | 4 ++-- .../src/main/java/org/apache/druid/msq/exec/Worker.java | 2 +- .../main/java/org/apache/druid/msq/exec/WorkerClient.java | 2 +- .../main/java/org/apache/druid/msq/exec/WorkerImpl.java | 6 +++--- .../org/apache/druid/msq/exec/WorkerSketchFetcher.java | 6 +++--- .../apache/druid/msq/indexing/IndexerWorkerClient.java | 4 ++-- .../org/apache/druid/msq/indexing/WorkerChatHandler.java | 8 ++++---- .../druid/msq/statistics/ClusterByStatisticsSnapshot.java | 2 +- .../org/apache/druid/msq/test/MSQTestWorkerClient.java | 4 ++-- 9 files changed, 19 insertions(+), 19 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java index 1f8cd10a76c7..6267987bc002 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java @@ -65,14 +65,14 @@ public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String worke } @Override - public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, long timeChunk ) throws ExecutionException, InterruptedException { - return client.fetchSingletonStatisticsSnapshot(workerTaskId, queryId, stageNumber, timeChunk); + return client.fetchClusterByStatisticsSnapshotForTimeChunk(workerTaskId, queryId, stageNumber, timeChunk); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index 9081f5e7ee6f..a7e2359c72e2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -77,7 +77,7 @@ public interface Worker /** * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk */ - ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(StageId stageId, long timeChunk) + ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) throws ExecutionException, InterruptedException; /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java index e6fddd982980..c43c6fb24ff3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java @@ -51,7 +51,7 @@ ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk. * This is intended to be used by the {@link WorkerSketchFetcher}. */ - ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, 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 397cbd69e814..e8ba3aad218c 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 @@ -581,14 +581,14 @@ public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) } @Override - public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(StageId stageId, long timeChunk) + public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) throws ExecutionException, InterruptedException { CompletableFuture future = new CompletableFuture<>(); kernelManipulationQueue.add(kernelHolder -> { ClusterByStatisticsSnapshot snapshot = kernelHolder.stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); - ClusterByStatisticsSnapshot singletonSnapshot = snapshot.getSingletonSnapshot(timeChunk); - future.complete(singletonSnapshot); + ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshot.getSnapshotForTimeChunk(timeChunk); + future.complete(snapshotForTimeChunk); }); return future.get(); } 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 be8f70baa7e6..5dd3efdec46e 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 @@ -194,8 +194,8 @@ public void submitFetchingTasksForNextTimeChunk() for (int workerNo : workerIdsWithTimeChunk) { executorService.submit(() -> { try { - ClusterByStatisticsSnapshot singletonStatisticsSnapshot = - workerClient.fetchSingletonStatisticsSnapshot( + ClusterByStatisticsSnapshot snapshotForTimeChunk = + workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( workerTaskIds.get(workerNo), stageDefinition.getId().getQueryId(), stageDefinition.getStageNumber(), @@ -206,7 +206,7 @@ public void submitFetchingTasksForNextTimeChunk() return; } synchronized (mergedStatisticsCollector) { - mergedStatisticsCollector.addAll(singletonStatisticsSnapshot); + mergedStatisticsCollector.addAll(snapshotForTimeChunk); finishedWorkers.add(workerNo); if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java index d0e4e06d2c2f..8fb792a07eee 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java @@ -126,14 +126,14 @@ public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot( } @Override - public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, long timeChunk ) throws ExecutionException, InterruptedException { - String path = StringUtils.format("/singletonKeyStatistics/%s/%d/%d", + String path = StringUtils.format("/keyStatisticsForTimeChunk/%s/%d/%d", StringUtils.urlEncode(queryId), stageNumber, timeChunk); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java index 09e713d5225f..cac3a919442e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java @@ -209,7 +209,7 @@ public Response httpFetchKeyStatistics( } @POST - @Path("/singletonKeyStatistics/{queryId}/{stageNumber}/{timeChunk}") + @Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) public Response httpSketch( @@ -220,10 +220,10 @@ public Response httpSketch( ) { ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); - ClusterByStatisticsSnapshot singletonSnapshot; + ClusterByStatisticsSnapshot snapshotForTimeChunk; try { StageId stageId = new StageId(queryId, stageNumber); - singletonSnapshot = worker.fetchSingletonStatisticsSnapshot(stageId, timeChunk); + snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); } catch (ExecutionException | InterruptedException e) { return Response @@ -231,7 +231,7 @@ public Response httpSketch( .build(); } return Response.status(Response.Status.ACCEPTED) - .entity(singletonSnapshot) + .entity(snapshotForTimeChunk) .build(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index 2aa4b959dbff..476b3a6510fb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -58,7 +58,7 @@ Map getBuckets() return buckets; } - public ClusterByStatisticsSnapshot getSingletonSnapshot(long timeChunk) + public ClusterByStatisticsSnapshot getSnapshotForTimeChunk(long timeChunk) { Bucket bucket = buckets.get(timeChunk); return new ClusterByStatisticsSnapshot(ImmutableMap.of(timeChunk, bucket), null); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java index f7d092f21a62..1ba2abb19ca0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java @@ -61,7 +61,7 @@ public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String worke } @Override - public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( + public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, @@ -69,7 +69,7 @@ public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot( ) throws ExecutionException, InterruptedException { StageId stageId = new StageId(queryId, stageNumber); - return inMemoryWorkers.get(workerTaskId).fetchSingletonStatisticsSnapshot(stageId, timeChunk); + return inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); } @Override From 3a2078ffba80e2680e38f91179449f8f621392c7 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 15 Nov 2022 14:40:32 +0530 Subject: [PATCH 16/31] Address review comments --- .../msq/exec/ClusterStatisticsMergeMode.java | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 3 ++ .../druid/msq/exec/WorkerSketchFetcher.java | 8 ++-- .../druid/frame/key/RowKeyReaderTest.java | 38 +++++++++++++++++++ 4 files changed, 46 insertions(+), 5 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java index 027affc852ee..82d95e278779 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java @@ -20,7 +20,7 @@ package org.apache.druid.msq.exec; /** - * Mode which dictates how {@link WorkerSketchFetcher} gets sketches from workers. + * Mode which dictates how {@link WorkerSketchFetcher} gets sketches for the partition boundaries from workers. */ public enum ClusterStatisticsMergeMode { 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 f365df729f76..8c28d0a4c87f 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 @@ -526,6 +526,9 @@ private QueryDefinition initializeQueryDefAndState(final Closer 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); 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 5dd3efdec46e..e6a2ef3dbd84 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 @@ -37,13 +37,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.stream.IntStream; /** * Queues up fetching sketches from workers and progressively generates partitions boundaries. */ public class WorkerSketchFetcher { - private static final int DEFAULT_THREAD_COUNT = 10; + private static final int DEFAULT_THREAD_COUNT = 4; private static final long BYTES_THRESHOLD = 1_000_000_000L; private static final long WORKER_THRESHOLD = 100; @@ -107,8 +108,7 @@ private CompletableFuture> inMemoryFullSketchM final int workerCount = workerTaskIds.size(); final Set finishedWorkers = new HashSet<>(); - for (int i = 0; i < workerCount; i++) { - final int workerNo = i; + IntStream.range(0, workerCount).forEach(workerNo -> { executorService.submit(() -> { try { ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = workerClient.fetchClusterByStatisticsSnapshot( @@ -135,7 +135,7 @@ private CompletableFuture> inMemoryFullSketchM partitionFuture.completeExceptionally(e); } }); - } + }); return partitionFuture; } diff --git a/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java b/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java index 7a9131729539..48d940384270 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java @@ -30,6 +30,7 @@ import org.junit.internal.matchers.ThrowableMessageMatcher; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.IntStream; @@ -144,4 +145,41 @@ public void test_trim_beyondFullLength() MatcherAssert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Cannot trim"))); } + + @Test + public void test_trimmedKeyReader_zero() + { + RowKey trimmedKey = keyReader.trim(key, 0); + RowKeyReader trimmedKeyReader = keyReader.trimmedKeyReader(0); + + Assert.assertEquals( + Collections.emptyList(), + trimmedKeyReader.read(trimmedKey) + ); + } + + @Test + public void test_trimmedKeyReader_one() + { + RowKey trimmedKey = keyReader.trim(key, 1); + RowKeyReader trimmedKeyReader = keyReader.trimmedKeyReader(1); + + Assert.assertEquals( + objects.subList(0, 1), + trimmedKeyReader.read(trimmedKey) + ); + } + + @Test + public void test_trimmedKeyReader_oneLessThanFullLength() + { + final int numFields = signature.size() - 1; + RowKey trimmedKey = keyReader.trim(key, numFields); + RowKeyReader trimmedKeyReader = keyReader.trimmedKeyReader(numFields); + + Assert.assertEquals( + objects.subList(0, numFields), + trimmedKeyReader.read(trimmedKey) + ); + } } From bcaeae9f58ebe1430f790a20fd6689626fdc30ee Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 15 Nov 2022 15:51:50 +0530 Subject: [PATCH 17/31] Address review comments --- docs/multi-stage-query/reference.md | 3 ++- .../controller/ControllerStageTracker.java | 7 +------ .../CompleteKeyStatisticsInformation.java | 19 +++++++++++++++++++ .../PartialKeyStatisticsInformation.java | 19 +++++++++++++++++++ 4 files changed, 41 insertions(+), 7 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index dee5a44300da..ca1c6e24f283 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -203,7 +203,8 @@ The following table lists the context parameters for the MSQ task engine: | `maxParseExceptions`| SELECT, INSERT, REPLACE

Maximum number of parse exceptions that are ignored while executing the query before it stops with `TooManyWarningsFault`. To ignore all the parse exceptions, set the value to -1.| 0 | | `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 [`indexSpec`](../ingestion/ingestion-spec.md#indexspec). | -| clusterStatisticsMergeMode | Whether to parallel or sequential merging of worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. On `AUTO` tries to find the best approach based on number of workers and size of input rows. | `AUTO` | + +| `clusterStatisticsMergeMode` | Whether to parallel or sequential merging of worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. On `AUTO` tries to find the best approach based on number of workers and size of input rows. | `AUTO` | ## Durable Storage This section enumerates the advantages and performance implications of enabling durable storage while executing MSQ tasks. 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 ada67e5b67c6..5a6969fa29a4 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 @@ -247,7 +247,7 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( if (phase != ControllerStagePhase.READING_INPUT) { throw new ISE("Cannot add result key statistics from stage [%s]", phase); } - if (completeKeyStatisticsInformation == null) { + if (!stageDef.doesShuffle() || completeKeyStatisticsInformation == null ) { throw new ISE("Stage does not gather result key statistics"); } @@ -264,11 +264,6 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers. transitionTo(ControllerStagePhase.MERGING_STATISTICS); - if (!stageDef.doesShuffle()) { - // We don't need to wait for key statistics in this case. We can generate parititions and skip to the next phase. - generateResultPartitionsAndBoundariesWithoutKeyStatistics(); - transitionTo(ControllerStagePhase.POST_READING); - } } } } 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 0206a73ac997..ac7c33694158 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 @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.msq.statistics; import java.util.HashSet; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java index a249d682876e..0040d6dcb4b1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.msq.statistics; import com.fasterxml.jackson.annotation.JsonCreator; From 18c60bc6f4703d5640d8bb855ce4f401f6a08e69 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 16 Nov 2022 11:59:27 +0530 Subject: [PATCH 18/31] Update documentation and add comments --- docs/multi-stage-query/reference.md | 22 ++++++++++++++- .../msq/exec/ClusterStatisticsMergeMode.java | 4 +++ .../org/apache/druid/msq/exec/Worker.java | 6 ++-- .../apache/druid/msq/exec/WorkerClient.java | 4 +-- .../druid/msq/exec/WorkerSketchFetcher.java | 28 ++++++++++++++++--- .../controller/ControllerStageTracker.java | 2 +- .../ClusterByStatisticsSnapshot.java | 4 +-- 7 files changed, 58 insertions(+), 12 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index ca1c6e24f283..e0deb2b4698a 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -203,8 +203,28 @@ The following table lists the context parameters for the MSQ task engine: | `maxParseExceptions`| SELECT, INSERT, REPLACE

Maximum number of parse exceptions that are ignored while executing the query before it stops with `TooManyWarningsFault`. To ignore all the parse exceptions, set the value to -1.| 0 | | `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 [`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. | `AUTO` | -| `clusterStatisticsMergeMode` | Whether to parallel or sequential merging of worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. On `AUTO` tries to find the best approach based on number of workers and size of input rows. | `AUTO` | +## Sketch Merging Mode +This section details the advantages and performance of various Cluster By Statistics Merge Modes. + +If a query requires key statistics to generate partition boundaries, key statistics are gathered by the workers while +reading rows from the datasource. These statistics must be transfered to the controller to be merged together. +`clusterStatisticsMergeMode` configures the way in which this happens. + +`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 +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 +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 +than 100 workers or if the combined sketch size among all workers is more than 1GB, `SEQUENTIAL` is chosen, otherwise, +`PARALLEL` is chosen. ## Durable Storage This section enumerates the advantages and performance implications of enabling durable storage while executing MSQ tasks. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java index 82d95e278779..fc158e58aaaf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java @@ -36,6 +36,10 @@ public enum ClusterStatisticsMergeMode /** * Tries to decide between sequential and parallel modes based on the number of workers and size of the input + * + * If there are more than {@link WorkerSketchFetcher.WORKER_THRESHOLD} workers or if the combined sketch size + * among all workers is more than {@link WorkerSketchFetcher.BYTES_THRESHOLD}, `SEQUENTIAL` is chosen, otherwise, + * `PARALLEL` is chosen. */ AUTO } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index a7e2359c72e2..e24a648fd6f0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -70,12 +70,14 @@ public interface Worker void postWorkOrder(WorkOrder workOrder); /** - * Returns the statistics snapshot for the given stageId + * Returns the statistics snapshot for the given stageId. This is called from {@link WorkerSketchFetcher} under + * PARALLEL OR AUTO modes. */ ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) throws ExecutionException, InterruptedException; /** - * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk + * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk. + * This is called from {@link WorkerSketchFetcher} under SEQUENTIAL OR AUTO modes. */ ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) throws ExecutionException, InterruptedException; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java index c43c6fb24ff3..a210e55300c5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java @@ -42,14 +42,14 @@ public interface WorkerClient extends AutoCloseable /** * Fetches the {@link ClusterByStatisticsSnapshot} from a worker. This is intended to be used by the - * {@link WorkerSketchFetcher}. + * {@link WorkerSketchFetcher} under PARALLEL or AUTO modes. */ ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) throws ExecutionException, InterruptedException; /** * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk. - * This is intended to be used by the {@link WorkerSketchFetcher}. + * This is intended to be used by the {@link WorkerSketchFetcher} under SEQUENTIAL or AUTO modes. */ ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, 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 e6a2ef3dbd84..346d8dc1f152 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 @@ -45,8 +45,11 @@ public class WorkerSketchFetcher { private static final int DEFAULT_THREAD_COUNT = 4; - private static final long BYTES_THRESHOLD = 1_000_000_000L; - private static final long WORKER_THRESHOLD = 100; + + // If the combined size of worker sketches is more than this threshold, SEQUENTIAL merging mode is used. + public static final long BYTES_THRESHOLD = 1_000_000_000L; + // If there are more workers than this threshold, SEQUENTIAL merging mode is used. + public static final long WORKER_THRESHOLD = 100; private final ClusterStatisticsMergeMode clusterStatisticsMergeMode; private final int statisticsMaxRetainedBytes; @@ -104,10 +107,13 @@ private CompletableFuture> inMemoryFullSketchM { CompletableFuture> partitionFuture = new CompletableFuture<>(); - final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); + // Create a new key statistics collector to merge worker sketches into + final ClusterByStatisticsCollector mergedStatisticsCollector = + stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); final int workerCount = workerTaskIds.size(); final Set finishedWorkers = new HashSet<>(); + // Submit a task for each worker to fetch statistics IntStream.range(0, workerCount).forEach(workerNo -> { executorService.submit(() -> { try { @@ -165,6 +171,7 @@ private class SequentialFetchStage 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( @@ -186,11 +193,15 @@ public void submitFetchingTasksForNextTimeChunk() 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(); - ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); + // Create a new key statistics collector to merge worker sketches into + ClusterByStatisticsCollector mergedStatisticsCollector = + stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); Set finishedWorkers = new HashSet<>(); + // Submits a task for every worker which has a certain time chunk for (int workerNo : workerIdsWithTimeChunk) { executorService.submit(() -> { try { @@ -235,12 +246,21 @@ public void submitFetchingTasksForNextTimeChunk() } } + /** + * 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())); } 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 5a6969fa29a4..62a1e3261526 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 @@ -247,7 +247,7 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( if (phase != ControllerStagePhase.READING_INPUT) { throw new ISE("Cannot add result key statistics from stage [%s]", phase); } - if (!stageDef.doesShuffle() || completeKeyStatisticsInformation == null ) { + if (!stageDef.doesShuffle() || completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index 476b3a6510fb..e54253ad2186 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -74,8 +74,8 @@ Set getHasMultipleValues() public PartialKeyStatisticsInformation partialKeyStatistics() { double bytesRetained = 0; - for (Long bucketKey : buckets.keySet()) { - bytesRetained += buckets.get(bucketKey).bytesRetained; + for (ClusterByStatisticsSnapshot.Bucket bucket : buckets.values()) { + bytesRetained += bucket.bytesRetained; } return new PartialKeyStatisticsInformation(buckets.keySet(), !getHasMultipleValues().isEmpty(), bytesRetained); } From 2f91002ce62a8b49b530e94556d92d40bf38de45 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 16 Nov 2022 13:24:29 +0530 Subject: [PATCH 19/31] Resolve build issues --- docs/multi-stage-query/reference.md | 2 +- .../druid/msq/exec/ClusterStatisticsMergeMode.java | 5 ++--- .../org/apache/druid/msq/exec/ControllerImpl.java | 14 +++++++------- .../apache/druid/msq/exec/WorkerSketchFetcher.java | 7 ++++--- website/.spelling | 3 +++ 5 files changed, 17 insertions(+), 14 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index e0deb2b4698a..9811ce06bd69 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -209,7 +209,7 @@ The following table lists the context parameters for the MSQ task engine: This section details the advantages and performance of various Cluster By Statistics Merge Modes. If a query requires key statistics to generate partition boundaries, key statistics are gathered by the workers while -reading rows from the datasource. These statistics must be transfered to the controller to be merged together. +reading rows from the datasource. These statistics must be transferred to the controller to be merged together. `clusterStatisticsMergeMode` configures the way in which this happens. `PARALLEL` mode fetches the key statistics for all time chunks from all workers together and the controller then downsamples diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java index fc158e58aaaf..92ed82ff5e11 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java @@ -37,9 +37,8 @@ public enum ClusterStatisticsMergeMode /** * Tries to decide between sequential and parallel modes based on the number of workers and size of the input * - * If there are more than {@link WorkerSketchFetcher.WORKER_THRESHOLD} workers or if the combined sketch size - * among all workers is more than {@link WorkerSketchFetcher.BYTES_THRESHOLD}, `SEQUENTIAL` is chosen, otherwise, - * `PARALLEL` is chosen. + * If there are more than 100 workers or if the combined sketch size among all workers is more than + * 1,000,000,000 bytes, SEQUENTIAL mode is chosen, otherwise, PARALLEL mode is chosen. */ AUTO } 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 8c28d0a4c87f..40059d223f65 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 @@ -628,14 +628,14 @@ public void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object // Add the listener to handle completion. clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> { - if (throwable != null) { - queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable)); - } else if (clusterByPartitionsEither.isError()) { - queryKernel.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); - } else { - queryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow()); - } kernelManipulationQueue.add(holder -> { + if (throwable != null) { + queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable)); + } else if (clusterByPartitionsEither.isError()) { + queryKernel.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); + } else { + queryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow()); + } holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId)); }); }); 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 346d8dc1f152..ddb63727fba3 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 @@ -45,11 +45,10 @@ public class WorkerSketchFetcher { 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 BYTES_THRESHOLD = 1_000_000_000L; + private static final long BYTES_THRESHOLD = 1_000_000_000L; // If there are more workers than this threshold, SEQUENTIAL merging mode is used. - public static final long WORKER_THRESHOLD = 100; + private static final long WORKER_THRESHOLD = 100; private final ClusterStatisticsMergeMode clusterStatisticsMergeMode; private final int statisticsMaxRetainedBytes; @@ -111,6 +110,7 @@ private CompletableFuture> inMemoryFullSketchM final ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); final int workerCount = workerTaskIds.size(); + // Guarded by synchronized mergedStatisticsCollector final Set finishedWorkers = new HashSet<>(); // Submit a task for each worker to fetch statistics @@ -199,6 +199,7 @@ public void submitFetchingTasksForNextTimeChunk() // Create a new key statistics collector to merge worker sketches into ClusterByStatisticsCollector mergedStatisticsCollector = stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); + // Guarded by synchronized mergedStatisticsCollector Set finishedWorkers = new HashSet<>(); // Submits a task for every worker which has a certain time chunk diff --git a/website/.spelling b/website/.spelling index c27c4d8ccc80..660f2d18b508 100644 --- a/website/.spelling +++ b/website/.spelling @@ -68,6 +68,9 @@ Double.NEGATIVE_INFINITY Double.NEGATIVE_INFINITY. Double.POSITIVE_INFINITY Double.POSITIVE_INFINITY. +Downsampled +Downsamples +Downsampling Dropwizard dropwizard DruidInputSource From 3852cd1dcbacb5c7ecae9afa49d1d3c2d0e4c4e0 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 16 Nov 2022 15:06:59 +0530 Subject: [PATCH 20/31] Resolve build issues --- .../java/org/apache/druid/msq/exec/WorkerSketchFetcher.java | 5 ++--- website/.spelling | 6 +++--- 2 files changed, 5 insertions(+), 6 deletions(-) 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 ddb63727fba3..b8b26b34ca0d 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 @@ -82,13 +82,12 @@ public CompletableFuture> submitFetcherTask( return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); case AUTO: if (clusterBy.getBucketByCount() == 0) { - // If there is no time cluserting, there is no scope for sequential merge + // 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) { return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); - } else { - return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); } + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); default: throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode); } diff --git a/website/.spelling b/website/.spelling index 660f2d18b508..c2eb66104ee8 100644 --- a/website/.spelling +++ b/website/.spelling @@ -68,9 +68,9 @@ Double.NEGATIVE_INFINITY Double.NEGATIVE_INFINITY. Double.POSITIVE_INFINITY Double.POSITIVE_INFINITY. -Downsampled -Downsamples -Downsampling +downsampled +downsamples +downsampling Dropwizard dropwizard DruidInputSource From 682aa71f9eb2d1d025f52c1587332c1e1851e9df Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 16 Nov 2022 18:37:51 +0530 Subject: [PATCH 21/31] Change worker APIs to async --- .../exec/ExceptionWrappingWorkerClient.java | 8 +-- .../apache/druid/msq/exec/WorkerClient.java | 12 ++-- .../druid/msq/exec/WorkerSketchFetcher.java | 67 ++++++++++--------- .../msq/indexing/IndexerWorkerClient.java | 21 +++--- .../druid/msq/test/MSQTestWorkerClient.java | 35 ++++++++-- 5 files changed, 84 insertions(+), 59 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java index 6267987bc002..3d78b7c9ced4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java @@ -35,7 +35,6 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.concurrent.ExecutionException; /** * Wrapper around any {@link WorkerClient} that converts exceptions into {@link MSQException} @@ -58,19 +57,18 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO } @Override - public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) - throws ExecutionException, InterruptedException + public ListenableFuture fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) { return client.fetchClusterByStatisticsSnapshot(workerTaskId, queryId, stageNumber); } @Override - public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( + public ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, long timeChunk - ) throws ExecutionException, InterruptedException + ) { return client.fetchClusterByStatisticsSnapshotForTimeChunk(workerTaskId, queryId, stageNumber, timeChunk); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java index a210e55300c5..5c02a79f89a3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java @@ -28,7 +28,6 @@ import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import java.io.IOException; -import java.util.concurrent.ExecutionException; /** * Client for multi-stage query workers. Used by the controller task. @@ -44,19 +43,22 @@ public interface WorkerClient extends AutoCloseable * Fetches the {@link ClusterByStatisticsSnapshot} from a worker. This is intended to be used by the * {@link WorkerSketchFetcher} under PARALLEL or AUTO modes. */ - ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) - throws ExecutionException, InterruptedException; + ListenableFuture fetchClusterByStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber + ); /** * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk. * This is intended to be used by the {@link WorkerSketchFetcher} under SEQUENTIAL or AUTO modes. */ - ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( + ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, long timeChunk - ) throws ExecutionException, InterruptedException; + ); /** * Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the 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 b8b26b34ca0d..67905669fb34 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,6 +19,7 @@ package org.apache.druid.msq.exec; +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; @@ -115,19 +116,17 @@ private CompletableFuture> inMemoryFullSketchM // Submit a task for each worker to fetch statistics IntStream.range(0, workerCount).forEach(workerNo -> { executorService.submit(() -> { - try { - ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = workerClient.fetchClusterByStatisticsSnapshot( - workerTaskIds.get(workerNo), - stageDefinition.getId().getQueryId(), - stageDefinition.getStageNumber() - ); + ListenableFuture snapshotFuture = + workerClient.fetchClusterByStatisticsSnapshot( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber() + ); + partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true)); - // If the future already failed for some reason, stop the task. - if (partitionFuture.isDone()) { - return; - } - - synchronized (mergedStatisticsCollector) { + synchronized (mergedStatisticsCollector) { + try { + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); finishedWorkers.add(workerNo); @@ -135,9 +134,9 @@ private CompletableFuture> inMemoryFullSketchM partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); } } - } - catch (Exception e) { - partitionFuture.completeExceptionally(e); + catch (Exception e) { + partitionFuture.completeExceptionally(e); + } } }); }); @@ -204,19 +203,18 @@ public void submitFetchingTasksForNextTimeChunk() // Submits a task for every worker which has a certain time chunk for (int workerNo : workerIdsWithTimeChunk) { executorService.submit(() -> { - try { - ClusterByStatisticsSnapshot snapshotForTimeChunk = - workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( - workerTaskIds.get(workerNo), - stageDefinition.getId().getQueryId(), - stageDefinition.getStageNumber(), - timeChunk - ); - // If the future already failed for some reason, stop the task. - if (partitionFuture.isDone()) { - return; - } - synchronized (mergedStatisticsCollector) { + ListenableFuture snapshotFuture = + workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber(), + timeChunk + ); + partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true)); + + synchronized (mergedStatisticsCollector) { + try { + ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get(); mergedStatisticsCollector.addAll(snapshotForTimeChunk); finishedWorkers.add(workerNo); @@ -234,12 +232,17 @@ public void submitFetchingTasksForNextTimeChunk() .ranges(); abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchPartitions); - submitFetchingTasksForNextTimeChunk(); + if (finalPartitionBoundries.size() > stageDefinition.getMaxPartitionCount()) { + // Fail fast if more partitions than the maximum have been reached. + partitionFuture.complete(Either.error((long) finalPartitionBoundries.size())); + } else { + submitFetchingTasksForNextTimeChunk(); + } } } - } - catch (Exception e) { - partitionFuture.completeExceptionally(e); + catch (Exception e) { + partitionFuture.completeExceptionally(e); + } } }); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java index 8fb792a07eee..980c7f97bee5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java @@ -60,7 +60,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; public class IndexerWorkerClient implements WorkerClient @@ -106,44 +105,44 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO } @Override - public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot( + public ListenableFuture fetchClusterByStatisticsSnapshot( String workerTaskId, String queryId, int stageNumber - ) throws ExecutionException, InterruptedException + ) { String path = StringUtils.format("/keyStatistics/%s/%d", StringUtils.urlEncode(queryId), stageNumber); - return deserialize( - getClient(workerTaskId).request( + return FutureUtils.transform( + getClient(workerTaskId).asyncRequest( new RequestBuilder(HttpMethod.POST, path), new BytesFullResponseHandler() ), - new TypeReference() {} + holder -> deserialize(holder, new TypeReference() {}) ); } @Override - public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( + public ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, long timeChunk - ) throws ExecutionException, InterruptedException + ) { String path = StringUtils.format("/keyStatisticsForTimeChunk/%s/%d/%d", StringUtils.urlEncode(queryId), stageNumber, timeChunk); - return deserialize( - getClient(workerTaskId).request( + return FutureUtils.transform( + getClient(workerTaskId).asyncRequest( new RequestBuilder(HttpMethod.POST, path), new BytesFullResponseHandler() ), - new TypeReference() {} + holder -> deserialize(holder, new TypeReference() {}) ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java index 1ba2abb19ca0..28566c082fbf 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java @@ -53,23 +53,46 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO } @Override - public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) - throws ExecutionException, InterruptedException + public ListenableFuture fetchClusterByStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber + ) { StageId stageId = new StageId(queryId, stageNumber); - return inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId); + try { + return Futures.immediateFuture(inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId)); + } + catch (ExecutionException e) { + // TODO: fix + throw new RuntimeException(e); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } } @Override - public ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk( + public ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( String workerTaskId, String queryId, int stageNumber, long timeChunk - ) throws ExecutionException, InterruptedException + ) { StageId stageId = new StageId(queryId, stageNumber); - return inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); + try { + return Futures.immediateFuture( + inMemoryWorkers.get(workerTaskId) + .fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk) + ); + } + catch (ExecutionException e) { + throw new RuntimeException(e); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } } @Override From 85ad968e733118c7d2a2036e6b65b805ba65647b Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 16 Nov 2022 18:54:43 +0530 Subject: [PATCH 22/31] Address review comments --- .../org/apache/druid/msq/exec/Worker.java | 6 ++--- .../org/apache/druid/msq/exec/WorkerImpl.java | 23 ++++------------- .../druid/msq/exec/WorkerSketchFetcher.java | 20 +++++++++------ .../druid/msq/indexing/WorkerChatHandler.java | 24 +++--------------- .../druid/msq/test/MSQTestWorkerClient.java | 25 ++----------------- 5 files changed, 25 insertions(+), 73 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index e24a648fd6f0..cc5f0fae1732 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -30,7 +30,6 @@ import javax.annotation.Nullable; import java.io.IOException; import java.io.InputStream; -import java.util.concurrent.ExecutionException; public interface Worker { @@ -73,14 +72,13 @@ public interface Worker * Returns the statistics snapshot for the given stageId. This is called from {@link WorkerSketchFetcher} under * PARALLEL OR AUTO modes. */ - ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) throws ExecutionException, InterruptedException; + ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId); /** * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk. * This is called from {@link WorkerSketchFetcher} under SEQUENTIAL OR AUTO modes. */ - ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) - throws ExecutionException, InterruptedException; + ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk); /** * Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber 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 e8ba3aad218c..49d6f9080d7c 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 @@ -136,10 +136,8 @@ import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -162,6 +160,7 @@ public class WorkerImpl implements Worker private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>(); private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); + private final ConcurrentHashMap stageKernelMap = new ConcurrentHashMap<>(); private final boolean durableStageStorageEnabled; /** @@ -571,26 +570,15 @@ public void postFinish() @Override public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) - throws ExecutionException, InterruptedException { - CompletableFuture future = new CompletableFuture<>(); - kernelManipulationQueue.add(kernelHolder -> { - future.complete(kernelHolder.stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot()); - }); - return future.get(); + return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); } @Override public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) - throws ExecutionException, InterruptedException { - CompletableFuture future = new CompletableFuture<>(); - kernelManipulationQueue.add(kernelHolder -> { - ClusterByStatisticsSnapshot snapshot = kernelHolder.stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); - ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshot.getSnapshotForTimeChunk(timeChunk); - future.complete(snapshotForTimeChunk); - }); - return future.get(); + ClusterByStatisticsSnapshot snapshot = stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); + return snapshot.getSnapshotForTimeChunk(timeChunk); } @Override @@ -1304,9 +1292,8 @@ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int p } } - private static class KernelHolder + private class KernelHolder { - private final Map stageKernelMap = new HashMap<>(); private boolean done = false; public Map getStageKernelMap() 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 67905669fb34..dc04e8c2b8bc 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 @@ -124,9 +124,9 @@ private CompletableFuture> inMemoryFullSketchM ); partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true)); - synchronized (mergedStatisticsCollector) { - try { - ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); + try { + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); + synchronized (mergedStatisticsCollector) { mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); finishedWorkers.add(workerNo); @@ -134,7 +134,9 @@ private CompletableFuture> inMemoryFullSketchM partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); } } - catch (Exception e) { + } + catch (Exception e) { + synchronized (mergedStatisticsCollector) { partitionFuture.completeExceptionally(e); } } @@ -212,9 +214,9 @@ public void submitFetchingTasksForNextTimeChunk() ); partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true)); - synchronized (mergedStatisticsCollector) { - try { - ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get(); + try { + ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get(); + synchronized (mergedStatisticsCollector) { mergedStatisticsCollector.addAll(snapshotForTimeChunk); finishedWorkers.add(workerNo); @@ -240,7 +242,9 @@ public void submitFetchingTasksForNextTimeChunk() } } } - catch (Exception e) { + } + catch (Exception e) { + synchronized (mergedStatisticsCollector) { partitionFuture.completeExceptionally(e); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java index cac3a919442e..dd6ea7cb7124 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java @@ -48,7 +48,6 @@ import javax.ws.rs.core.StreamingOutput; import java.io.IOException; import java.io.InputStream; -import java.util.concurrent.ExecutionException; public class WorkerChatHandler implements ChatHandler { @@ -193,16 +192,8 @@ public Response httpFetchKeyStatistics( { ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); ClusterByStatisticsSnapshot clusterByStatisticsSnapshot; - try { - StageId stageId = new StageId(queryId, stageNumber); - clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); - } - catch (ExecutionException | InterruptedException e) { - return Response - .status(Response.Status.INTERNAL_SERVER_ERROR) - .build(); - - } + StageId stageId = new StageId(queryId, stageNumber); + clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); return Response.status(Response.Status.ACCEPTED) .entity(clusterByStatisticsSnapshot) .build(); @@ -221,15 +212,8 @@ public Response httpSketch( { ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); ClusterByStatisticsSnapshot snapshotForTimeChunk; - try { - StageId stageId = new StageId(queryId, stageNumber); - snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); - } - catch (ExecutionException | InterruptedException e) { - return Response - .status(Response.Status.INTERNAL_SERVER_ERROR) - .build(); - } + StageId stageId = new StageId(queryId, stageNumber); + snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); return Response.status(Response.Status.ACCEPTED) .entity(snapshotForTimeChunk) .build(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java index 28566c082fbf..ae892c34500a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java @@ -34,7 +34,6 @@ import java.io.InputStream; import java.util.Arrays; import java.util.Map; -import java.util.concurrent.ExecutionException; public class MSQTestWorkerClient implements WorkerClient { @@ -60,16 +59,7 @@ public ListenableFuture fetchClusterByStatisticsSna ) { StageId stageId = new StageId(queryId, stageNumber); - try { - return Futures.immediateFuture(inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId)); - } - catch (ExecutionException e) { - // TODO: fix - throw new RuntimeException(e); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } + return Futures.immediateFuture(inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId)); } @Override @@ -81,18 +71,7 @@ public ListenableFuture fetchClusterByStatisticsSna ) { StageId stageId = new StageId(queryId, stageNumber); - try { - return Futures.immediateFuture( - inMemoryWorkers.get(workerTaskId) - .fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk) - ); - } - catch (ExecutionException e) { - throw new RuntimeException(e); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } + return Futures.immediateFuture(inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk)); } @Override From 090ce0747d96072cae74ce29ede54b630ea115ad Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 16 Nov 2022 22:33:49 +0530 Subject: [PATCH 23/31] Resolve build issues --- integration-tests-ex/cases/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 5456d4b81b01..2b89caa829e2 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -178,6 +178,11 @@ + + org.apache.druid + druid-sql + 25.0.0-SNAPSHOT + org.apache.druid.extensions druid-multi-stage-query From aa58285fa398352240663a997526c1760c7169f0 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 17 Nov 2022 06:57:33 +0530 Subject: [PATCH 24/31] Add null time check --- .../msq/kernel/controller/ControllerStageTracker.java | 8 ++++++++ 1 file changed, 8 insertions(+) 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 62a1e3261526..54a6f940f9ad 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 @@ -28,6 +28,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.msq.indexing.error.InsertTimeNullFault; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; import org.apache.druid.msq.indexing.error.UnknownFault; @@ -257,6 +258,13 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( try { if (workersWithReportedKeyStatistics.add(workerNumber)) { + + if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) { + // Time should not contain null value + failForReason(InsertTimeNullFault.instance()); + return getPhase(); + } + completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation); if (workersWithReportedKeyStatistics.size() == workerCount) { From 522c09086a6405822487ceabc824cc1c5f7a50dd Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 17 Nov 2022 10:18:19 +0530 Subject: [PATCH 25/31] Update integration tests --- .../msq/ITKeyStatisticsSketchMergeMode.java | 187 ++++++++++++++++++ .../druid/testsEx/msq/ITMultiStageQuery.java | 144 -------------- 2 files changed, 187 insertions(+), 144 deletions(-) create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java 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 new file mode 100644 index 000000000000..b1d1d7c77bc9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java @@ -0,0 +1,187 @@ +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.StringUtils; +import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; +import org.apache.druid.msq.sql.SqlTaskStatus; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.sql.http.SqlQuery; +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.MsqTestQueryHelper; +import org.apache.druid.testsEx.categories.MultiStageQuery; +import org.apache.druid.testsEx.config.DruidTestRunner; +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 ITKeyStatisticsSketchMergeMode +{ + @Inject + private MsqTestQueryHelper msqHelper; + + @Inject + private SqlResourceTestClient msqClient; + + @Inject + private IntegrationTestingConfig config; + + @Inject + private ObjectMapper jsonMapper; + + @Inject + private DataLoaderHelper dataLoaderHelper; + + @Inject + private CoordinatorResourceTestClient coordinatorClient; + + private static final String QUERY_FILE = "/multi-stage-query/wikipedia_msq_select_query1.json"; + + + @Test + public void testMsqIngestionParallelMerging() 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\"]}',\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 + ); + + ImmutableMap context = ImmutableMap.of( + MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, + ClusterStatisticsMergeMode.PARALLEL + ); + + // Submit the task and wait for the datasource to get loaded + SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } + + @Test + public void testMsqIngestionSequentialMerging() 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\"]}',\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 + ); + + ImmutableMap context = ImmutableMap.of( + MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, + ClusterStatisticsMergeMode.SEQUENTIAL + ); + + // Submit the task and wait for the datasource to get loaded + SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForCompletion(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 78a23019039d..e7ec1c0e3e4b 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 @@ -20,13 +20,9 @@ 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.StringUtils; -import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.sql.SqlTaskStatus; -import org.apache.druid.msq.util.MultiStageQueryContext; -import org.apache.druid.sql.http.SqlQuery; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; import org.apache.druid.testing.clients.SqlResourceTestClient; @@ -126,144 +122,4 @@ public void testMsqIngestionAndQuerying() throws Exception msqHelper.testQueriesFromFile(QUERY_FILE, datasource); } - - @Test - public void testMsqIngestionParallelMerging() 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\"]}',\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 - ); - - ImmutableMap context = ImmutableMap.of( - MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, - ClusterStatisticsMergeMode.PARALLEL - ); - - // Submit the task and wait for the datasource to get loaded - SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); - - if (sqlTaskStatus.getState().isFailure()) { - Assert.fail(StringUtils.format( - "Unable to start the task successfully.\nPossible exception: %s", - sqlTaskStatus.getError() - )); - } - - msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); - dataLoaderHelper.waitUntilDatasourceIsReady(datasource); - - msqHelper.testQueriesFromFile(QUERY_FILE, datasource); - } - - @Test - public void testMsqIngestionSequentialMerging() 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\"]}',\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 - ); - - ImmutableMap context = ImmutableMap.of( - MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, - ClusterStatisticsMergeMode.SEQUENTIAL - ); - - // Submit the task and wait for the datasource to get loaded - SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); - - if (sqlTaskStatus.getState().isFailure()) { - Assert.fail(StringUtils.format( - "Unable to start the task successfully.\nPossible exception: %s", - sqlTaskStatus.getError() - )); - } - - msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); - dataLoaderHelper.waitUntilDatasourceIsReady(datasource); - - msqHelper.testQueriesFromFile(QUERY_FILE, datasource); - } } From 953ff05edbf414de59eff552af7a226437a5cb09 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 17 Nov 2022 15:05:07 +0530 Subject: [PATCH 26/31] Address review comments --- .../org/apache/druid/msq/exec/Controller.java | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 13 +- .../druid/msq/exec/WorkerSketchFetcher.java | 49 +++++- .../msq/indexing/ControllerChatHandler.java | 4 +- .../msq/indexing/IndexerControllerClient.java | 2 +- .../controller/ControllerStagePhase.java | 1 + .../controller/ControllerStageTracker.java | 14 +- .../CompleteKeyStatisticsInformation.java | 23 ++- .../PartialKeyStatisticsInformation.java | 12 +- .../msq/exec/WorkerSketchFetcherTest.java | 139 ++++++++++++++++++ ...tialKeyStatisticsInformationSerdeTest.java | 2 +- .../msq/test/MSQTestControllerClient.java | 2 +- 12 files changed, 229 insertions(+), 34 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java 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 4b3808dee505..fe010b219e49 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 @@ -85,7 +85,7 @@ public String getId() * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate partiton boundaries. * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ - void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject); + void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject); /** * System error reported by a subtask. Note that the errors are organized by 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 40059d223f65..cafc0f389258 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) * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ @Override - public void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject) + public void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject) { addToKernelManipulationQueue( queryKernel -> { @@ -600,7 +600,7 @@ public void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object final PartialKeyStatisticsInformation partialKeyStatisticsInformation; try { - partialKeyStatisticsInformation = mapper.convertValue(partialKeyStatisticsObject, PartialKeyStatisticsInformation.class); + partialKeyStatisticsInformation = mapper.convertValue(partialKeyStatisticsInformationObject, PartialKeyStatisticsInformation.class); } catch (IllegalArgumentException e) { throw new IAE( @@ -628,13 +628,14 @@ public void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object // Add the listener to handle completion. clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> { - kernelManipulationQueue.add(holder -> { + addToKernelManipulationQueue(holder -> { if (throwable != null) { - queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable)); + holder.failStageForReason(stageId, UnknownFault.forException(throwable)); } else if (clusterByPartitionsEither.isError()) { - queryKernel.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); + holder.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); } else { - queryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow()); + log.debug("Query [%s] Partition boundaries generated for stage %s", id(), stageId); + holder.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow()); } holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId)); }); 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 dc04e8c2b8bc..5999eaaf20a1 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 @@ -24,6 +24,8 @@ 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.ISE; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; @@ -45,11 +47,12 @@ */ 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. - private static final long BYTES_THRESHOLD = 1_000_000_000L; + static final long BYTES_THRESHOLD = 1_000_000_000L; // If there are more workers than this threshold, SEQUENTIAL merging mode is used. - private static final long WORKER_THRESHOLD = 100; + static final long WORKER_THRESHOLD = 100; private final ClusterStatisticsMergeMode clusterStatisticsMergeMode; private final int statisticsMaxRetainedBytes; @@ -83,11 +86,14 @@ public CompletableFuture> submitFetcherTask( 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); @@ -99,7 +105,7 @@ public CompletableFuture> submitFetcherTask( * 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. */ - private CompletableFuture> inMemoryFullSketchMerging( + CompletableFuture> inMemoryFullSketchMerging( StageDefinition stageDefinition, List workerTaskIds ) @@ -122,10 +128,17 @@ private CompletableFuture> inMemoryFullSketchM stageDefinition.getId().getQueryId(), stageDefinition.getStageNumber() ); - partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true)); + partitionFuture.whenComplete((result, exception) -> { + if (exception != null || (result != null && result.isError())) { + snapshotFuture.cancel(true); + } + }); try { ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); + if (clusterByStatisticsSnapshot == null) { + throw new ISE("Worker %s returned empty sketch, this should never happen", workerNo); + } synchronized (mergedStatisticsCollector) { mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); finishedWorkers.add(workerNo); @@ -138,6 +151,7 @@ private CompletableFuture> inMemoryFullSketchM catch (Exception e) { synchronized (mergedStatisticsCollector) { partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); } } }); @@ -150,7 +164,7 @@ private CompletableFuture> inMemoryFullSketchM * This takes longer due to the overhead of fetching sketches, however, this prevents any loss in accuracy from * downsampling on the controller. */ - private CompletableFuture> sequentialTimeChunkMerging( + CompletableFuture> sequentialTimeChunkMerging( CompleteKeyStatisticsInformation completeKeyStatisticsInformation, StageDefinition stageDefinition, List workerTaskIds @@ -187,6 +201,21 @@ public SequentialFetchStage( this.partitionFuture = new CompletableFuture<>(); } + /** + * 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()) { @@ -212,10 +241,17 @@ public void submitFetchingTasksForNextTimeChunk() stageDefinition.getStageNumber(), timeChunk ); - partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true)); + 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 empty sketch for %s, this should never happen", workerNo, timeChunk); + } synchronized (mergedStatisticsCollector) { mergedStatisticsCollector.addAll(snapshotForTimeChunk); finishedWorkers.add(workerNo); @@ -246,6 +282,7 @@ public void submitFetchingTasksForNextTimeChunk() catch (Exception e) { synchronized (mergedStatisticsCollector) { partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); } } }); 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 8557a8ca3f24..8fa04ce6d90b 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 @@ -65,7 +65,7 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller) * for the client-side code that calls this API. */ @POST - @Path("/partialKeyStatistics/{queryId}/{stageNumber}/{workerNumber}") + @Path("/partialKeyStatisticsInformation/{queryId}/{stageNumber}/{workerNumber}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) public Response httpPostPartialKeyStatistics( @@ -77,7 +77,7 @@ public Response httpPostPartialKeyStatistics( ) { ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - controller.updatePartialKeyStatistics(stageNumber, workerNumber, partialKeyStatisticsObject); + controller.updatePartialKeyStatisticsInformation(stageNumber, workerNumber, partialKeyStatisticsObject); return Response.status(Response.Status.ACCEPTED).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 2b130c7ab599..1ff5952a8958 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 @@ -66,7 +66,7 @@ public void postPartialKeyStatistics( ) throws IOException { final String path = StringUtils.format( - "/partialKeyStatistics/%s/%d/%d", + "/partialKeyStatisticsInformation/%s/%d/%d", StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), workerNumber 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 332138914eeb..6416c52ede23 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,6 +49,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) }, // Waiting to fetch key statistics in the background from the workers and incrementally generate partitions. + // Transitioning to this phase should also enqueue the task to fetch key statistics to WorkerSketchFetcher. MERGING_STATISTICS { @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 54a6f940f9ad..bf53239335c5 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 @@ -180,7 +180,7 @@ boolean collectorEncounteredAnyMultiValueField() } else if (workersWithReportedKeyStatistics.size() != workerCount) { throw new ISE("Result key statistics are not ready"); } else { - return completeKeyStatisticsInformation.isHasMultipleValues(); + return completeKeyStatisticsInformation.hasMultipleValues(); } } @@ -248,7 +248,7 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( if (phase != ControllerStagePhase.READING_INPUT) { throw new ISE("Cannot add result key statistics from stage [%s]", phase); } - if (!stageDef.doesShuffle() || completeKeyStatisticsInformation == null) { + if (!stageDef.mustGatherResultKeyStatistics() || !stageDef.doesShuffle() || completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); } @@ -268,7 +268,7 @@ ControllerStagePhase addPartialKeyStatisticsForWorker( completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation); if (workersWithReportedKeyStatistics.size() == workerCount) { - // All workers have sent the report. + // All workers have sent the partial key statistics information. // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers. transitionTo(ControllerStagePhase.MERGING_STATISTICS); @@ -292,6 +292,14 @@ void setClusterByPartitionBoundaries(ClusterByPartitions clusterByPartitions) throw new ISE("Result partitions have already been generated"); } + if (!stageDef.mustGatherResultKeyStatistics()) { + throw new ISE("Result partitions does not require key statistics, should not have set partition boundries here"); + } + + if (!ControllerStagePhase.MERGING_STATISTICS.equals(getPhase())) { + throw new ISE("Cannot set partition boundires from key statistics from stage [%s]", getPhase()); + } + this.resultPartitionBoundaries = clusterByPartitions; this.resultPartitions = ReadablePartitions.striped( stageDef.getStageNumber(), 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 ac7c33694158..e364dedfcfc1 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,6 +19,8 @@ package org.apache.druid.msq.statistics; +import org.apache.curator.shaded.com.google.common.collect.ImmutableSortedMap; + import java.util.HashSet; import java.util.Set; import java.util.SortedMap; @@ -30,21 +32,28 @@ public class CompleteKeyStatisticsInformation { private final SortedMap> timeSegmentVsWorkerMap; - private boolean hasMultipleValues; + private boolean multipleValues; private double bytesRetained; public CompleteKeyStatisticsInformation( final SortedMap> timeChunks, - boolean hasMultipleValues, + boolean multipleValues, double bytesRetained ) { this.timeSegmentVsWorkerMap = timeChunks; - this.hasMultipleValues = hasMultipleValues; + this.multipleValues = multipleValues; this.bytesRetained = bytesRetained; } + /** + * Merges the {@link PartialKeyStatisticsInformation} into the complete key statistics information object. + * {@link #timeSegmentVsWorkerMap} is updated in sorted order with the timechunks from + * {@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}. + */ public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation) { for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) { @@ -52,18 +61,18 @@ public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInform .computeIfAbsent(timeSegment, key -> new HashSet<>()) .add(workerNumber); } - this.hasMultipleValues = this.hasMultipleValues || partialKeyStatisticsInformation.isHasMultipleValues(); + this.multipleValues = this.multipleValues || partialKeyStatisticsInformation.hasMultipleValues(); this.bytesRetained += bytesRetained; } public SortedMap> getTimeSegmentVsWorkerMap() { - return timeSegmentVsWorkerMap; + return ImmutableSortedMap.copyOfSorted(timeSegmentVsWorkerMap); } - public boolean isHasMultipleValues() + public boolean hasMultipleValues() { - return hasMultipleValues; + return multipleValues; } public double getBytesRetained() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java index 0040d6dcb4b1..535af8dafb0a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java @@ -31,19 +31,19 @@ public class PartialKeyStatisticsInformation { private final Set timeSegments; - private final boolean hasMultipleValues; + private final boolean multipleValues; private final double bytesRetained; @JsonCreator public PartialKeyStatisticsInformation( @JsonProperty("timeSegments") Set timeSegments, - @JsonProperty("hasMultipleValues") boolean hasMultipleValues, + @JsonProperty("multipleValues") boolean hasMultipleValues, @JsonProperty("bytesRetained") double bytesRetained ) { this.timeSegments = timeSegments; - this.hasMultipleValues = hasMultipleValues; + this.multipleValues = hasMultipleValues; this.bytesRetained = bytesRetained; } @@ -53,10 +53,10 @@ public Set getTimeSegments() return timeSegments; } - @JsonProperty("hasMultipleValues") - public boolean isHasMultipleValues() + @JsonProperty("multipleValues") + public boolean hasMultipleValues() { - return hasMultipleValues; + return multipleValues; } @JsonProperty("bytesRetained") 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 new file mode 100644 index 000000000000..98bb1b4c36a3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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 WorkerSketchFetcherTest +{ + @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/statistics/PartialKeyStatisticsInformationSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java index 213ababd231f..c23106c0299d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java @@ -56,7 +56,7 @@ public void testSerde() throws JsonProcessingException PartialKeyStatisticsInformation.class ); Assert.assertEquals(json, partialInformation.getTimeSegments(), deserializedKeyStatistics.getTimeSegments()); - Assert.assertEquals(json, partialInformation.isHasMultipleValues(), deserializedKeyStatistics.isHasMultipleValues()); + Assert.assertEquals(json, partialInformation.hasMultipleValues(), deserializedKeyStatistics.hasMultipleValues()); Assert.assertEquals(json, partialInformation.getBytesRetained(), deserializedKeyStatistics.getBytesRetained(), 0); } } 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 0f474b381195..06b0bae29690 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 @@ -47,7 +47,7 @@ public void postPartialKeyStatistics( ) { try { - controller.updatePartialKeyStatistics(stageId.getStageNumber(), workerNumber, partialKeyStatisticsInformation); + controller.updatePartialKeyStatisticsInformation(stageId.getStageNumber(), workerNumber, partialKeyStatisticsInformation); } catch (Exception e) { throw new ISE(e, "unable to post partial key statistics"); From 3efa5b6f89ecca761cca6a32229ea5c4e621035d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 17 Nov 2022 16:39:37 +0530 Subject: [PATCH 27/31] Add log messages and comments --- .../druid/msq/exec/WorkerSketchFetcher.java | 42 ++++++++++++++----- .../controller/ControllerStagePhase.java | 1 + .../msq/ITKeyStatisticsSketchMergeMode.java | 19 +++++++++ 3 files changed, 51 insertions(+), 11 deletions(-) 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 5999eaaf20a1..61ac85da941d 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 @@ -144,6 +144,7 @@ CompletableFuture> inMemoryFullSketchMerging( finishedWorkers.add(workerNo); if (finishedWorkers.size() == workerCount) { + log.debug("Query [%s] parallel mode. Received all statistics, generating partitions", stageDefinition.getId().getQueryId()); partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); } } @@ -231,6 +232,11 @@ public void submitFetchingTasksForNextTimeChunk() // 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) { executorService.submit(() -> { @@ -260,20 +266,21 @@ public void submitFetchingTasksForNextTimeChunk() Either longClusterByPartitionsEither = stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector); - if (longClusterByPartitionsEither.isError()) { - partitionFuture.complete(longClusterByPartitionsEither); - } - - List timeSketchPartitions = - stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector) - .valueOrThrow() - .ranges(); - abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchPartitions); + log.debug("Query [%s]. Received all statistics for time chunk %s, generating partitions", + stageDefinition.getId().getQueryId(), + timeChunk); - if (finalPartitionBoundries.size() > stageDefinition.getMaxPartitionCount()) { + 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((long) finalPartitionBoundries.size())); + partitionFuture.complete(Either.error(totalPartitionCount)); } 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(); } } @@ -316,4 +323,17 @@ 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/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index 6416c52ede23..896b345b4d22 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,6 +49,7 @@ 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. MERGING_STATISTICS { @Override 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 b1d1d7c77bc9..c0f22f2c762d 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 @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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; From 7514cee34c56da174c4a388c042d6fb8c95820e5 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 17 Nov 2022 21:12:19 +0530 Subject: [PATCH 28/31] Resolve build issues --- .../apache/druid/msq/exec/WorkerSketchFetcher.java | 12 ++++-------- .../statistics/CompleteKeyStatisticsInformation.java | 2 +- 2 files changed, 5 insertions(+), 9 deletions(-) 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 61ac85da941d..85823019dea7 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 @@ -150,10 +150,8 @@ CompletableFuture> inMemoryFullSketchMerging( } } catch (Exception e) { - synchronized (mergedStatisticsCollector) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); - } + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); } }); }); @@ -287,10 +285,8 @@ public void submitFetchingTasksForNextTimeChunk() } } catch (Exception e) { - synchronized (mergedStatisticsCollector) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); - } + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); } }); } 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 e364dedfcfc1..8a915fc8cd8e 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 org.apache.curator.shaded.com.google.common.collect.ImmutableSortedMap; +import com.google.common.collect.ImmutableSortedMap; import java.util.HashSet; import java.util.Set; From 08d5f9ce30e485ea1542c5c63b77050ee2dad05a Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 18 Nov 2022 13:09:20 +0530 Subject: [PATCH 29/31] Add unit tests --- .../druid/msq/exec/WorkerSketchFetcher.java | 6 +- .../exec/WorkerSketchFetcherAutoModeTest.java | 139 ++++++++++++++++ .../msq/exec/WorkerSketchFetcherTest.java | 152 ++++++++++-------- 3 files changed, 227 insertions(+), 70 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java 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 85823019dea7..0573f04b1938 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 @@ -137,14 +137,14 @@ CompletableFuture> inMemoryFullSketchMerging( try { ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); if (clusterByStatisticsSnapshot == null) { - throw new ISE("Worker %s returned empty sketch, this should never happen", workerNo); + 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] parallel mode. Received all statistics, generating partitions", stageDefinition.getId().getQueryId()); + log.debug("Query [%s] Received all statistics, generating partitions", stageDefinition.getId().getQueryId()); partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); } } @@ -254,7 +254,7 @@ public void submitFetchingTasksForNextTimeChunk() try { ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get(); if (snapshotForTimeChunk == null) { - throw new ISE("Worker %s returned empty sketch for %s, this should never happen", workerNo, timeChunk); + throw new ISE("Worker %s returned null sketch for %s, this should never happen", workerNo, timeChunk); } synchronized (mergedStatisticsCollector) { mergedStatisticsCollector.addAll(snapshotForTimeChunk); 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 new file mode 100644 index 000000000000..42f6f0437f59 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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 98bb1b4c36a3..9f2dc3a9a8c5 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 @@ -19,21 +19,37 @@ package org.apache.druid.msq.exec; +import com.google.common.collect.ImmutableList; +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.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; +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; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.Collections; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import static org.easymock.EasyMock.mock; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; 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; @@ -46,6 +62,10 @@ public class WorkerSketchFetcherTest private StageDefinition stageDefinition; @Mock private ClusterBy clusterBy; + @Mock + private ClusterByStatisticsCollector mergedClusterByStatisticsCollector; + @Mock + private WorkerClient workerClient; private AutoCloseable mocks; private WorkerSketchFetcher target; @@ -53,14 +73,11 @@ public class WorkerSketchFetcherTest 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()); + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); doReturn(clusterBy).when(stageDefinition).getClusterBy(); + doReturn(mergedClusterByStatisticsCollector).when(stageDefinition).createResultKeyStatisticsCollector(anyInt()); } @After @@ -70,70 +87,71 @@ public void tearDown() throws Exception } @Test - public void test_submitFetcherTask_belowThresholds_ShouldBeParallel() + public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCancelOtherTasks() { - // 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()); + // Store futures in a queue + final Queue> futureQueue = new ConcurrentLinkedQueue<>(); + final CountDownLatch latch = new CountDownLatch(5); + + // When fetching snapshots, return a mock and add future to queue + doAnswer(invocation -> { + ListenableFuture future = spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); + futureQueue.add(future); + latch.countDown(); + return future; + }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); + + // Cause a worker to fail instead of returning the result + doAnswer(invocation -> { + latch.countDown(); + return Futures.immediateFailedFuture(new InterruptedException("interrupted")); + }).when(workerClient).fetchClusterByStatisticsSnapshot(eq("2"), any(), anyInt()); + + CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( + completeKeyStatisticsInformation, + ImmutableList.of("1", "2", "3", "4", "5"), + stageDefinition + ); + + // Assert that the final result is failed and all other task futures are also cancelled. + Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); + for (ListenableFuture snapshotFuture : futureQueue) { + verify(snapshotFuture, times(1)).cancel(eq(true)); + } } @Test - public void test_submitFetcherTask_workerCountAboveThreshold_shouldBeSequential() + public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() + throws ExecutionException, InterruptedException { - // 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()); + // Store snapshots in a queue + final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); + final CountDownLatch latch = new CountDownLatch(5); + + final Either expectedPartitions = mock(Either.class); + doReturn(expectedPartitions).when(stageDefinition).generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector)); + + // 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, + ImmutableList.of("1", "2", "3", "4", "5"), + stageDefinition + ); + + // Assert that the final result is complete and all other sketches returned have been merged. + eitherCompletableFuture.join(); + Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); + for (ClusterByStatisticsSnapshot snapshot : snapshotQueue) { + verify(mergedClusterByStatisticsCollector, times(1)).addAll(eq(snapshot)); + } + Assert.assertEquals(expectedPartitions, eitherCompletableFuture.get()); } } From 4c7e2dbe65b91b70acca2d018600ac85a81f22e8 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 18 Nov 2022 19:15:49 +0530 Subject: [PATCH 30/31] Add unit tests --- .../druid/msq/exec/WorkerSketchFetcher.java | 13 +- .../msq/exec/WorkerSketchFetcherTest.java | 161 ++++++++++++++++-- 2 files changed, 155 insertions(+), 19 deletions(-) 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 0573f04b1938..3482b50daaff 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 @@ -150,8 +150,10 @@ CompletableFuture> inMemoryFullSketchMerging( } } catch (Exception e) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); + synchronized (mergedStatisticsCollector) { + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); + } } }); }); @@ -272,6 +274,7 @@ public void submitFetchingTasksForNextTimeChunk() 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); @@ -285,8 +288,10 @@ public void submitFetchingTasksForNextTimeChunk() } } catch (Exception e) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); + synchronized (mergedStatisticsCollector) { + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); + } } }); } 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 9f2dc3a9a8c5..3b35f16a1c58 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 @@ -20,10 +20,14 @@ package org.apache.druid.msq.exec; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.ImmutableSortedMap; 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.frame.key.RowKey; import org.apache.druid.java.util.common.Either; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; @@ -37,16 +41,22 @@ import org.mockito.Mock; 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; import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -63,9 +73,13 @@ public class WorkerSketchFetcherTest @Mock private ClusterBy clusterBy; @Mock - private ClusterByStatisticsCollector mergedClusterByStatisticsCollector; + private ClusterByStatisticsCollector mergedClusterByStatisticsCollector1; + @Mock + private ClusterByStatisticsCollector mergedClusterByStatisticsCollector2; @Mock private WorkerClient workerClient; + private ClusterByPartitions expectedPartitions1; + private ClusterByPartitions expectedPartitions2; private AutoCloseable mocks; private WorkerSketchFetcher target; @@ -73,11 +87,20 @@ public class WorkerSketchFetcherTest public void setUp() { mocks = MockitoAnnotations.openMocks(this); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); - doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); doReturn(clusterBy).when(stageDefinition).getClusterBy(); - doReturn(mergedClusterByStatisticsCollector).when(stageDefinition).createResultKeyStatisticsCollector(anyInt()); + 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( + mergedClusterByStatisticsCollector1, + mergedClusterByStatisticsCollector2 + ).when(stageDefinition).createResultKeyStatisticsCollector(anyInt()); } @After @@ -91,31 +114,41 @@ public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCan { // Store futures in a queue final Queue> futureQueue = new ConcurrentLinkedQueue<>(); - final CountDownLatch latch = new CountDownLatch(5); + 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)); // When fetching snapshots, return a mock and add future to queue doAnswer(invocation -> { - ListenableFuture future = spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); - futureQueue.add(future); + ListenableFuture snapshotListenableFuture = + spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); + futureQueue.add(snapshotListenableFuture); latch.countDown(); - return future; + latch.await(); + return snapshotListenableFuture; }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); // Cause a worker to fail instead of returning the result doAnswer(invocation -> { latch.countDown(); + latch.await(); return Futures.immediateFailedFuture(new InterruptedException("interrupted")); }).when(workerClient).fetchClusterByStatisticsSnapshot(eq("2"), any(), anyInt()); CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( completeKeyStatisticsInformation, - ImmutableList.of("1", "2", "3", "4", "5"), + workerIds, stageDefinition ); // Assert that the final result is failed and all other task futures are also cancelled. Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); 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. + Assert.assertFalse(futureQueue.isEmpty()); for (ListenableFuture snapshotFuture : futureQueue) { verify(snapshotFuture, times(1)).cancel(eq(true)); } @@ -127,10 +160,10 @@ public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() { // Store snapshots in a queue final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); - final CountDownLatch latch = new CountDownLatch(5); + final List workerIds = ImmutableList.of("0", "1", "2", "3", "4"); + final CountDownLatch latch = new CountDownLatch(workerIds.size()); - final Either expectedPartitions = mock(Either.class); - doReturn(expectedPartitions).when(stageDefinition).generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector)); + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); // When fetching snapshots, return a mock and add it to queue doAnswer(invocation -> { @@ -142,16 +175,114 @@ public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( completeKeyStatisticsInformation, - ImmutableList.of("1", "2", "3", "4", "5"), + workerIds, stageDefinition ); // Assert that the final result is complete and all other sketches returned have been merged. eitherCompletableFuture.join(); Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); + Assert.assertFalse(snapshotQueue.isEmpty()); + // Verify that all statistics were added to controller. for (ClusterByStatisticsSnapshot snapshot : snapshotQueue) { - verify(mergedClusterByStatisticsCollector, times(1)).addAll(eq(snapshot)); + verify(mergedClusterByStatisticsCollector1, times(1)).addAll(eq(snapshot)); } - Assert.assertEquals(expectedPartitions, eitherCompletableFuture.get()); + // Check that the partitions returned by the merged collector is returned by the final future. + Assert.assertEquals(expectedPartitions1, eitherCompletableFuture.get().valueOrThrow()); + } + + @Test + public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldCancelOtherTasks() + { + // 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)); + doReturn(timeSegmentVsWorkerMap).when(completeKeyStatisticsInformation).getTimeSegmentVsWorkerMap(); + + final CyclicBarrier barrier = new CyclicBarrier(3); + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); + + // When fetching snapshots, return a mock and add future to queue + doAnswer(invocation -> { + ListenableFuture snapshotListenableFuture = + spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); + futureQueue.add(snapshotListenableFuture); + barrier.await(); + return snapshotListenableFuture; + }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(anyString(), anyString(), anyInt(), anyLong()); + + // Cause a worker in the second time chunk to fail instead of returning the result + doAnswer(invocation -> { + barrier.await(); + 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 + ); + + // Assert that the final result is failed and all other task futures are also cancelled. + Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + 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(); + // Verify that other task futures were requested to be cancelled. + Assert.assertFalse(futureQueue.isEmpty()); + for (ListenableFuture snapshotFuture : futureQueue) { + verify(snapshotFuture, times(1)).cancel(eq(true)); + } + } + + @Test + public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() + throws ExecutionException, 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 CyclicBarrier barrier = new CyclicBarrier(3); + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); + + // 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 + ); + + // Assert that the final result is complete and all other sketches returned have been merged. + eitherCompletableFuture.join(); + 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()); } } From 837ed3d35c00aec85bd1e8c91ff03feeebbaf01d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 21 Nov 2022 14:25:02 +0530 Subject: [PATCH 31/31] Fix timing issue in tests --- .../druid/msq/exec/WorkerSketchFetcherTest.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 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 3b35f16a1c58..54c9a792e558 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 @@ -110,7 +110,7 @@ public void tearDown() throws Exception } @Test - public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCancelOtherTasks() + public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCancelOtherTasks() throws Exception { // Store futures in a queue final Queue> futureQueue = new ConcurrentLinkedQueue<>(); @@ -144,6 +144,8 @@ public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCan // Assert that the final result is failed and all other task futures are also cancelled. Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + Thread.sleep(1000); + Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); // Verify that the statistics collector was cleared due to the error. verify(mergedClusterByStatisticsCollector1, times(1)).clear(); @@ -181,6 +183,7 @@ public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() // 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. @@ -192,7 +195,7 @@ public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() } @Test - public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldCancelOtherTasks() + public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldCancelOtherTasks() throws Exception { // Store futures in a queue final Queue> futureQueue = new ConcurrentLinkedQueue<>(); @@ -226,6 +229,8 @@ public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldC // Assert that the final result is failed and all other task futures are also cancelled. Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + Thread.sleep(1000); + Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); // Verify that the correct statistics collector was cleared due to the error. verify(mergedClusterByStatisticsCollector1, times(0)).clear(); @@ -266,6 +271,8 @@ public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() // 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.