xiangfu0 commented on code in PR #14698:
URL: https://github.com/apache/pinot/pull/14698#discussion_r3011926692
##########
pinot-core/src/main/java/org/apache/pinot/core/util/GroupByUtils.java:
##########
@@ -193,18 +193,17 @@ public static IndexedTable
createIndexedTableForDataTableReducer(DataTable dataT
private static IndexedTable getTrimDisabledIndexedTable(DataSchema
dataSchema, boolean hasFinalInput,
QueryContext queryContext, int resultSize, int initialCapacity, int
numThreads, ExecutorService executorService) {
+ if (numThreads == 1) {
+ return new SimpleIndexedTable(dataSchema, hasFinalInput, queryContext,
resultSize, Integer.MAX_VALUE,
+ Integer.MAX_VALUE, initialCapacity, executorService);
+ }
if (queryContext.isAccurateGroupByWithoutOrderBy() &&
queryContext.getOrderByExpressions() == null
&& queryContext.getHavingFilter() == null) {
return new DeterministicConcurrentIndexedTable(dataSchema,
hasFinalInput, queryContext, resultSize,
Integer.MAX_VALUE, Integer.MAX_VALUE, initialCapacity,
executorService);
}
Review Comment:
Good catch. The current ordering preserves the existing behavior where
`accurateGroupByWithoutOrderBy` takes precedence regardless of thread count.
For the partitioned path (numThreads=1 local tables), the query context
typically has ORDER BY so the deterministic path isn't triggered. We can
revisit if this causes issues in practice.
##########
pinot-core/src/main/java/org/apache/pinot/core/operator/combine/PartitionedGroupByCombineOperator.java:
##########
@@ -0,0 +1,317 @@
+/**
+ * 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.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
+ * </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) {
Review Comment:
Addressed — the CompositePartitionTable and its partition-local finish()
fast path were removed in commit dc4ea6e. The PR description has been updated
to no longer reference it. The merge-based `mergePartitionTables()` flow is the
only implementation.
##########
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.debug("Using {} for group-by combine with {} tasks", ALGORITHM,
_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 indexedTable = 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 (indexedTable == null) {
+ synchronized (this) {
+ if (_indexedTable != null) {
+ indexedTable = _indexedTable;
+ _indexedTable = null;
+ }
+ }
+ if (indexedTable == null) {
+ indexedTable = createIndexedTable(resultsBlock, 1);
+ }
+ }
+ mergeGroupByResultsBlock(indexedTable, resultsBlock, EXPLAIN_NAME);
+ } catch (RuntimeException e) {
+ throw wrapOperatorException(operator, e);
+ } finally {
+ if (operator instanceof AcquireReleaseColumnsSegmentOperator) {
+ ((AcquireReleaseColumnsSegmentOperator) operator).release();
+ }
+ }
+ }
+
+ boolean setGroupByResult = false;
+ while (indexedTable != null && !setGroupByResult) {
+ IndexedTable indexedTableToMerge;
+ synchronized (this) {
+ if (_indexedTable == null) {
+ _indexedTable = indexedTable;
+ setGroupByResult = true;
+ continue;
+ } else {
+ indexedTableToMerge = _indexedTable;
+ _indexedTable = null;
+ }
+ }
+ if (indexedTable.size() > indexedTableToMerge.size()) {
+ indexedTable.merge(indexedTableToMerge);
+ } else {
+ indexedTableToMerge.merge(indexedTable);
Review Comment:
Valid observation. The NonblockingGroupByCombineOperator's merge path
doesn't propagate trim stats from the absorbed table. This is a pre-existing
issue shared with the base `BaseTable.merge()` — when the absorbed table was
trimmed during its own processing, that trim history is lost. Filed as a
follow-up item.
##########
pinot-core/src/test/java/org/apache/pinot/core/data/table/IndexedTableTest.java:
##########
@@ -299,4 +299,37 @@ private void testNoMoreNewRecordsInTable(IndexedTable
indexedTable) {
checkEvicted(indexedTable, "f", "g");
}
+
+ @Test
+ public void testMergePartitionTableInvalidatesFinishedTopRecords() {
+ QueryContext queryContext = QueryContextConverterUtils.getQueryContext(
+ "SELECT SUM(m1) FROM testTable GROUP BY d1 ORDER BY SUM(m1) DESC");
+ DataSchema dataSchema = new DataSchema(new String[]{"d1", "sum(m1)"},
+ new ColumnDataType[]{ColumnDataType.STRING, ColumnDataType.DOUBLE});
+
+ IndexedTable indexedTable =
+ new SimpleIndexedTable(dataSchema, false, queryContext, 1, TRIM_SIZE,
TRIM_THRESHOLD, INITIAL_CAPACITY,
+ Executors.newCachedThreadPool());
+ indexedTable.upsert(getRecord(new Object[]{"a", 10d}));
+ indexedTable.upsert(getRecord(new Object[]{"b", 20d}));
+ indexedTable.finish(true);
+ Assert.assertEquals(indexedTable.size(), 1);
+
+ IndexedTable mergeTable =
+ new SimpleIndexedTable(dataSchema, false, queryContext, 1, TRIM_SIZE,
TRIM_THRESHOLD, INITIAL_CAPACITY,
+ Executors.newCachedThreadPool());
+ mergeTable.upsert(getRecord(new Object[]{"c", 30d}));
+
+ indexedTable.mergePartitionTable(mergeTable);
+
+ Assert.assertEquals(indexedTable.size(), 3);
+ List<String> mergedRecords = new ArrayList<>();
+ indexedTable.iterator().forEachRemaining(record ->
mergedRecords.add((String) record.getValues()[0]));
+ Assert.assertEquals(mergedRecords.size(), 3);
+ Assert.assertTrue(mergedRecords.containsAll(Arrays.asList("a", "b", "c")));
+
+ indexedTable.finish(true);
+ Assert.assertEquals(indexedTable.size(), 1);
+ Assert.assertEquals(indexedTable.iterator().next().getValues()[0], "c");
Review Comment:
Fixed — wrapped the shared executor in try-finally with `shutdownNow()` in
the new test method. See commit e7daf21.
--
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]