adarshsanjeev commented on code in PR #13205: URL: https://github.com/apache/druid/pull/13205#discussion_r1024801541
########## extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java: ########## @@ -0,0 +1,282 @@ +/* + * 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 com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; +import 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.CompleteKeyStatisticsInformation; + +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; +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 = 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; + // If there are more workers than this threshold, SEQUENTIAL merging mode is used. + 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, int statisticsMaxRetainedBytes) + { + this.workerClient = workerClient; + this.clusterStatisticsMergeMode = clusterStatisticsMergeMode; + this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); + this.statisticsMaxRetainedBytes = statisticsMaxRetainedBytes; + } + + /** + * 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( + CompleteKeyStatisticsInformation completeKeyStatisticsInformation, + List<String> workerTaskIds, + StageDefinition stageDefinition + ) + { + ClusterBy clusterBy = stageDefinition.getClusterBy(); + + switch (clusterStatisticsMergeMode) { + case SEQUENTIAL: + return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); + case PARALLEL: + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + case AUTO: + if (clusterBy.getBucketByCount() == 0) { + // 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); + } + 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<>(); + + // Create a new key statistics collector to merge worker sketches into + final ClusterByStatisticsCollector mergedStatisticsCollector = + stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); + final int workerCount = workerTaskIds.size(); + // Guarded by synchronized mergedStatisticsCollector + final Set<Integer> finishedWorkers = new HashSet<>(); + + // Submit a task for each worker to fetch statistics + IntStream.range(0, workerCount).forEach(workerNo -> { + executorService.submit(() -> { + ListenableFuture<ClusterByStatisticsSnapshot> snapshotFuture = + workerClient.fetchClusterByStatisticsSnapshot( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber() + ); + partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true)); + + try { + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); + synchronized (mergedStatisticsCollector) { + mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); + finishedWorkers.add(workerNo); + + if (finishedWorkers.size() == workerCount) { + partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); + } + } + } + catch (Exception e) { + synchronized (mergedStatisticsCollector) { Review Comment: Would it not help update the future cancellation across the next thread instead of letting it happen? I also added a sketch.clear() here now on exception. -- 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]
