-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add a non-blocking and partitioned groupBy implementation
- Loading branch information
Showing
11 changed files
with
728 additions
and
12 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
154 changes: 154 additions & 0 deletions
154
...c/main/java/org/apache/pinot/core/operator/combine/NonblockingGroupByCombineOperator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,154 @@ | ||
/** | ||
* 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.Arrays; | ||
import java.util.Collection; | ||
import java.util.Iterator; | ||
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.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.GroupByResultsBlock; | ||
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.GroupByUtils; | ||
import org.apache.pinot.spi.trace.Tracing; | ||
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); | ||
} | ||
|
||
@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; | ||
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(); | ||
IndexedTable indexedTable = null; | ||
if (_indexedTable != null) { | ||
synchronized (this) { | ||
if (_indexedTable != null) { | ||
indexedTable = _indexedTable; | ||
_indexedTable = null; | ||
} | ||
} | ||
} | ||
if (indexedTable == null) { | ||
indexedTable = GroupByUtils.createIndexedTableForCombineOperator(resultsBlock, _queryContext, 1); | ||
} | ||
|
||
// Set groups limit reached flag. | ||
if (resultsBlock.isNumGroupsLimitReached()) { | ||
_numGroupsLimitReached = true; | ||
} | ||
|
||
// Merge aggregation group-by result. | ||
// Iterate over the group-by keys, for each key, update the group-by result in the indexedTable | ||
Collection<IntermediateRecord> intermediateRecords = resultsBlock.getIntermediateRecords(); | ||
// Count the number of merged keys | ||
int mergedKeys = 0; | ||
// For now, only GroupBy OrderBy query has pre-constructed intermediate records | ||
if (intermediateRecords == null) { | ||
// Merge aggregation group-by result. | ||
AggregationGroupByResult aggregationGroupByResult = resultsBlock.getAggregationGroupByResult(); | ||
if (aggregationGroupByResult != null) { | ||
// Iterate over the group-by keys, for each key, update the group-by result in the indexedTable | ||
Iterator<GroupKeyGenerator.GroupKey> dicGroupKeyIterator = aggregationGroupByResult.getGroupKeyIterator(); | ||
while (dicGroupKeyIterator.hasNext()) { | ||
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); | ||
} | ||
indexedTable.upsert(new Key(keys), new Record(values)); | ||
Tracing.ThreadAccountantOps.sampleAndCheckInterruptionPeriodically(mergedKeys); | ||
mergedKeys++; | ||
} | ||
} | ||
} else { | ||
for (IntermediateRecord intermediateResult : intermediateRecords) { | ||
//TODO: change upsert api so that it accepts intermediateRecord directly | ||
indexedTable.upsert(intermediateResult._key, intermediateResult._record); | ||
Tracing.ThreadAccountantOps.sampleAndCheckInterruptionPeriodically(mergedKeys); | ||
mergedKeys++; | ||
} | ||
} | ||
boolean setGroupByResult = false; | ||
while (!setGroupByResult) { | ||
IndexedTable indexedTableToMerge = null; | ||
synchronized (this) { | ||
if (_indexedTable == null) { | ||
_indexedTable = indexedTable; | ||
setGroupByResult = true; | ||
} else { | ||
indexedTableToMerge = _indexedTable; | ||
_indexedTable = null; | ||
} | ||
} | ||
if (indexedTableToMerge != null) { | ||
indexedTable.merge(indexedTableToMerge); | ||
} | ||
} | ||
} catch (RuntimeException e) { | ||
throw wrapOperatorException(operator, e); | ||
} finally { | ||
if (operator instanceof AcquireReleaseColumnsSegmentOperator) { | ||
((AcquireReleaseColumnsSegmentOperator) operator).release(); | ||
} | ||
} | ||
} | ||
} | ||
} |
Oops, something went wrong.