You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/10/11 02:32:32 UTC

[GitHub] [druid] adarshsanjeev opened a new pull request, #13205: Add sequential sketch merging to MSQ

adarshsanjeev opened a new pull request, #13205:
URL: https://github.com/apache/druid/pull/13205

   <!-- Thanks for trying to help us make Apache Druid be the best it can be! Please fill out as much of the following information as is possible (where relevant, and remove it when irrelevant) to help make the intention and scope of this PR clear in order to ease review. -->
   
   <!-- Please read the doc for contribution (https://github.com/apache/druid/blob/master/CONTRIBUTING.md) before making this PR. Also, once you open a PR, please _avoid using force pushes and rebasing_ since these make it difficult for reviewers to see what you've changed in response to their reviews. See [the 'If your pull request shows conflicts with master' section](https://github.com/apache/druid/blob/master/CONTRIBUTING.md#if-your-pull-request-shows-conflicts-with-master) for more details. -->
   
   Fixes #XXXX.
   
   <!-- Replace XXXX with the id of the issue fixed in this PR. Remove this section if there is no corresponding issue. Don't reference the issue in the title of this pull-request. -->
   
   <!-- If you are a committer, follow the PR action item checklist for committers:
   https://github.com/apache/druid/blob/master/dev/committer-instructions.md#pr-and-issue-action-item-checklist-for-committers. -->
   
   ### Description
   
   <!-- Describe the goal of this PR, what problem are you fixing. If there is a corresponding issue (referenced above), it's not necessary to repeat the description here, however, you may choose to keep one summary sentence. -->
   
   <!-- Describe your patch: what did you change in code? How did you fix the problem? -->
   
   <!-- If there are several relatively logically separate changes in this PR, create a mini-section for each of them. For example: -->
   
   #### Fixed the bug ...
   #### Renamed the class ...
   #### Added a forbidden-apis entry ...
   
   <!--
   In each section, please describe design decisions made, including:
    - Choice of algorithms
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error conditions handled, such as when there are insufficient resources?
    - Class organization and design (how the logic is split between classes, inheritance, composition, design patterns)
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
   -->
   
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design (or naming) decision point and compare the alternatives with the designs that you've implemented (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), link to that discussion from this PR description and explain what have changed in your final design compared to your original proposal or the consensus version in the end of the discussion. If something hasn't changed since the original discussion, you can omit a detailed discussion of those aspects of the design here, perhaps apart from brief mentioning for the sake of readability of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. -->
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `MyFoo`
    * `OurBar`
    * `TheirBaz`
   
   <hr>
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022526025


##########
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:
   Done! Thanks for the tip.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1026004506


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -122,22 +128,31 @@ private CompletableFuture<Either<Long, ClusterByPartitions>> inMemoryFullSketchM
                 stageDefinition.getId().getQueryId(),
                 stageDefinition.getStageNumber()
             );
