cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1021360261
##########
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)
Review Comment:
We can easily UT this I think
##########
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
Review Comment:
Curious to understand how auto works here. Maybe a comment here would be
useful.
##########
integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java:
##########
@@ -122,4 +126,144 @@ 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<String, Object> context = ImmutableMap.of(
+ MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE,
+ ClusterStatisticsMergeMode.PARALLEL
Review Comment:
Do we need tests for auto mode ?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -227,19 +221,28 @@ WorkerInputs getWorkerInputs()
return workerInputs;
}
+ /**
+ * Returns the merged key statistics.
+ */
+ @Nullable
+ public WorkerAggregatedKeyStatistics getAggregatedKeyStatistics()
+ {
+ return aggregatedKeyStatistics;
+ }
+
/**
* 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 aggregatedKeyStatistics aggregated key statistics
*/
- ControllerStagePhase addResultKeyStatisticsForWorker(
+ ControllerStagePhase addAggregatedStatisticsForWorker(
final int workerNumber,
- final ClusterByStatisticsSnapshot snapshot
+ final WorkerAggregatedKeyStatistics aggregatedKeyStatistics
Review Comment:
PartialStageKeyStats? seems more apt no ?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -590,7 +599,34 @@ public void updateStatus(int stageNumber, int
workerNumber, Object keyStatistics
);
}
- queryKernel.addResultKeyStatisticsForStageAndWorker(stageId,
workerNumber, keyStatistics);
+ queryKernel.addResultStatisticsReportForStageAndWorker(stageId,
workerNumber, aggregatedKeyStatistics);
Review Comment:
addAggregateKeyStatisticsForStageAndWorker ?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -252,14 +255,17 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
}
try {
- if (workersWithResultKeyStatistics.add(workerNumber)) {
- resultKeyStatisticsCollector.addAll(snapshot);
+ if (workersWithFinishedReport.add(workerNumber)) {
+ aggregatedKeyStatistics.addAll(aggregatedKeyStatistics);
- 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()) {
Review Comment:
Lets throw an exception if stage mustgatherResultStats is false.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java:
##########
@@ -38,6 +40,24 @@ public interface WorkerClient extends AutoCloseable
*/
ListenableFuture<Void> 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;
+
+ /**
+ * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the
sketch of the specified timeChunk.
+ * This is intended to be used by the {@link WorkerSketchFetcher}.
Review Comment:
Lets document the mode as well
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -590,7 +599,34 @@ public void updateStatus(int stageNumber, int
workerNumber, Object keyStatistics
);
}
- queryKernel.addResultKeyStatisticsForStageAndWorker(stageId,
workerNumber, keyStatistics);
+ queryKernel.addResultStatisticsReportForStageAndWorker(stageId,
workerNumber, aggregatedKeyStatistics);
+
+ if
(queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS))
{
+ List<String> workerTaskIds = workerTaskLauncher.getTaskList();
+ WorkerAggregatedKeyStatistics mergedKeyStatistics =
queryKernel.getAggregatedKeyStatistics(stageId);
+
+ // Queue the sketch fetching task into the worker sketch fetcher.
+ CompletableFuture<Either<Long, ClusterByPartitions>>
clusterByPartitionsCompletableFuture =
+ workerSketchFetcher.submitFetcherTask(
+ mergedKeyStatistics,
+ 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());
Review Comment:
All these operations are required to be added to kernalManipulationQueue so
that they are linearized.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java:
##########
@@ -55,6 +57,24 @@ public ListenableFuture<Void> 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
+ {
+ return client.fetchClusterByStatisticsSnapshot(workerTaskId, queryId,
stageNumber);
+ }
+
+ @Override
+ public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(
Review Comment:
fetchClusterByStatisticsSnapshotForTimeChunk ?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.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;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics;
+
+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.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * 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 long BYTES_THRESHOLD = 1_000_000_000L;
+ private static final long WORKER_THRESHOLD = 100;
+
+ private final ClusterStatisticsMergeMode clusterStatisticsMergeMode;
+ private final WorkerClient workerClient;
+ private final ExecutorService executorService;
+
+ public WorkerSketchFetcher(WorkerClient workerClient,
ClusterStatisticsMergeMode clusterStatisticsMergeMode)
+ {
+ this.workerClient = workerClient;
+ this.clusterStatisticsMergeMode = clusterStatisticsMergeMode;
+ this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT);
+ }
+
+ /**
+ * 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<Either<Long, ClusterByPartitions>>
submitFetcherTask(
+ WorkerAggregatedKeyStatistics aggregatedKeyStatistics,
+ List<String> workerTaskIds,
+ StageDefinition stageDefinition
+ )
+ {
+ ClusterBy clusterBy = stageDefinition.getClusterBy();
+
+ switch (clusterStatisticsMergeMode) {
+ case SEQUENTIAL:
+ return sequentialTimeChunkMerging(aggregatedKeyStatistics,
stageDefinition, workerTaskIds);
+ case PARALLEL:
+ return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+ case AUTO:
+ if (clusterBy.getBucketByCount() == 0) {
Review Comment:
Is this an exception?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.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;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics;
+
+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.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * 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 long BYTES_THRESHOLD = 1_000_000_000L;
+ private static final long WORKER_THRESHOLD = 100;
+
+ private final ClusterStatisticsMergeMode clusterStatisticsMergeMode;
+ private final WorkerClient workerClient;
+ private final ExecutorService executorService;
+
+ public WorkerSketchFetcher(WorkerClient workerClient,
ClusterStatisticsMergeMode clusterStatisticsMergeMode)
+ {
+ this.workerClient = workerClient;
+ this.clusterStatisticsMergeMode = clusterStatisticsMergeMode;
+ this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT);
+ }
+
+ /**
+ * 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<Either<Long, ClusterByPartitions>>
submitFetcherTask(
+ WorkerAggregatedKeyStatistics aggregatedKeyStatistics,
+ List<String> workerTaskIds,
+ StageDefinition stageDefinition
+ )
+ {
+ ClusterBy clusterBy = stageDefinition.getClusterBy();
+
+ 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);
+ }
+ }
+
+ /**
+ * 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<Either<Long, ClusterByPartitions>>
inMemoryFullSketchMerging(
+ StageDefinition stageDefinition,
+ List<String> workerTaskIds
+ )
+ {
+ CompletableFuture<Either<Long, ClusterByPartitions>> partitionFuture = new
CompletableFuture<>();
+
+ final ClusterByStatisticsCollector mergedStatisticsCollector =
stageDefinition.createResultKeyStatisticsCollector();
+ final int workerCount = workerTaskIds.size();
+ final Set<Integer> 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;
+ }
+
+ /**
+ * 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<Either<Long, ClusterByPartitions>>
sequentialTimeChunkMerging(
+ WorkerAggregatedKeyStatistics aggregatedKeyStatistics,
+ StageDefinition stageDefinition,
+ List<String> workerTaskIds
+ )
+ {
+ SequentialFetchStage sequentialFetchStage = new SequentialFetchStage(
+ stageDefinition,
+ workerTaskIds,
+
aggregatedKeyStatistics.getTimeSegmentVsWorkerIdMap().entrySet().iterator()
+ );
+ sequentialFetchStage.submitFetchingTasksForNextTimeChunk();
+ return sequentialFetchStage.getPartitionFuture();
+ }
+
+ private class SequentialFetchStage
+ {
+ private final StageDefinition stageDefinition;
+ private final List<String> workerTaskIds;
+ private final Iterator<Map.Entry<Long, Set<Integer>>>
timeSegmentVsWorkerIdIterator;
+ private final CompletableFuture<Either<Long, ClusterByPartitions>>
partitionFuture;
+ private final List<ClusterByPartition> finalPartitionBoundries;
+
+ public SequentialFetchStage(
+ StageDefinition stageDefinition,
+ List<String> workerTaskIds,
+ Iterator<Map.Entry<Long, Set<Integer>>> timeSegmentVsWorkerIdIterator
+ )
+ {
+ this.finalPartitionBoundries = new ArrayList<>();
+ this.stageDefinition = stageDefinition;
+ this.workerTaskIds = workerTaskIds;
+ this.timeSegmentVsWorkerIdIterator = timeSegmentVsWorkerIdIterator;
+ this.partitionFuture = new CompletableFuture<>();
+ }
+
+ public void submitFetchingTasksForNextTimeChunk()
+ {
+ if (!timeSegmentVsWorkerIdIterator.hasNext()) {
+ partitionFuture.complete(Either.value(new
ClusterByPartitions(finalPartitionBoundries)));
+ } else {
+ Map.Entry<Long, Set<Integer>> entry =
timeSegmentVsWorkerIdIterator.next();
+ Long timeChunk = entry.getKey();
+ Set<Integer> workerIdsWithTimeChunk = entry.getValue();
+ ClusterByStatisticsCollector mergedStatisticsCollector =
stageDefinition.createResultKeyStatisticsCollector();
+ Set<Integer> finishedWorkers = new HashSet<>();
+
+ for (int workerNo : workerIdsWithTimeChunk) {
+ executorService.submit(() -> {
+ 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()) {
Review Comment:
can this be stale?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -526,6 +533,30 @@ public void postFinish()
kernelManipulationQueue.add(KernelHolder::setDone);
}
+ @Override
+ public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+ throws ExecutionException, InterruptedException
+ {
+ CompletableFuture<ClusterByStatisticsSnapshot> future = new
CompletableFuture<>();
+ kernelManipulationQueue.add(kernelHolder -> {
+
future.complete(kernelHolder.stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot());
+ });
+ return future.get();
+ }
+
+ @Override
+ public ClusterByStatisticsSnapshot fetchSingletonStatisticsSnapshot(StageId
stageId, long timeChunk)
+ throws ExecutionException, InterruptedException
+ {
+ CompletableFuture<ClusterByStatisticsSnapshot> future = new
CompletableFuture<>();
+ kernelManipulationQueue.add(kernelHolder -> {
+ ClusterByStatisticsSnapshot snapshot =
kernelHolder.stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();
+ ClusterByStatisticsSnapshot singletonSnapshot =
snapshot.getSingletonSnapshot(timeChunk);
+ future.complete(singletonSnapshot);
+ });
+ return future.get();
Review Comment:
Extract the stage, keyStats map outside the kernel, and make it thread-safe?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.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;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics;
+
+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.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Queues up fetching sketches from workers and progressively generates
partitions boundaries.
+ */
+public class WorkerSketchFetcher
+{
+ private static final int DEFAULT_THREAD_COUNT = 10;
Review Comment:
good catch. Yeah, we need to limit the number of parallel threads here. Lets
start with 4 ?
##########
integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java:
##########
@@ -122,4 +126,144 @@ public void testMsqIngestionAndQuerying() throws Exception
msqHelper.testQueriesFromFile(QUERY_FILE, datasource);
}
+
+ @Test
+ public void testMsqIngestionParallelMerging() throws Exception
Review Comment:
Let's create a new test class for these tests no ?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java:
##########
@@ -67,6 +69,17 @@
*/
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
Review Comment:
Please also document which mode is this API getting used in.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -252,14 +255,17 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
}
try {
- if (workersWithResultKeyStatistics.add(workerNumber)) {
- resultKeyStatisticsCollector.addAll(snapshot);
+ if (workersWithFinishedReport.add(workerNumber)) {
Review Comment:
Lets rename accordingly when `PartialStageKeyStats` is used
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.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;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.WorkerAggregatedKeyStatistics;
+
+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.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * 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 long BYTES_THRESHOLD = 1_000_000_000L;
+ private static final long WORKER_THRESHOLD = 100;
+
+ private final ClusterStatisticsMergeMode clusterStatisticsMergeMode;
+ private final WorkerClient workerClient;
+ private final ExecutorService executorService;
+
+ public WorkerSketchFetcher(WorkerClient workerClient,
ClusterStatisticsMergeMode clusterStatisticsMergeMode)
+ {
+ this.workerClient = workerClient;
+ this.clusterStatisticsMergeMode = clusterStatisticsMergeMode;
+ this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT);
+ }
+
+ /**
+ * 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<Either<Long, ClusterByPartitions>>
submitFetcherTask(
+ WorkerAggregatedKeyStatistics aggregatedKeyStatistics,
+ List<String> workerTaskIds,
+ StageDefinition stageDefinition
+ )
+ {
+ ClusterBy clusterBy = stageDefinition.getClusterBy();
+
+ 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);
+ }
+ }
+
+ /**
+ * 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<Either<Long, ClusterByPartitions>>
inMemoryFullSketchMerging(
+ StageDefinition stageDefinition,
+ List<String> workerTaskIds
+ )
+ {
+ CompletableFuture<Either<Long, ClusterByPartitions>> partitionFuture = new
CompletableFuture<>();
+
+ final ClusterByStatisticsCollector mergedStatisticsCollector =
stageDefinition.createResultKeyStatisticsCollector();
+ final int workerCount = workerTaskIds.size();
+ final Set<Integer> finishedWorkers = new HashSet<>();
+
+ for (int i = 0; i < workerCount; i++) {
Review Comment:
IntStream might help
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]