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


Reply via email to