-        partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true));
+        partitionFuture.whenComplete((result, exception) -> {
+          if (exception != null || (result != null && result.isError())) {
+            snapshotFuture.cancel(true);
+          }

Review Comment:
   Oops. Spot on. I somehow though the future always completes on error. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1021458464


##########
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:
   Thanks for the explanation. I utilize this as well so I don't have a cleaner way! 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1021457604


##########
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:
   Ohh okay, I think this should be cool in that case. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1021328482


##########
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:
   Going through this again, with regards to memory usage, we should limit the threads in relation to the number of sketches the controller can maintain at a time in memory.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024014702


##########
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:
   Moved key stats map



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023556180


##########
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:
   Added



##########
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:
   Added a comment for this



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024748548


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -259,16 +257,21 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     }
 
     try {
-      if (workersWithResultKeyStatistics.add(workerNumber)) {
-        resultKeyStatisticsCollector.addAll(snapshot);
+      if (workersWithReportedKeyStatistics.add(workerNumber)) {
 
-        if (workersWithResultKeyStatistics.size() == workerCount) {
-          generateResultPartitionsAndBoundaries();
+        if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) {

Review Comment:
   Yes, testInsertNullTimestamp



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1021437766


##########
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:
   Thanks for the explanation. I generally do it using this method as well, so I don't have any suggestions. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1005257998


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class sent by worker to controller after reading input to generate partition boundries.

Review Comment:
   It could also be done in that way on the controller, which would have to do the same for all workers sequentially. Would the smaller size of the payload be a better approach?



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on PR #13205:
URL: https://github.com/apache/druid/pull/13205#issuecomment-1323033816

   Thanks for the contribution @adarshsanjeev. 


-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1021334721


##########
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:
   The line below is only to make the variable effectively final to use inside the lambda function. Is there a cleaner way to do this?



##########
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 think this is possible, but the kernelHolder is a local variable in runTask. Is there a way to use it directly here?



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024017918


##########
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 workerTaskId, WorkOrder workOrder);
 
+  /**
+   * Fetches the {@link ClusterByStatisticsSnapshot} from a worker. This is intended to be used by the
+   * {@link WorkerSketchFetcher} under PARALLEL or AUTO modes.
+   */
+  ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber)
+      throws ExecutionException, InterruptedException;
+
+  /**
+   * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk.
+   * This is intended to be used by the {@link WorkerSketchFetcher} under SEQUENTIAL or AUTO modes.
+   */
+  ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk(

Review Comment:
   Thanks for the suggestion!
   Changed APIs to be non blocking, and optimised merging so that ongoing tasks are cancelled if an exception is encountered. Please let me know if any further optimisation can be done here. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022394626


##########
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:
   Done



##########
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:
   Changed



##########
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:
   Changed



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class sent by worker to controller after reading input to generate partition boundries.

Review Comment:
   Changed



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022525492


##########
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:
   Added logging



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023702717


##########
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:
   I think its okay if we are working on the stale value. I would actually make this call before line 195 as well since we donot want the extra http call. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024972790


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -562,6 +568,19 @@ public void postFinish()
     kernelManipulationQueue.add(KernelHolder::setDone);
   }
 
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+  {
+    return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();
+  }
+
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk)
+  {
+    ClusterByStatisticsSnapshot snapshot = stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();

Review Comment:
   Added a check that the future does not return null in the WorkerSketchFetcher



##########
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);

Review Comment:
   Added



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025060077


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -234,23 +224,31 @@ WorkerInputs getWorkerInputs()
     return workerInputs;
   }
 
+  /**
+   * Returns the merged key statistics.
+   */
+  @Nullable
+  public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation()
+  {
+    return completeKeyStatisticsInformation;
+  }
+
   /**
    * 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 partialKeyStatisticsInformation partial key statistics
    */
-  ControllerStagePhase addResultKeyStatisticsForWorker(
+  ControllerStagePhase addPartialKeyStatisticsForWorker(
       final int workerNumber,
-      final ClusterByStatisticsSnapshot snapshot
+      final PartialKeyStatisticsInformation partialKeyStatisticsInformation
   )
   {
     if (phase != ControllerStagePhase.READING_INPUT) {
       throw new ISE("Cannot add result key statistics from stage [%s]", phase);
     }
-
-    if (resultKeyStatisticsCollector == null) {
+    if (!stageDef.doesShuffle() || completeKeyStatisticsInformation == null) {

Review Comment:
   Added a check



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1005418398


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java:
##########
@@ -59,6 +59,7 @@
   private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true;
 
   public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage";
+  public static final String CTX_FORCE_NON_SEQUENTIAL_MERGE = "forceNonSequentialMerging";

Review Comment:
   Done



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024760092


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -397,15 +418,11 @@ private void generateResultPartitionsAndBoundaries()
    *
    * @param fault reason why this stage has failed
    */
-  private void failForReason(final MSQFault fault)
+  void failForReason(final MSQFault fault)
   {
     transitionTo(ControllerStagePhase.FAILED);
 
     this.failureReason = fault;
-
-    if (resultKeyStatisticsCollector != null) {

Review Comment:
   resultKeyStatisticsCollector has been moved from controller stage tracker, there shouldn't be any large memory object to clear here. Added clearing sketch to worker sketch fetcher.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024979403


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker.
+ */
+public class CompleteKeyStatisticsInformation
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap;
+
+  private boolean hasMultipleValues;
+
+  private double bytesRetained;
+
+  public CompleteKeyStatisticsInformation(
+      final SortedMap<Long, Set<Integer>> timeChunks,
+      boolean hasMultipleValues,
+      double bytesRetained
+  )
+  {
+    this.timeSegmentVsWorkerMap = timeChunks;
+    this.hasMultipleValues = hasMultipleValues;
+    this.bytesRetained = bytesRetained;
+  }
+
+  public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation)
+  {
+    for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) {
+      this.timeSegmentVsWorkerMap
+          .computeIfAbsent(timeSegment, key -> new HashSet<>())
+          .add(workerNumber);
+    }
+    this.hasMultipleValues = this.hasMultipleValues || partialKeyStatisticsInformation.isHasMultipleValues();
+    this.bytesRetained += bytesRetained;
+  }
+
+  public SortedMap<Long, Set<Integer>> getTimeSegmentVsWorkerMap()
+  {
+    return timeSegmentVsWorkerMap;

Review Comment:
   Changed



##########
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) {
+            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(
+      CompleteKeyStatisticsInformation completeKeyStatisticsInformation,
+      StageDefinition stageDefinition,
+      List<String> workerTaskIds
+  )
+  {
+    SequentialFetchStage sequentialFetchStage = new SequentialFetchStage(
+        stageDefinition,
+        workerTaskIds,
+        completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().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;
+    // Final sorted list of partition boundaries. This is appended to after statistics for each time chunk are gathered.
+    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()

Review Comment:
   Added a javadoc here



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025073031


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -259,16 +257,21 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     }
 
     try {
-      if (workersWithResultKeyStatistics.add(workerNumber)) {
-        resultKeyStatisticsCollector.addAll(snapshot);
+      if (workersWithReportedKeyStatistics.add(workerNumber)) {
 
-        if (workersWithResultKeyStatistics.size() == workerCount) {
-          generateResultPartitionsAndBoundaries();
+        if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) {
+          // Time should not contain null value
+          failForReason(InsertTimeNullFault.instance());
+          return getPhase();
+        }
+
+        completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation);
+
+        if (workersWithReportedKeyStatistics.size() == workerCount) {
+          // All workers have sent the report.
+          // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers.
+          transitionTo(ControllerStagePhase.MERGING_STATISTICS);

Review Comment:
   Added comment



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024975091


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker.
+ */
+public class CompleteKeyStatisticsInformation
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap;
+
+  private boolean hasMultipleValues;
+
+  private double bytesRetained;
+
+  public CompleteKeyStatisticsInformation(
+      final SortedMap<Long, Set<Integer>> timeChunks,
+      boolean hasMultipleValues,
+      double bytesRetained
+  )
+  {
+    this.timeSegmentVsWorkerMap = timeChunks;
+    this.hasMultipleValues = hasMultipleValues;
+    this.bytesRetained = bytesRetained;
+  }
+
+  public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation)
+  {
+    for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) {
+      this.timeSegmentVsWorkerMap
+          .computeIfAbsent(timeSegment, key -> new HashSet<>())
+          .add(workerNumber);
+    }
+    this.hasMultipleValues = this.hasMultipleValues || partialKeyStatisticsInformation.isHasMultipleValues();
+    this.bytesRetained += bytesRetained;
+  }
+
+  public SortedMap<Long, Set<Integer>> getTimeSegmentVsWorkerMap()
+  {
+    return timeSegmentVsWorkerMap;
+  }
+
+  public boolean isHasMultipleValues()

Review Comment:
   Done



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024971819


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -565,10 +579,12 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer)
   }
 
   /**
-   * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key
+   * statistics information has been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate
+   * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}.
    */
   @Override
-  public void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject)
+  public void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject)

Review Comment:
   Changed



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -595,7 +611,35 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
             );
           }
 
-          queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+          queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation);
+
+          if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) {
+            List<String> workerTaskIds = workerTaskLauncher.getTaskList();
+            CompleteKeyStatisticsInformation completeKeyStatisticsInformation =
+                queryKernel.getCompleteKeyStatisticsInformation(stageId);
+
+            // Queue the sketch fetching task into the worker sketch fetcher.
+            CompletableFuture<Either<Long, ClusterByPartitions>> clusterByPartitionsCompletableFuture =
+                workerSketchFetcher.submitFetcherTask(
+                    completeKeyStatisticsInformation,
+                    workerTaskIds,
+                    stageDef
+                );
+
+            // Add the listener to handle completion.
+            clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> {
+              kernelManipulationQueue.add(holder -> {
+                if (throwable != null) {
+                  queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable));
+                } else if (clusterByPartitionsEither.isError()) {
+                  queryKernel.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount()));
+                } else {
+                  queryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow());
+                }
+                holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId));
+              });
+            });
+          }

