LakshSingla commented on code in PR #13205: URL: https://github.com/apache/druid/pull/13205#discussion_r1014973452
########## 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<br /><br />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` | Review Comment: I think we should either explain all the modes here or omit the explanation of the `AUTO` mode. Would it also be possible to briefly explain the tradeoffs between the modes in this Javadoc? We can also consider having a separate section for this merge mode if it warrants. Also, the wording seems a bit off. ```suggestion | clusterStatisticsMergeMode | Whether to use parallel or sequential mode for merging of the worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. `AUTO` mode tries to find the best approach based on number of workers and size of input rows. | `AUTO` | ``` ########## extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java: ########## @@ -516,6 +520,9 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); + ClusterStatisticsMergeMode clusterStatisticsMergeMode = MultiStageQueryContext.getClusterStatisticsMergeMode(task.getSqlQueryContext()); Review Comment: I think we should be logging the merge mode somewhere when we are initializing the state. ########## 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: nit: We can rename i to `workerNumber` which would eliminate the requirement of the line below. ########## 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; Review Comment: Should we explain these thresholds in the comments? ########## 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: I am wondering if there is an easier way to extract the information out of the lambda holder. Instead of going through the kernelManipulationQueue, can we directly extract out the information from the kernelHolder (since it seems that the lambda call isn't blocking anywhere for the singletonSnapshot?) Would there be any race condition in my suggestion? ########## 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<>(); Review Comment: Should the implementation be a ConcurrentHashSet since it will be accessed from multiple threads? I see that we have synchronized its access under `mergedStatisticsCollector`. If that's the case we should add a comment here that the access/modification of the set should be guarded by that object for someone making changes in the future. ########## 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. Review Comment: nit ```suggestion * Mode which dictates how {@link WorkerSketchFetcher} gets sketches for the partition boundaries from workers. ``` ########## 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<>(); Review Comment: Similar comment as the one left above ########## 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()) { + return; + } + synchronized (mergedStatisticsCollector) { + mergedStatisticsCollector.addAll(singletonStatisticsSnapshot); + finishedWorkers.add(workerNo); + + if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) { + Either<Long, ClusterByPartitions> longClusterByPartitionsEither = + stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector); + + if (longClusterByPartitionsEither.isError()) { + partitionFuture.complete(longClusterByPartitionsEither); + } + + List<ClusterByPartition> timeSketchPartitions = + stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector) + .valueOrThrow() + .ranges(); + abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchPartitions); + + submitFetchingTasksForNextTimeChunk(); + } + } + } + catch (Exception e) { + partitionFuture.completeExceptionally(e); + } + }); + } + } + } + + private void abutAndAppendPartitionBoundries( Review Comment: nit: Please add a javadoc for this method. Are there any preconditions on the arguments of this method for it to work as intended? (I am thinking something like `timeSketchPartitions` should be ordered s.t they are in an 'ascending' order. Also what if there's a gap in between the `timeSketchPartitions`'s boundaries? If that is disallowed then maybe we should add that as well). -- 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: commits-unsubscr...@druid.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org For additional commands, e-mail: commits-h...@druid.apache.org