Copilot commented on code in PR #14698: URL: https://github.com/apache/pinot/pull/14698#discussion_r2999273195
########## pinot-core/src/main/java/org/apache/pinot/core/operator/combine/PartitionedGroupByCombineOperator.java: ########## @@ -0,0 +1,300 @@ +/** + * 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 { + public static final String ALGORITHM = "PARTITIONED"; + + 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.info("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 + * </li> + * </ul> + */ + @Override + public BaseResultsBlock mergeResults() + throws Exception { + long timeoutMs = _queryContext.getEndTimeMs() - System.currentTimeMillis(); + boolean opCompleted = _operatorLatch.await(timeoutMs, TimeUnit.MILLISECONDS); + if (!opCompleted) { + return getTimeoutResultsBlock(timeoutMs); + } + + Throwable processingException = _processingException.get(); + if (processingException != null) { + return getExceptionResultsBlock(processingException); + } + + List<IndexedTable> partitionTables = new ArrayList<>(_indexedTables.length); + for (IndexedTable partitionTable : _indexedTables) { + if (partitionTable != null) { + partitionTables.add(partitionTable); + } + } + IndexedTable indexedTable; + try { + indexedTable = mergePartitionTables(partitionTables); + } catch (TimeoutException e) { + long remainingTimeMs = _queryContext.getEndTimeMs() - System.currentTimeMillis(); + return getTimeoutResultsBlock(Math.max(0L, remainingTimeMs)); + } + if (indexedTable == null) { + return ResultsBlockUtils.buildEmptyQueryResults(_queryContext); + } + return getMergedResultsBlock(indexedTable); + } + + private void mergeResultsBlockIntoLocalPartitions(IndexedTable[] localIndexedTables, + GroupByResultsBlock resultsBlock) { + updateCombineResultsStats(resultsBlock); + + Collection<IntermediateRecord> intermediateRecords = resultsBlock.getIntermediateRecords(); + int mergedKeys = 0; + if (intermediateRecords == null) { + AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult(); + if (aggregationGroupByResult != null) { + try { + Iterator<GroupKeyGenerator.GroupKey> dicGroupKeyIterator = aggregationGroupByResult.getGroupKeyIterator(); + while (dicGroupKeyIterator.hasNext()) { + QueryThreadContext.checkTerminationAndSampleUsagePeriodically(mergedKeys++, EXPLAIN_NAME); + GroupKeyGenerator.GroupKey groupKey = dicGroupKeyIterator.next(); + Object[] keys = groupKey._keys; + Object[] values = Arrays.copyOf(keys, _numColumns); + int groupId = groupKey._groupId; + for (int i = 0; i < _numAggregationFunctions; i++) { + values[_numGroupByExpressions + i] = aggregationGroupByResult.getResultForGroupId(i, groupId); + } + Key key = new Key(keys); + getOrCreateLocalPartitionTable(localIndexedTables, getPartitionId(key), resultsBlock).upsert(key, + new Record(values)); + } + } finally { + aggregationGroupByResult.closeGroupKeyGenerator(); + } + } + } else { + for (IntermediateRecord intermediateResult : intermediateRecords) { + QueryThreadContext.checkTerminationAndSampleUsagePeriodically(mergedKeys++, EXPLAIN_NAME); + getOrCreateLocalPartitionTable(localIndexedTables, getPartitionId(intermediateResult._key), resultsBlock) + .upsert(intermediateResult._key, intermediateResult._record); + } + } + } + + private IndexedTable getOrCreateLocalPartitionTable(IndexedTable[] localIndexedTables, int partitionId, + GroupByResultsBlock resultsBlock) { + IndexedTable indexedTable = localIndexedTables[partitionId]; + if (indexedTable == null) { + indexedTable = createIndexedTable(resultsBlock, 1); + localIndexedTables[partitionId] = indexedTable; + } + return indexedTable; + } + + private void publishLocalPartitions(IndexedTable[] localIndexedTables) { + if (localIndexedTables == null) { + return; + } + for (int partitionId = 0; partitionId < localIndexedTables.length; partitionId++) { + IndexedTable localIndexedTable = localIndexedTables[partitionId]; + if (localIndexedTable == null) { + continue; + } + synchronized (_partitionLocks[partitionId]) { + IndexedTable indexedTable = _indexedTables[partitionId]; + if (indexedTable == null) { + _indexedTables[partitionId] = localIndexedTable; + } else if (localIndexedTable.size() > indexedTable.size()) { + localIndexedTable.merge(indexedTable); + _indexedTables[partitionId] = localIndexedTable; + } else { + indexedTable.merge(localIndexedTable); + } + } + } + } + + private IndexedTable mergePartitionTables(List<IndexedTable> partitionTables) + throws Exception { + int numPartitionTables = partitionTables.size(); + if (numPartitionTables == 0) { + return null; + } + if (numPartitionTables == 1) { + return partitionTables.get(0); + } + + List<IndexedTable> tablesToMerge = partitionTables; + if (tablesToMerge.size() <= 2 || _numTasks <= 1) { + return mergePartitionTablesSequentially(tablesToMerge); + } + + while (tablesToMerge.size() > 1) { + if (_queryContext.getEndTimeMs() - System.currentTimeMillis() <= 0) { + throw new TimeoutException("Timed out while reducing partitioned group-by results"); + } + + int numPairs = tablesToMerge.size() / 2; + List<Future<IndexedTable>> futures = new ArrayList<>(numPairs); + List<IndexedTable> nextRoundTables = new ArrayList<>((tablesToMerge.size() + 1) / 2); + for (int i = 0; i < numPairs; i++) { + IndexedTable leftTable = tablesToMerge.get(i * 2); + IndexedTable rightTable = tablesToMerge.get(i * 2 + 1); + futures.add(_executorService.submit(new TraceCallable<IndexedTable>() { + @Override + public IndexedTable callJob() { + return mergePartitionTables(leftTable, rightTable); + } + })); + } + if ((tablesToMerge.size() & 1) == 1) { + nextRoundTables.add(tablesToMerge.get(tablesToMerge.size() - 1)); + } + try { + for (Future<IndexedTable> future : futures) { + long timeoutMs = _queryContext.getEndTimeMs() - System.currentTimeMillis(); + if (timeoutMs <= 0) { + throw new TimeoutException("Timed out while reducing partitioned group-by results"); + } + nextRoundTables.add(future.get(timeoutMs, TimeUnit.MILLISECONDS)); + } + } catch (ExecutionException e) { + throw new RuntimeException("Error while reducing partitioned group-by results", e.getCause()); + } Review Comment: In `mergePartitionTables()`, the reduction tasks are submitted to `_executorService`, but on timeout/interruption the method throws without cancelling the in-flight futures. Because the merge work itself doesn’t check `QueryThreadContext` termination, these tasks can keep running after the query has already timed out/cancelled, wasting CPU and potentially extending tail latency for other queries. Consider catching `TimeoutException`/`InterruptedException` as well, cancelling all outstanding futures, and restoring the interrupt status where appropriate. ########## pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java: ########## @@ -390,7 +390,7 @@ private static List<Expression> computeInOperands(List<Object[]> dataContainer, for (int rowIdx = 0; rowIdx < numRows; rowIdx++) { arrString[rowIdx] = (String) dataContainer.get(rowIdx)[colIdx]; } - Arrays.sort(arrString); + Arrays.parallelSort(arrString); Review Comment: `computeInOperands()` switched to `Arrays.parallelSort(arrString)`. This uses the JVM common ForkJoinPool and adds extra overhead/parallelism in the query runtime, while the other types still use `Arrays.sort()`. For typical IN operand sizes this can regress latency and cause thread contention; consider reverting to `Arrays.sort()` or only using a parallel sort above a clearly justified size threshold (ideally without relying on the common pool). ```suggestion Arrays.sort(arrString); ``` ########## pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java: ########## @@ -257,6 +262,23 @@ private void applyQueryOptions(QueryContext queryContext) { } else { queryContext.setMinInitialIndexedTableCapacity(_minInitialIndexedTableCapacity); } + String groupByAlgorithm = QueryOptionsUtils.getGroupByAlgorithm(queryOptions); + if (groupByAlgorithm != null) { + groupByAlgorithm = groupByAlgorithm.trim().toUpperCase(Locale.ROOT); + queryContext.setGroupByAlgorithm(groupByAlgorithm); + } else { + groupByAlgorithm = DEFAULT_GROUP_BY_ALGORITHM; + queryContext.setGroupByAlgorithm(DEFAULT_GROUP_BY_ALGORITHM); + } Review Comment: `groupByAlgorithm` is normalized via `trim().toUpperCase(...)`, but if the query option is present and blank/whitespace this will set `QueryContext` to an empty algorithm string. That can silently disable the DEFAULT-specific behavior (e.g., safe-trim sorted group-by selection) even though the user effectively didn’t choose an algorithm. Consider treating a blank value the same as unset and falling back to `DEFAULT_GROUP_BY_ALGORITHM`. ########## pinot-core/src/main/java/org/apache/pinot/core/operator/combine/NonblockingGroupByCombineOperator.java: ########## @@ -0,0 +1,111 @@ +/** + * 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.List; +import java.util.concurrent.ExecutorService; +import org.apache.pinot.core.common.Operator; +import org.apache.pinot.core.data.table.IndexedTable; +import org.apache.pinot.core.operator.AcquireReleaseColumnsSegmentOperator; +import org.apache.pinot.core.operator.blocks.results.GroupByResultsBlock; +import org.apache.pinot.core.query.request.context.QueryContext; +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 NonblockingGroupByCombineOperator extends GroupByCombineOperator { + public static final String ALGORITHM = "NON-BLOCKING"; + + private static final Logger LOGGER = LoggerFactory.getLogger(NonblockingGroupByCombineOperator.class); + private static final String EXPLAIN_NAME = "NON_BLOCKING_COMBINE_GROUP_BY"; + + public NonblockingGroupByCombineOperator(List<Operator> operators, QueryContext queryContext, + ExecutorService executorService) { + super(operators, queryContext, executorService); + LOGGER.info("Using {} for group-by combine with {} tasks", ALGORITHM, _numTasks); Review Comment: Both new group-by combine operators log their selection at INFO from the constructor. Since combine operators are instantiated per query, this can generate very high log volume on busy servers and drown out actionable logs. Consider lowering this to DEBUG/TRACE or gating it behind a query option / explain/trace path instead of logging per query at INFO. ```suggestion LOGGER.debug("Using {} for group-by combine with {} tasks", ALGORITHM, _numTasks); ``` ########## pinot-core/src/main/java/org/apache/pinot/core/plan/CombinePlanNode.java: ########## @@ -165,4 +159,27 @@ private BaseCombineOperator getCombineOperator() { } } } + + private BaseCombineOperator getGroupByCombineOperator(List<Operator> operators) { + String groupByAlgorithm = _queryContext.getGroupByAlgorithm(); + String normalizedGroupByAlgorithm = groupByAlgorithm != null ? groupByAlgorithm.trim().toUpperCase(Locale.ROOT) + : GroupByCombineOperator.ALGORITHM; + + if (GroupByCombineOperator.ALGORITHM.equals(normalizedGroupByAlgorithm) + && _queryContext.shouldSortAggregateUnderSafeTrim()) { + if (operators.size() < _queryContext.getSortAggregateSequentialCombineNumSegmentsThreshold()) { Review Comment: `normalizedGroupByAlgorithm` is derived from `groupByAlgorithm.trim().toUpperCase(...)`, but if `groupByAlgorithm` is the empty string (or whitespace) this normalization yields an empty algorithm instead of falling back to `GroupByCombineOperator.ALGORITHM`. That can bypass the DEFAULT-only safe-trim optimization path. Consider checking for blank after trimming and defaulting to `GroupByCombineOperator.ALGORITHM`. ########## pinot-core/src/main/java/org/apache/pinot/core/operator/combine/PartitionedGroupByCombineOperator.java: ########## @@ -0,0 +1,300 @@ +/** + * 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 { + public static final String ALGORITHM = "PARTITIONED"; + + 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.info("Using {} for group-by combine, with {} partitions and {} numTasks", EXPLAIN_NAME, numGroupByPartitions, Review Comment: This operator logs algorithm/partition details at INFO in the constructor. Because combine operators are created per query, this can cause excessive log volume under load. Consider lowering the level to DEBUG/TRACE or emitting this information via explain/trace instead of INFO logging for every query. ```suggestion LOGGER.debug("Using {} for group-by combine, with {} partitions and {} numTasks", EXPLAIN_NAME, numGroupByPartitions, ``` -- 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]