Review Comment:
   Added a log message



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1006663425


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class sent by worker to controller after reading input to generate partition boundries.

Review Comment:
    I think we require workerID on the controller as all workers will not have all the time chunks.
    
    But instead of using a the same class at the controller, and for the worker payload change the add method to take another appropriate payload class ?
    WDYT ?
    
    



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022526351


##########
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:
   Added



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023557977


##########
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:
   Added



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023556054


##########
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:
   Done



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024976005


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -259,16 +257,21 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     }
 
     try {
-      if (workersWithResultKeyStatistics.add(workerNumber)) {
-        resultKeyStatisticsCollector.addAll(snapshot);
+      if (workersWithReportedKeyStatistics.add(workerNumber)) {
 
-        if (workersWithResultKeyStatistics.size() == workerCount) {
-          generateResultPartitionsAndBoundaries();
+        if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) {
+          // Time should not contain null value
+          failForReason(InsertTimeNullFault.instance());
+          return getPhase();
+        }
+
+        completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation);
+
+        if (workersWithReportedKeyStatistics.size() == workerCount) {
+          // All workers have sent the report.

Review Comment:
   Changed. This one instance seems to have escaped all the text searches I did.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022519900


##########
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:
   This case happens when clusterBy is none(), for partitioning by all time
    https://github.com/apache/druid/blob/master/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java#L66



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022435808


##########
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:
   Changed to PartialKeyStatisticsInformation



##########
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:
   Changed to PartialKeyStatisticsInformation. Is this fine?



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023638488


##########
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:
   Moved this to inside the kernalManipulationQueue



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024002459


##########
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:
   Done



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1004264481


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java:
##########
@@ -59,6 +59,7 @@
   private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true;
 
   public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage";
+  public static final String CTX_FORCE_NON_SEQUENTIAL_MERGE = "forceNonSequentialMerging";

Review Comment:
   Some suggestions : 
   1. Let's document the context parameter in the PR description. 
   2. Let's add it in the SQL readme docs. 
   3. Should we call the parameter segment size estimator mode: and have 2 values for it, serial, and parallel? In the future, we can have an auto mode that automatically switches the size estimator based on the inputs. 
   wdyt ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java:
##########
@@ -105,4 +107,15 @@ boolean postResultPartitionBoundaries(
    * Called when the work required for the query has been finished
    */
   void postFinish();
+
+  /**
+   * Returns the statistics snapshot for the given stageId
+   */
+  ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) throws ExecutionException, InterruptedException;

Review Comment:
   Nit: Lets rearrange the new methods to below postWorkOrder. 
   It's generally nicer to have the last method as a finish. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java:
##########
@@ -81,9 +81,9 @@ public String getId()
   // Worker-to-controller messages
 
   /**
-   * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Accepts a {@link ClusterByStatisticsWorkerReport} for generating fetching sketches.

Review Comment:
   We might want to add some details to the javadocs of this method. A good way to write this would be 
   1. What does this api do ?
   2. Who uses this api?  



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java:
##########
@@ -81,9 +81,9 @@ public String getId()
   // Worker-to-controller messages
 
   /**
-   * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Accepts a {@link ClusterByStatisticsWorkerReport} for generating fetching sketches.
    */
-  void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject);
+  void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerReport);

Review Comment:
   ```suggestion
     void updateWorkerReportStatus(int stageNumber, int workerNumber, Object workerStatisticsReport);
   ```



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java:
##########
@@ -34,13 +34,13 @@
 public interface ControllerClient extends AutoCloseable
 {
   /**
-   * Client side method to update the controller with key statistics for a particular stage and worker.
-   * Controller's implementation collates all the key statistics for a stage to generate the partition boundaries.
+   * Client side method to update the controller with worker reports for a particular stage and worker.

Review Comment:
   We already have a worker report. Its a bit confusing as to which report are you referring to. 
   Suggest changing the name to workerAggregatedKeyStatistics



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.ClusterByStatisticsWorkerReport;
+
+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 boolean forceNonSequentialMerging;
+  private final WorkerClient workerClient;
+  private final ExecutorService executorService;
+
+  public WorkerSketchFetcher(WorkerClient workerClient, boolean forceNonSequentialMerging)
+  {
+    this.workerClient = workerClient;
+    this.forceNonSequentialMerging = forceNonSequentialMerging;
+    this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT);
+  }
+
+  /**
+   * Submits a request to fetch and generate partitions for the given worker report and returns a future for it. It
+   * decides based on the report if it should fetch sketches one by one or together.
+   */
+  public CompletableFuture<Either<Long, ClusterByPartitions>> submitFetcherTask(
+      ClusterByStatisticsWorkerReport workerReport,
+      List<String> workerTaskIds,
+      StageDefinition stageDefinition
+  )
+  {
+    ClusterBy clusterBy = stageDefinition.getClusterBy();
+
+    if (forceNonSequentialMerging || clusterBy.getBucketByCount() == 0) {
+      return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+    } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || workerReport.getBytesRetained() > BYTES_THRESHOLD) {
+      return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);

Review Comment:
   shouldn't this be `sequentialTimeChunkMerging` ? since we have a lot of workers or bytes?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java:
##########
@@ -315,13 +316,18 @@ public ClusterByStatisticsSnapshot snapshot()
   {
     assertRetainedByteCountsAreTrackedCorrectly();
 
-    final List<ClusterByStatisticsSnapshot.Bucket> bucketSnapshots = new ArrayList<>();
+    final Map<Long, ClusterByStatisticsSnapshot.Bucket> bucketSnapshots = new HashMap<>();
+    final RowKeyReader trimmedRowReader = keyReader.trimmedKeyReader(clusterBy.getBucketByCount());
 
     for (final Map.Entry<RowKey, BucketHolder> bucketEntry : buckets.entrySet()) {
       //noinspection rawtypes, unchecked
       final KeyCollectorSnapshot keyCollectorSnapshot =
           ((KeyCollectorFactory) keyCollectorFactory).toSnapshot(bucketEntry.getValue().keyCollector);
-      bucketSnapshots.add(new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot));
+      Long bucketKey = Long.MIN_VALUE;
+      if (clusterBy.getBucketByCount() == 1) {

Review Comment:
   Nit: It would help if you could provide a comment here as to why this is required.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class sent by worker to controller after reading input to generate partition boundries.
+ */
+public class ClusterByStatisticsWorkerReport
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerIdMap;
+
+  private Boolean hasMultipleValues;

Review Comment:
   is there a reason this is not `boolean`



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class sent by worker to controller after reading input to generate partition boundries.

Review Comment:
   If each worker is sending this class in the payload then why does it need to send a redundant worker ID ?
   {
   "2022-01-01":[1],
   "2022-01-02":[1]
   }
   
   Maybe I am missing something 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024760092


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -397,15 +418,11 @@ private void generateResultPartitionsAndBoundaries()
    *
    * @param fault reason why this stage has failed
    */
-  private void failForReason(final MSQFault fault)
+  void failForReason(final MSQFault fault)
   {
     transitionTo(ControllerStagePhase.FAILED);
 
     this.failureReason = fault;
-
-    if (resultKeyStatisticsCollector != null) {

Review Comment:
   resultKeyStatisticsCollector has been moved from controller stage tracker, there shouldn't be any large memory object to clear here.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024974874


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java:
##########
@@ -58,24 +59,25 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller)
   }
 
   /**
-   * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Used by subtasks to post {@link PartialKeyStatisticsInformation} for shuffling stages.
    *
-   * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API.
+   * See {@link ControllerClient#postPartialKeyStatistics(StageId, int, PartialKeyStatisticsInformation)}
+   * for the client-side code that calls this API.
    */
   @POST
-  @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}")
+  @Path("/partialKeyStatistics/{queryId}/{stageNumber}/{workerNumber}")

Review Comment:
   Changed
   



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025072847


##########
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));

Review Comment:
   Changed behaviour to this. 
   Ideally, it should have not completed successfully unless snapshotFuture has already finished, but changed to avoid cancelling tasks without error.



##########
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) {
+            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(
+      CompleteKeyStatisticsInformation completeKeyStatisticsInformation,
+      StageDefinition stageDefinition,
+      List<String> workerTaskIds
+  )
+  {
+    SequentialFetchStage sequentialFetchStage = new SequentialFetchStage(
+        stageDefinition,
+        workerTaskIds,
+        completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().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;
+    // Final sorted list of partition boundaries. This is appended to after statistics for each time chunk are gathered.
+    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();
+        // Time chunk for which partition boundries are going to be generated for
+        Long timeChunk = entry.getKey();
+        Set<Integer> workerIdsWithTimeChunk = entry.getValue();
+        // Create a new key statistics collector to merge worker sketches into
+        ClusterByStatisticsCollector mergedStatisticsCollector =
+            stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes);
+        // Guarded by synchronized mergedStatisticsCollector
+        Set<Integer> finishedWorkers = new HashSet<>();
+
+        // Submits a task for every worker which has a certain time chunk
+        for (int workerNo : workerIdsWithTimeChunk) {
+          executorService.submit(() -> {
+            ListenableFuture<ClusterByStatisticsSnapshot> snapshotFuture =
+                workerClient.fetchClusterByStatisticsSnapshotForTimeChunk(
+                    workerTaskIds.get(workerNo),
+                    stageDefinition.getId().getQueryId(),
+                    stageDefinition.getStageNumber(),
+                    timeChunk
+                );
+            partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true));

Review Comment:
   Changed behaviour to this. 
   Ideally, it should have not completed successfully unless snapshotFuture has already finished, but changed to avoid cancelling tasks without error.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1021437766


##########
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:
   Thanks for the explanation. I generally do it using this method as well, so I don't have any suggestions. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025073648


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -280,6 +283,25 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     return getPhase();
   }
 
+  /**
+   * Sets the {@link #resultPartitions} and {@link #resultPartitionBoundaries} and transitions the phase to POST_READING.
+   */
+  void setClusterByPartitionBoundaries(ClusterByPartitions clusterByPartitions)
+  {
+    if (resultPartitions != null) {

Review Comment:
   Added checks for both



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1026458056


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -0,0 +1,335 @@
+/*
+ * 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.java.util.common.ISE;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+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 Logger log = new Logger(WorkerSketchFetcher.class);
+  private static final int DEFAULT_THREAD_COUNT = 4;
+  // If the combined size of worker sketches is more than this threshold, SEQUENTIAL merging mode is used.
+  static final long BYTES_THRESHOLD = 1_000_000_000L;
+  // If there are more workers than this threshold, SEQUENTIAL merging mode is used.
+  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) {
+          log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId());
+          // If there is no time clustering, there is no scope for sequential merge
+          return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+        } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || completeKeyStatisticsInformation.getBytesRetained() > BYTES_THRESHOLD) {
+          log.debug("Query [%s] AUTO mode: chose SEQUENTIAL mode to merge key statistics", stageDefinition.getId().getQueryId());
+          return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds);
+        }
+        log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId());
+        return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+      default:
+        throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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) -> {
+          if (exception != null || (result != null && result.isError())) {
+            snapshotFuture.cancel(true);
+          }
+        });
+
+        try {
+          ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get();
+          if (clusterByStatisticsSnapshot == null) {
+            throw new ISE("Worker %s returned null sketch, this should never happen", workerNo);
+          }
+          synchronized (mergedStatisticsCollector) {
+            mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot);
+            finishedWorkers.add(workerNo);
+
+            if (finishedWorkers.size() == workerCount) {
+              log.debug("Query [%s] Received all statistics, generating partitions", stageDefinition.getId().getQueryId());
+              partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector));
+            }
+          }
+        }
+        catch (Exception e) {
+          partitionFuture.completeExceptionally(e);
+          mergedStatisticsCollector.clear();

Review Comment:
   Changed



##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.util.Collections;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class WorkerSketchFetcherTest
+{
+  @Mock
+  private CompleteKeyStatisticsInformation completeKeyStatisticsInformation;
+  @Mock
+  private StageDefinition stageDefinition;
+  @Mock
+  private ClusterBy clusterBy;
+  private AutoCloseable mocks;
+  private WorkerSketchFetcher target;
+
+  @Before
+  public void setUp()

Review Comment:
   Added



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024973059


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -595,7 +611,35 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
             );
           }
 
-          queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+          queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation);
+
+          if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) {
+            List<String> workerTaskIds = workerTaskLauncher.getTaskList();
+            CompleteKeyStatisticsInformation completeKeyStatisticsInformation =
+                queryKernel.getCompleteKeyStatisticsInformation(stageId);
+
+            // Queue the sketch fetching task into the worker sketch fetcher.
+            CompletableFuture<Either<Long, ClusterByPartitions>> clusterByPartitionsCompletableFuture =
+                workerSketchFetcher.submitFetcherTask(
+                    completeKeyStatisticsInformation,
+                    workerTaskIds,
+                    stageDef
+                );
+
+            // Add the listener to handle completion.
+            clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> {
+              kernelManipulationQueue.add(holder -> {
+                if (throwable != null) {
+                  queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable));

Review Comment:
   Changed



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -595,7 +611,35 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
             );
           }
 
-          queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+          queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation);
+
+          if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) {
+            List<String> workerTaskIds = workerTaskLauncher.getTaskList();
+            CompleteKeyStatisticsInformation completeKeyStatisticsInformation =
+                queryKernel.getCompleteKeyStatisticsInformation(stageId);
+
+            // Queue the sketch fetching task into the worker sketch fetcher.
+            CompletableFuture<Either<Long, ClusterByPartitions>> clusterByPartitionsCompletableFuture =
+                workerSketchFetcher.submitFetcherTask(
+                    completeKeyStatisticsInformation,
+                    workerTaskIds,
+                    stageDef
+                );
+
+            // Add the listener to handle completion.
+            clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> {
+              kernelManipulationQueue.add(holder -> {
+                if (throwable != null) {
+                  queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable));

Review Comment:
   Changed



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1006663425


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsWorkerReport.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class sent by worker to controller after reading input to generate partition boundries.

Review Comment:
    I think we require workerID on the controller as all workers will not have all the time chunks.
    
    But instead of using the same class at the controller, and for the worker payload, change the add method to take another appropriate payload class ?
    WDYT ?
    
    



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024033929


##########
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:
   An IT test for AUTO mode would test either PARALLEL or SEQUENTIAL, which are already tested. Would a unit test that checks the thresholds for AUTO be better?



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023638079


##########
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:
   Added a comment to make it more clear



##########
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:
   Added a comment to make it more clear



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023638717


##########
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:
   Changed to 4 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022525297


##########
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:
   Changed



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1022607933


##########
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:
   Done.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024727883


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -562,6 +568,19 @@ public void postFinish()
     kernelManipulationQueue.add(KernelHolder::setDone);
   }
 
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+  {
+    return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();

Review Comment:
   The key statistics should be populated for the worker report to be sent which then might call this, so this should never be the case.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023556506


##########
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:
   Added a separate section for this



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025361291


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -122,22 +128,31 @@ private CompletableFuture<Either<Long, ClusterByPartitions>> inMemoryFullSketchM
                 stageDefinition.getId().getQueryId(),
                 stageDefinition.getStageNumber()
             );
-        partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true));
+        partitionFuture.whenComplete((result, exception) -> {
+          if (exception != null || (result != null && result.isError())) {
+            snapshotFuture.cancel(true);
+          }

Review Comment:
   Wouldn't this also be called with non null result if the future successfully returns later, which shouldn't be an ISE?



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1026393929


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -0,0 +1,335 @@
+/*
+ * 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.java.util.common.ISE;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+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 Logger log = new Logger(WorkerSketchFetcher.class);
+  private static final int DEFAULT_THREAD_COUNT = 4;
+  // If the combined size of worker sketches is more than this threshold, SEQUENTIAL merging mode is used.
+  static final long BYTES_THRESHOLD = 1_000_000_000L;
+  // If there are more workers than this threshold, SEQUENTIAL merging mode is used.
+  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) {
+          log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId());
+          // If there is no time clustering, there is no scope for sequential merge
+          return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+        } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || completeKeyStatisticsInformation.getBytesRetained() > BYTES_THRESHOLD) {
+          log.debug("Query [%s] AUTO mode: chose SEQUENTIAL mode to merge key statistics", stageDefinition.getId().getQueryId());
+          return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds);
+        }
+        log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId());
+        return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+      default:
+        throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode);
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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) -> {
+          if (exception != null || (result != null && result.isError())) {
+            snapshotFuture.cancel(true);
+          }
+        });
+
+        try {
+          ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get();
+          if (clusterByStatisticsSnapshot == null) {
+            throw new ISE("Worker %s returned null sketch, this should never happen", workerNo);
+          }
+          synchronized (mergedStatisticsCollector) {
+            mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot);
+            finishedWorkers.add(workerNo);
+
+            if (finishedWorkers.size() == workerCount) {
+              log.debug("Query [%s] Received all statistics, generating partitions", stageDefinition.getId().getQueryId());
+              partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector));
+            }
+          }
+        }
+        catch (Exception e) {
+          partitionFuture.completeExceptionally(e);
+          mergedStatisticsCollector.clear();

Review Comment:
   This line should be in the synchronized block. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024971007


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java:
##########
@@ -81,9 +81,11 @@ public String getId()
   // Worker-to-controller messages
 
   /**
-   * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key
+   * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate partiton boundaries.
+   * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}.
    */
-  void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject);
+  void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject);

Review Comment:
   Renamed method to updatePartialKeyStatisticsInformation



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024972408


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -562,6 +568,19 @@ public void postFinish()
     kernelManipulationQueue.add(KernelHolder::setDone);
   }
 
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+  {
+    return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();

Review Comment:
   Added a check that the future does not return null in the WorkerSketchFetcher



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1005197776


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java:
##########
@@ -81,9 +81,9 @@ public String getId()
   // Worker-to-controller messages
 
   /**
-   * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Accepts a {@link ClusterByStatisticsWorkerReport} for generating fetching sketches.

Review Comment:
   Updated



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java:
##########
@@ -34,13 +34,13 @@
 public interface ControllerClient extends AutoCloseable
 {
   /**
-   * Client side method to update the controller with key statistics for a particular stage and worker.
-   * Controller's implementation collates all the key statistics for a stage to generate the partition boundaries.
+   * Client side method to update the controller with worker reports for a particular stage and worker.

Review Comment:
   Changed



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024229108


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -595,7 +611,35 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
             );
           }
 
-          queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+          queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation);
+
+          if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) {
+            List<String> workerTaskIds = workerTaskLauncher.getTaskList();
+            CompleteKeyStatisticsInformation completeKeyStatisticsInformation =
+                queryKernel.getCompleteKeyStatisticsInformation(stageId);
+
+            // Queue the sketch fetching task into the worker sketch fetcher.
+            CompletableFuture<Either<Long, ClusterByPartitions>> clusterByPartitionsCompletableFuture =
+                workerSketchFetcher.submitFetcherTask(
+                    completeKeyStatisticsInformation,
+                    workerTaskIds,
+                    stageDef
+                );
+
+            // Add the listener to handle completion.
+            clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> {
+              kernelManipulationQueue.add(holder -> {
+                if (throwable != null) {
+                  queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable));
+                } else if (clusterByPartitionsEither.isError()) {
+                  queryKernel.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount()));
+                } else {
+                  queryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow());
+                }
+                holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId));
+              });
+            });
+          }

Review Comment:
   lets debug log the else part as well ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -562,6 +568,19 @@ public void postFinish()
     kernelManipulationQueue.add(KernelHolder::setDone);
   }
 
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+  {
+    return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();

Review Comment:
   What happens if the stage does not have result key stats. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java:
##########
@@ -81,9 +81,11 @@ public String getId()
   // Worker-to-controller messages
 
   /**
-   * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key
+   * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate partiton boundaries.
+   * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}.
    */
-  void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject);
+  void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject);

