Copilot commented on code in PR #14698:
URL: https://github.com/apache/pinot/pull/14698#discussion_r3012265735


##########
pinot-core/src/main/java/org/apache/pinot/core/plan/CombinePlanNode.java:
##########
@@ -165,4 +154,11 @@ private BaseCombineOperator getCombineOperator() {
       }
     }
   }
+
+  private BaseCombineOperator getGroupByCombineOperator(List<Operator> 
operators) {
+    // Always use PartitionedGroupByCombineOperator. When 
numGroupByPartitions=1 it degenerates
+    // to a single partition (equivalent to the old DEFAULT path). Users 
control partitioning via
+    // the numGroupByPartitions query option.
+    return new PartitionedGroupByCombineOperator(operators, _queryContext, 
_executorService);
+  }

Review Comment:
   `getGroupByCombineOperator()` now always returns 
`PartitionedGroupByCombineOperator`, which makes the `groupByAlgorithm` query 
option (mentioned in the PR description/tests) effectively unused, and also 
makes the existing safe-trim sort-aggregation combine operators 
(`SortedGroupByCombineOperator` / `SequentialSortedGroupByCombineOperator`) 
unreachable. If this is intended, consider removing/cleaning up the unused 
algorithm option and dead combine operators; otherwise, reintroduce selection 
logic so safe-trim sort-aggregation queries still use the specialized sorted 
combine path and `groupByAlgorithm` continues to control behavior as documented.



##########
pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java:
##########
@@ -257,6 +259,14 @@ private void applyQueryOptions(QueryContext queryContext) {
       } else {
         
queryContext.setMinInitialIndexedTableCapacity(_minInitialIndexedTableCapacity);
       }
+      Integer numGroupByPartitions = 
QueryOptionsUtils.getNumGroupByPartitions(queryOptions);
+      if (numGroupByPartitions != null) {
+        queryContext.setNumGroupByPartitions(numGroupByPartitions);
+      } else {
+        int effectiveParallelism = queryContext.getMaxExecutionThreads() > 0 ? 
queryContext.getMaxExecutionThreads()
+            : QueryMultiThreadingUtils.MAX_NUM_THREADS_PER_QUERY;

Review Comment:
   The default `numGroupByPartitions` is derived from 
`queryContext.getMaxExecutionThreads()` / 
`QueryMultiThreadingUtils.MAX_NUM_THREADS_PER_QUERY`, but group-by combine 
later overrides `maxExecutionThreads` when it’s <= 0 (see 
`GroupByCombineOperator.overrideMaxExecutionThreads`). This can make the 
default partition count diverge from the actual task parallelism used by the 
combine operator (e.g., partitions < tasks), undermining the “based on 
effective parallelism” behavior described in the PR. Consider computing the 
default partition count closer to where `_numTasks` is known (e.g., in 
`PartitionedGroupByCombineOperator` when the option is unset), or use the same 
effective-thread calculation as the combine operator so partitions track the 
real execution parallelism.
   ```suggestion
           int effectiveParallelism =
               
QueryMultiThreadingUtils.getEffectiveNumThreadsPerQuery(queryContext.getMaxExecutionThreads());
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/combine/PartitionedGroupByCombineOperator.java:
##########
@@ -0,0 +1,313 @@
+/**
+ * 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.pinot.core.operator.combine;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.data.table.IndexedTable;
+import org.apache.pinot.core.data.table.IntermediateRecord;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.data.table.Record;
+import org.apache.pinot.core.operator.AcquireReleaseColumnsSegmentOperator;
+import org.apache.pinot.core.operator.blocks.results.BaseResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock;
+import org.apache.pinot.core.operator.blocks.results.ResultsBlockUtils;
+import 
org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import org.apache.pinot.core.query.aggregation.groupby.GroupKeyGenerator;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.util.trace.TraceCallable;
+import org.apache.pinot.spi.query.QueryThreadContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Combine operator for group-by queries.
+ * TODO: Use CombineOperatorUtils.getNumThreadsForQuery() to get the 
parallelism of the query instead of using
+ *       all threads
+ */
+@SuppressWarnings("rawtypes")
+public class PartitionedGroupByCombineOperator extends GroupByCombineOperator {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PartitionedGroupByCombineOperator.class);
+  private static final String EXPLAIN_NAME = "PARTITIONED_COMBINE_GROUP_BY";
+
+  protected final IndexedTable[] _indexedTables;
+  private final Object[] _partitionLocks;
+  private final int _partitionMask;
+
+  public PartitionedGroupByCombineOperator(List<Operator> operators, 
QueryContext queryContext,
+      ExecutorService executorService) {
+    super(operators, queryContext, executorService);
+    int numGroupByPartitions = Math.max(1, 
_queryContext.getNumGroupByPartitions());
+    _indexedTables = new IndexedTable[numGroupByPartitions];
+    _partitionLocks = new Object[numGroupByPartitions];
+    Arrays.setAll(_partitionLocks, ignored -> new Object());
+    _partitionMask = Integer.bitCount(numGroupByPartitions) == 1 ? 
numGroupByPartitions - 1 : -1;
+    LOGGER.debug("Using {} for group-by combine, with {} partitions and {} 
numTasks", EXPLAIN_NAME,
+        numGroupByPartitions,
+        _numTasks);
+  }
+
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  /**
+   * Executes query on one segment in a worker thread and merges the results 
into the indexed table.
+   */
+  @Override
+  protected void processSegments() {
+    int operatorId;
+    IndexedTable[] localIndexedTables = null;
+    while (_processingException.get() == null && (operatorId = 
_nextOperatorId.getAndIncrement()) < _numOperators) {
+      Operator operator = _operators.get(operatorId);
+      try {
+        if (operator instanceof AcquireReleaseColumnsSegmentOperator) {
+          ((AcquireReleaseColumnsSegmentOperator) operator).acquire();
+        }
+        GroupByResultsBlock resultsBlock = (GroupByResultsBlock) 
operator.nextBlock();
+        if (localIndexedTables == null) {
+          localIndexedTables = new IndexedTable[_indexedTables.length];
+        }
+        mergeResultsBlockIntoLocalPartitions(localIndexedTables, resultsBlock);
+      } catch (RuntimeException e) {
+        throw wrapOperatorException(operator, e);
+      } finally {
+        if (operator instanceof AcquireReleaseColumnsSegmentOperator) {
+          ((AcquireReleaseColumnsSegmentOperator) operator).release();
+        }
+      }
+    }
+    publishLocalPartitions(localIndexedTables);
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>Combines intermediate selection result blocks from underlying 
operators and returns a merged one.
+   * <ul>
+   *   <li>
+   *     Merges multiple intermediate selection result blocks as a merged one.
+   *   </li>
+   *   <li>
+   *     Set all exceptions encountered during execution into the merged 
result block

Review Comment:
   The Javadoc for `mergeResults()` (and the surrounding comment block) refers 
to "selection result blocks" and "intermediate selection result blocks", but 
this operator is merging `GroupByResultsBlock`s. Please update the wording to 
avoid confusion when debugging/explaining group-by combine behavior.
   ```suggestion
      * <p>Combines intermediate group-by results produced by underlying 
operators and returns a merged results block.
      * <ul>
      *   <li>
      *     Merges multiple intermediate group-by result partitions into a 
single merged group-by result.
      *   </li>
      *   <li>
      *     Sets all exceptions encountered during execution into the merged 
result 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to