Review Comment:
   Lets rename the variable to partialKeyStatisticsInformation



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -595,7 +611,35 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
             );
           }
 
-          queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+          queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation);
+
+          if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) {
+            List<String> workerTaskIds = workerTaskLauncher.getTaskList();
+            CompleteKeyStatisticsInformation completeKeyStatisticsInformation =
+                queryKernel.getCompleteKeyStatisticsInformation(stageId);
+
+            // Queue the sketch fetching task into the worker sketch fetcher.
+            CompletableFuture<Either<Long, ClusterByPartitions>> clusterByPartitionsCompletableFuture =
+                workerSketchFetcher.submitFetcherTask(
+                    completeKeyStatisticsInformation,
+                    workerTaskIds,
+                    stageDef
+                );
+
+            // Add the listener to handle completion.
+            clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> {
+              kernelManipulationQueue.add(holder -> {
+                if (throwable != null) {
+                  queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable));

Review Comment:
   Also there is a method called addToKernelManipulationQueue which may be used. 



##########
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);

Review Comment:
   Lets debug log the mode eventually chosen here?



##########
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));

Review Comment:
   I think the logic here should be if the partition future completes exceptionally, cancel the correct work and not cancel the work everytime the partitionFuture gets completed. Wdyt ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker.
+ */
+public class CompleteKeyStatisticsInformation
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap;
+
+  private boolean hasMultipleValues;
+
+  private double bytesRetained;
+
+  public CompleteKeyStatisticsInformation(
+      final SortedMap<Long, Set<Integer>> timeChunks,
+      boolean hasMultipleValues,
+      double bytesRetained
+  )
+  {
+    this.timeSegmentVsWorkerMap = timeChunks;
+    this.hasMultipleValues = hasMultipleValues;
+    this.bytesRetained = bytesRetained;
+  }
+
+  public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation)

Review Comment:
   I think we should java doc this especially the sorted key part here SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -259,16 +257,21 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     }
 
     try {
-      if (workersWithResultKeyStatistics.add(workerNumber)) {
-        resultKeyStatisticsCollector.addAll(snapshot);
+      if (workersWithReportedKeyStatistics.add(workerNumber)) {
 
-        if (workersWithResultKeyStatistics.size() == workerCount) {
-          generateResultPartitionsAndBoundaries();
+        if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) {

Review Comment:
   Is there an associated UT for this ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -234,23 +224,31 @@ WorkerInputs getWorkerInputs()
     return workerInputs;
   }
 
+  /**
+   * Returns the merged key statistics.
+   */
+  @Nullable
+  public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation()
+  {
+    return completeKeyStatisticsInformation;
+  }
+
   /**
    * 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 partialKeyStatisticsInformation partial key statistics
    */
-  ControllerStagePhase addResultKeyStatisticsForWorker(
+  ControllerStagePhase addPartialKeyStatisticsForWorker(
       final int workerNumber,
-      final ClusterByStatisticsSnapshot snapshot
+      final PartialKeyStatisticsInformation partialKeyStatisticsInformation
   )
   {
     if (phase != ControllerStagePhase.READING_INPUT) {
       throw new ISE("Cannot add result key statistics from stage [%s]", phase);
     }
-
-    if (resultKeyStatisticsCollector == null) {
+    if (!stageDef.doesShuffle() || completeKeyStatisticsInformation == null) {

Review Comment:
   mustGatherResultKeyStatisticss should be checked as we sometimes add an empty shuffle spec.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -259,16 +257,21 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     }
 
     try {
-      if (workersWithResultKeyStatistics.add(workerNumber)) {
-        resultKeyStatisticsCollector.addAll(snapshot);
+      if (workersWithReportedKeyStatistics.add(workerNumber)) {
 
-        if (workersWithResultKeyStatistics.size() == workerCount) {
-          generateResultPartitionsAndBoundaries();
+        if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) {
+          // Time should not contain null value
+          failForReason(InsertTimeNullFault.instance());
+          return getPhase();
+        }
+
+        completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation);
+
+        if (workersWithReportedKeyStatistics.size() == workerCount) {
+          // All workers have sent the report.
+          // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers.
+          transitionTo(ControllerStagePhase.MERGING_STATISTICS);

Review Comment:
   We should mention similar comments on 271:272 in the JAVA doc of ControllerStagePhase.MERGING_STATISTICS



##########
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) {
+            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(
+      CompleteKeyStatisticsInformation completeKeyStatisticsInformation,
+      StageDefinition stageDefinition,
+      List<String> workerTaskIds
+  )
+  {
+    SequentialFetchStage sequentialFetchStage = new SequentialFetchStage(
+        stageDefinition,
+        workerTaskIds,
+        completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().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;
+    // Final sorted list of partition boundaries. This is appended to after statistics for each time chunk are gathered.
+    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()

Review Comment:
   Lets java doc this method.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -562,6 +568,19 @@ public void postFinish()
     kernelManipulationQueue.add(KernelHolder::setDone);
   }
 
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+  {
+    return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();

Review Comment:
   Or the result key stats are not yet populated.



##########
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:
   Sure as long we test that the correct mode is engaged



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -562,6 +568,19 @@ public void postFinish()
     kernelManipulationQueue.add(KernelHolder::setDone);
   }
 
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+  {
+    return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();
+  }
+
+  @Override
+  public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk)
+  {
+    ClusterByStatisticsSnapshot snapshot = stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();

Review Comment:
   Similar Q here.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -595,7 +611,35 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
             );
           }
 
-          queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+          queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation);
+
+          if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) {
+            List<String> workerTaskIds = workerTaskLauncher.getTaskList();
+            CompleteKeyStatisticsInformation completeKeyStatisticsInformation =
+                queryKernel.getCompleteKeyStatisticsInformation(stageId);
+
+            // Queue the sketch fetching task into the worker sketch fetcher.
+            CompletableFuture<Either<Long, ClusterByPartitions>> clusterByPartitionsCompletableFuture =
+                workerSketchFetcher.submitFetcherTask(
+                    completeKeyStatisticsInformation,
+                    workerTaskIds,
+                    stageDef
+                );
+
+            // Add the listener to handle completion.
+            clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> {
+              kernelManipulationQueue.add(holder -> {
+                if (throwable != null) {
+                  queryKernel.failStageForReason(stageId, UnknownFault.forException(throwable));

Review Comment:
   this should be the holder object. I donot know if it matters but sake of pattern we are using. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java:
##########
@@ -58,24 +59,25 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller)
   }
 
   /**
-   * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Used by subtasks to post {@link PartialKeyStatisticsInformation} for shuffling stages.
    *
-   * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API.
+   * See {@link ControllerClient#postPartialKeyStatistics(StageId, int, PartialKeyStatisticsInformation)}
+   * for the client-side code that calls this API.
    */
   @POST
-  @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}")
+  @Path("/partialKeyStatistics/{queryId}/{stageNumber}/{workerNumber}")

Review Comment:
   nit : I would change this to : partialKeyStatisticsInformation



##########
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:
   I could not reason about this lock. 
   



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -565,10 +579,12 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer)
   }
 
   /**
-   * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key
+   * statistics information has been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate
+   * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}.
    */
   @Override
-  public void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject)
+  public void updatePartialKeyStatistics(int stageNumber, int workerNumber, Object partialKeyStatisticsObject)

Review Comment:
   nit: partialKeyStatisticsInfObject?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker.
+ */
+public class CompleteKeyStatisticsInformation
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap;
+
+  private boolean hasMultipleValues;
+
+  private double bytesRetained;
+
+  public CompleteKeyStatisticsInformation(
+      final SortedMap<Long, Set<Integer>> timeChunks,
+      boolean hasMultipleValues,
+      double bytesRetained
+  )
+  {
+    this.timeSegmentVsWorkerMap = timeChunks;
+    this.hasMultipleValues = hasMultipleValues;
+    this.bytesRetained = bytesRetained;
+  }
+
+  public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation)
+  {
+    for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) {
+      this.timeSegmentVsWorkerMap
+          .computeIfAbsent(timeSegment, key -> new HashSet<>())
+          .add(workerNumber);
+    }
+    this.hasMultipleValues = this.hasMultipleValues || partialKeyStatisticsInformation.isHasMultipleValues();
+    this.bytesRetained += bytesRetained;
+  }
+
+  public SortedMap<Long, Set<Integer>> getTimeSegmentVsWorkerMap()
+  {
+    return timeSegmentVsWorkerMap;
+  }
+
+  public boolean isHasMultipleValues()

Review Comment:
   nit: this seems weird. Can we do hasMultipleValues()?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -397,15 +418,11 @@ private void generateResultPartitionsAndBoundaries()
    *
    * @param fault reason why this stage has failed
    */
-  private void failForReason(final MSQFault fault)
+  void failForReason(final MSQFault fault)
   {
     transitionTo(ControllerStagePhase.FAILED);
 
     this.failureReason = fault;
-
-    if (resultKeyStatisticsCollector != null) {

Review Comment:
   is there a reason we are reverting this change ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker.
+ */
+public class CompleteKeyStatisticsInformation
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap;
+
+  private boolean hasMultipleValues;

Review Comment:
   Nit: we can remove the has from this variable name and then add it to the function name 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -280,6 +283,25 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     return getPhase();
   }
 
+  /**
+   * Sets the {@link #resultPartitions} and {@link #resultPartitionBoundaries} and transitions the phase to POST_READING.
+   */
+  void setClusterByPartitionBoundaries(ClusterByPartitions clusterByPartitions)
+  {
+    if (resultPartitions != null) {

Review Comment:
   Lets do a state check that the current state should be MergingStats?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -259,16 +257,21 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     }
 
     try {
-      if (workersWithResultKeyStatistics.add(workerNumber)) {
-        resultKeyStatisticsCollector.addAll(snapshot);
+      if (workersWithReportedKeyStatistics.add(workerNumber)) {
 
-        if (workersWithResultKeyStatistics.size() == workerCount) {
-          generateResultPartitionsAndBoundaries();
+        if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) {
+          // Time should not contain null value
+          failForReason(InsertTimeNullFault.instance());
+          return getPhase();
+        }
+
+        completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation);
+
+        if (workersWithReportedKeyStatistics.size() == workerCount) {
+          // All workers have sent the report.

Review Comment:
   ```suggestion
             // All workers have sent the partial key statistics information.
   ```



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java:
##########
@@ -280,6 +283,25 @@ ControllerStagePhase addResultKeyStatisticsForWorker(
     return getPhase();
   }
 
+  /**
+   * Sets the {@link #resultPartitions} and {@link #resultPartitionBoundaries} and transitions the phase to POST_READING.
+   */
+  void setClusterByPartitionBoundaries(ClusterByPartitions clusterByPartitions)
+  {
+    if (resultPartitions != null) {

Review Comment:
   And also if this stage.needsKeyStats() 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java:
##########
@@ -58,24 +59,25 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller)
   }
 
   /**
-   * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Used by subtasks to post {@link PartialKeyStatisticsInformation} for shuffling stages.
    *
-   * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API.
+   * See {@link ControllerClient#postPartialKeyStatistics(StageId, int, PartialKeyStatisticsInformation)}
+   * for the client-side code that calls this API.
    */
   @POST
-  @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}")
+  @Path("/partialKeyStatistics/{queryId}/{stageNumber}/{workerNumber}")

Review Comment:
   Also, we would break backward compatibility here so it's worth mentioning that in the release notes.



##########
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) {
+            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(
+      CompleteKeyStatisticsInformation completeKeyStatisticsInformation,
+      StageDefinition stageDefinition,
+      List<String> workerTaskIds
+  )
+  {
+    SequentialFetchStage sequentialFetchStage = new SequentialFetchStage(
+        stageDefinition,
+        workerTaskIds,
+        completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().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;
+    // Final sorted list of partition boundaries. This is appended to after statistics for each time chunk are gathered.
+    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();
+        // Time chunk for which partition boundries are going to be generated for
+        Long timeChunk = entry.getKey();
+        Set<Integer> workerIdsWithTimeChunk = entry.getValue();
+        // Create a new key statistics collector to merge worker sketches into
+        ClusterByStatisticsCollector mergedStatisticsCollector =
+            stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes);
+        // Guarded by synchronized mergedStatisticsCollector
+        Set<Integer> finishedWorkers = new HashSet<>();
+
+        // Submits a task for every worker which has a certain time chunk
+        for (int workerNo : workerIdsWithTimeChunk) {
+          executorService.submit(() -> {
+            ListenableFuture<ClusterByStatisticsSnapshot> snapshotFuture =
+                workerClient.fetchClusterByStatisticsSnapshotForTimeChunk(
+                    workerTaskIds.get(workerNo),
+                    stageDefinition.getId().getQueryId(),
+                    stageDefinition.getStageNumber(),
+                    timeChunk
+                );
+            partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true));

Review Comment:
   Similar comment as above.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker.
+ */
+public class CompleteKeyStatisticsInformation
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap;
+
+  private boolean hasMultipleValues;
+
+  private double bytesRetained;
+
+  public CompleteKeyStatisticsInformation(
+      final SortedMap<Long, Set<Integer>> timeChunks,
+      boolean hasMultipleValues,
+      double bytesRetained
+  )
+  {
+    this.timeSegmentVsWorkerMap = timeChunks;
+    this.hasMultipleValues = hasMultipleValues;
+    this.bytesRetained = bytesRetained;
+  }
+
+  public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation)
+  {
+    for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) {
+      this.timeSegmentVsWorkerMap
+          .computeIfAbsent(timeSegment, key -> new HashSet<>())
+          .add(workerNumber);
+    }
+    this.hasMultipleValues = this.hasMultipleValues || partialKeyStatisticsInformation.isHasMultipleValues();
+    this.bytesRetained += bytesRetained;
+  }
+
+  public SortedMap<Long, Set<Integer>> getTimeSegmentVsWorkerMap()
+  {
+    return timeSegmentVsWorkerMap;

Review Comment:
   Should this be an immutable copy ?



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024969944


##########
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:
   Created a new test class



##########
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:
   Added a unit test for this



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025073432


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java:
##########
@@ -58,24 +59,25 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller)
   }
 
   /**
-   * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages.
+   * Used by subtasks to post {@link PartialKeyStatisticsInformation} for shuffling stages.
    *
-   * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API.
+   * See {@link ControllerClient#postPartialKeyStatistics(StageId, int, PartialKeyStatisticsInformation)}
+   * for the client-side code that calls this API.
    */
   @POST
-  @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}")
+  @Path("/partialKeyStatistics/{queryId}/{stageNumber}/{workerNumber}")

Review Comment:
   Added a release notes section.



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025154079


##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.util.Collections;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class WorkerSketchFetcherTest
+{
+  @Mock
+  private CompleteKeyStatisticsInformation completeKeyStatisticsInformation;
+  @Mock
+  private StageDefinition stageDefinition;
+  @Mock
+  private ClusterBy clusterBy;
+  private AutoCloseable mocks;
+  private WorkerSketchFetcher target;
+
+  @Before
+  public void setUp()

Review Comment:
   I think we need to add more tests related to concurrency.
   You could do that by mocking worker clients to basically return a dummy sketch along with doing CountDownLatch or cyclicBarrier operations. 
   
   This way you can control which threads are getting the result. 
   
   https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/CyclicBarrier.html
   https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/CountDownLatch.html
   



##########
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:
   The partitionFutureCallback would eventually get triggered so I do not think it will help. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -122,22 +128,31 @@ private CompletableFuture<Either<Long, ClusterByPartitions>> inMemoryFullSketchM
                 stageDefinition.getId().getQueryId(),
                 stageDefinition.getStageNumber()
             );
-        partitionFuture.whenComplete((result, exception) -> snapshotFuture.cancel(true));
+        partitionFuture.whenComplete((result, exception) -> {
+          if (exception != null || (result != null && result.isError())) {
+            snapshotFuture.cancel(true);
+          }

Review Comment:
   Else should be an ISE ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -19,6 +19,8 @@
 
 package org.apache.druid.msq.statistics;
 
+import org.apache.curator.shaded.com.google.common.collect.ImmutableSortedMap;

Review Comment:
   Lets use the non shaded version 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023709353


##########
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 workerTaskId, WorkOrder workOrder);
 
+  /**
+   * Fetches the {@link ClusterByStatisticsSnapshot} from a worker. This is intended to be used by the
+   * {@link WorkerSketchFetcher} under PARALLEL or AUTO modes.
+   */
+  ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber)
+      throws ExecutionException, InterruptedException;
+
+  /**
+   * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk.
+   * This is intended to be used by the {@link WorkerSketchFetcher} under SEQUENTIAL or AUTO modes.
+   */
+  ClusterByStatisticsSnapshot fetchClusterByStatisticsSnapshotForTimeChunk(

Review Comment:
   I think we should make this API non blocking. 
   Return 
   LIstenableFuture<ClusterByStatisticsSnapshot> and on complete we can do stuff on it. 
   Check the TaskContactFn() in controller impl.
   This will simplify the WorkerSketchFetcher.java a lot. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1024979921


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.statistics;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+/**
+ * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker.
+ */
+public class CompleteKeyStatisticsInformation
+{
+  private final SortedMap<Long, Set<Integer>> timeSegmentVsWorkerMap;
+
+  private boolean hasMultipleValues;
+
+  private double bytesRetained;
+
+  public CompleteKeyStatisticsInformation(
+      final SortedMap<Long, Set<Integer>> timeChunks,
+      boolean hasMultipleValues,
+      double bytesRetained
+  )
+  {
+    this.timeSegmentVsWorkerMap = timeChunks;
+    this.hasMultipleValues = hasMultipleValues;
+    this.bytesRetained = bytesRetained;
+  }
+
+  public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation)

Review Comment:
   Added javadoc



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1023702717


##########
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:
   I think its okay if we are working on the stale value. I would actually make this call before line 195 since we donot want the extra http call. 



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] adarshsanjeev commented on a diff in pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
adarshsanjeev commented on code in PR #13205:
URL: https://github.com/apache/druid/pull/13205#discussion_r1025420452


##########
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:
   Removed this lock



-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cryptoe merged pull request #13205: Add sequential sketch merging to MSQ

Posted by GitBox <gi...@apache.org>.
cryptoe merged PR #13205:
URL: https://github.com/apache/druid/pull/13205


-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org