walterddr commented on code in PR #10006: URL: https://github.com/apache/pinot/pull/10006#discussion_r1070102222
########## pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java: ########## @@ -264,17 +264,9 @@ public Plan makeStreamingInstancePlan(List<IndexSegment> indexSegments, QueryCon planNodes.add(makeStreamingSegmentPlanNode(indexSegment, queryContext)); } CombinePlanNode combinePlanNode = new CombinePlanNode(planNodes, queryContext, executorService, streamObserver); - if (QueryContextUtils.isSelectionOnlyQuery(queryContext)) { - // selection-only is streamed in StreamingSelectionPlanNode --> here only metadata block is returned. - return new GlobalPlanImplV0( - new InstanceResponsePlanNode(combinePlanNode, indexSegments, Collections.emptyList(), queryContext)); - } else { - // non-selection-only requires a StreamingInstanceResponsePlanNode to stream data block back and metadata block - // as final return. - return new GlobalPlanImplV0( - new StreamingInstanceResponsePlanNode(combinePlanNode, indexSegments, Collections.emptyList(), queryContext, - streamObserver)); - } + return new GlobalPlanImplV0( + new StreamingInstanceResponsePlanNode(combinePlanNode, indexSegments, Collections.emptyList(), queryContext, Review Comment: correct. we are going to `makeStreamingInstancePlan` all the time. b/c for selection only it is going to stream block by block. and for aggregation we will do segment level combine first then still string data block by block. next PR will make this configurable. e.g. do we want to do segment level combine, and how much combine we want to do. but the idea is use streaming to configure it otherwise we are limited to only able to return 1 data block ########## pinot-core/src/main/java/org/apache/pinot/core/plan/CombinePlanNode.java: ########## @@ -175,35 +179,54 @@ public List<Operator> callJob() { } } - if (_streamObserver != null && QueryContextUtils.isSelectionOnlyQuery(_queryContext)) { - // Streaming query (only support selection only) - return new StreamingSelectionOnlyCombineOperator(operators, _queryContext, _executorService, _streamObserver); - } - if (QueryContextUtils.isAggregationQuery(_queryContext)) { - if (_queryContext.getGroupByExpressions() == null) { - // Aggregation only - return new AggregationCombineOperator(operators, _queryContext, _executorService); + if (_streamObserver != null) { Review Comment: yes i think we can follow up with separate PR, in fact my next PR will include changes to this file so will look into it next ########## pinot-core/src/main/java/org/apache/pinot/core/operator/combine/BaseCombineOperator.java: ########## @@ -122,166 +118,64 @@ public void runJob() { } else { LOGGER.error("Caught serious error while processing query: " + _queryContext, t); } - onException(t); + onProcessSegmentsException(t); } finally { - onFinish(); - phaser.arriveAndDeregister(); + onProcessSegmentsFinish(); + _phaser.arriveAndDeregister(); Tracing.ThreadAccountantOps.clear(); } _totalWorkerThreadCpuTimeNs.getAndAdd(threadResourceUsageProvider.getThreadTimeNs()); } }); } - - BaseResultsBlock mergedBlock; - try { - mergedBlock = mergeResults(); - } catch (InterruptedException | EarlyTerminationException e) { - Exception killedErrorMsg = Tracing.getThreadAccountant().getErrorStatus(); - throw new QueryCancelledException( - "Cancelled while merging results blocks" - + (killedErrorMsg == null ? StringUtils.EMPTY : " " + killedErrorMsg), e); - } catch (Exception e) { - LOGGER.error("Caught exception while merging results blocks (query: {})", _queryContext, e); - mergedBlock = new ExceptionResultsBlock(QueryException.getException(QueryException.INTERNAL_ERROR, e)); - } finally { - // Cancel all ongoing jobs - for (Future future : _futures) { - if (!future.isDone()) { - future.cancel(true); - } - } - // Deregister the main thread and wait for all threads done - phaser.awaitAdvance(phaser.arriveAndDeregister()); - } - /* - * _numTasks are number of async tasks submitted to the _executorService, but it does not mean Pinot server - * use those number of threads to concurrently process segments. Instead, if _executorService thread pool has - * less number of threads than _numTasks, the number of threads that used to concurrently process segments equals - * to the pool size. - * TODO: Get the actual number of query worker threads instead of using the default value. - */ - int numServerThreads = Math.min(_numTasks, ResourceManager.DEFAULT_QUERY_WORKER_THREADS); - CombineOperatorUtils.setExecutionStatistics(mergedBlock, _operators, _totalWorkerThreadCpuTimeNs.get(), - numServerThreads); - return mergedBlock; } /** - * Executes query on one or more segments in a worker thread. + * Stop the combine operator process. This will stop all sub-tasks that were spun up to process data segments. */ - protected void processSegments() { - int operatorId; - while ((operatorId = _nextOperatorId.getAndIncrement()) < _numOperators) { - Operator operator = _operators.get(operatorId); - T resultsBlock; - try { - if (operator instanceof AcquireReleaseColumnsSegmentOperator) { - ((AcquireReleaseColumnsSegmentOperator) operator).acquire(); - } - resultsBlock = (T) operator.nextBlock(); - } finally { - if (operator instanceof AcquireReleaseColumnsSegmentOperator) { - ((AcquireReleaseColumnsSegmentOperator) operator).release(); - } - } - - if (isQuerySatisfied(resultsBlock)) { - // Query is satisfied, skip processing the remaining segments - _blockingQueue.offer(resultsBlock); - return; - } else { - _blockingQueue.offer(resultsBlock); + protected void stopProcess() { + // Cancel all ongoing jobs + for (Future future : _futures) { + if (!future.isDone()) { + future.cancel(true); } } + // Deregister the main thread and wait for all threads done + _phaser.awaitAdvance(_phaser.arriveAndDeregister()); Review Comment: this entire `stopProcess()` is being called in the finally clause of `getNextBlock()`. is this what you are referring to? ########## pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/BaseStreamBlockCombineOperator.java: ########## @@ -0,0 +1,176 @@ +/** + * 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.streaming; + +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.pinot.common.exception.QueryException; +import org.apache.pinot.core.common.Operator; +import org.apache.pinot.core.operator.AcquireReleaseColumnsSegmentOperator; +import org.apache.pinot.core.operator.blocks.results.BaseResultsBlock; +import org.apache.pinot.core.operator.blocks.results.ExceptionResultsBlock; +import org.apache.pinot.core.operator.blocks.results.MetadataResultsBlock; +import org.apache.pinot.core.operator.combine.BaseCombineOperator; +import org.apache.pinot.core.operator.combine.CombineOperatorUtils; +import org.apache.pinot.core.operator.combine.merger.ResultBlockMerger; +import org.apache.pinot.core.query.request.context.QueryContext; +import org.apache.pinot.core.query.scheduler.resources.ResourceManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public abstract class BaseStreamBlockCombineOperator<T extends BaseResultsBlock> + extends BaseCombineOperator<BaseResultsBlock> { + private static final Logger LOGGER = LoggerFactory.getLogger(BaseStreamBlockCombineOperator.class); + + /** + * Special results block to indicate that this is the last results block for a child operator in the list + */ + public static final MetadataResultsBlock LAST_RESULTS_BLOCK = new MetadataResultsBlock(); + + // Use a BlockingQueue to store the intermediate results blocks + protected final BlockingQueue<BaseResultsBlock> _blockingQueue = new LinkedBlockingQueue<>(); + protected final ResultBlockMerger<T> _combineFunction; + + protected int _numOperatorsFinished; + protected BaseResultsBlock _exceptionBlock; + + public BaseStreamBlockCombineOperator(ResultBlockMerger<T> combineFunction, List<Operator> operators, + QueryContext queryContext, ExecutorService executorService) { + super(operators, queryContext, executorService); + _combineFunction = combineFunction; + _numOperatorsFinished = 0; + _exceptionBlock = null; + } + + @Override + protected BaseResultsBlock getNextBlock() { + long endTimeMs = _queryContext.getEndTimeMs(); + while (_exceptionBlock == null && _numOperatorsFinished < _numOperators) { + try { + BaseResultsBlock resultsBlock = + _blockingQueue.poll(endTimeMs - System.currentTimeMillis(), TimeUnit.MILLISECONDS); + + if (resultsBlock == null) { + // Query times out, skip streaming the remaining results blocks + LOGGER.error("Timed out while polling results block (query: {})", _queryContext); + _exceptionBlock = new ExceptionResultsBlock(QueryException.getException( + QueryException.EXECUTION_TIMEOUT_ERROR, new TimeoutException("Timed out while polling results block"))); + return _exceptionBlock; + } + if (resultsBlock.getProcessingExceptions() != null) { + // Caught exception while processing segment, skip streaming the remaining results blocks and directly return + // the exception + _exceptionBlock = resultsBlock; + return _exceptionBlock; + } + if (resultsBlock == LAST_RESULTS_BLOCK) { + // Caught LAST_RESULTS_BLOCK from a specific task, indicated it has finished. + // Skip returning this metadata block and continue to process the next from the _blockingQueue. + _numOperatorsFinished++; + continue; + } + return resultsBlock; + } catch (Exception e) { + LOGGER.error("Caught exception while merging results blocks (query: {})", _queryContext, e); + _exceptionBlock = new ExceptionResultsBlock(QueryException.getException(QueryException.INTERNAL_ERROR, e)); + return _exceptionBlock; + } + } + // Setting the execution stats for the final return + BaseResultsBlock finalBlock = new MetadataResultsBlock(); + int numServerThreads = Math.min(_numTasks, ResourceManager.DEFAULT_QUERY_WORKER_THREADS); + CombineOperatorUtils.setExecutionStatistics(finalBlock, _operators, _totalWorkerThreadCpuTimeNs.get(), + numServerThreads); + return finalBlock; + } + + /** + * Executes query on one or more segments in a worker thread. + */ + @Override + protected void processSegments() { + int operatorId; + while ((operatorId = _nextOperatorId.getAndIncrement()) < _numOperators) { + Operator operator = _operators.get(operatorId); + T resultsBlock; + try { + if (operator instanceof AcquireReleaseColumnsSegmentOperator) { + ((AcquireReleaseColumnsSegmentOperator) operator).acquire(); + } + while ((resultsBlock = (T) operator.nextBlock()) != null) { + if (shouldFinishStream(resultsBlock) || _combineFunction.isQuerySatisfied(resultsBlock)) { + // Query is satisfied, skip processing the remaining segments + _blockingQueue.offer(resultsBlock); Review Comment: correct. let me add a TODO and address in separate PR. ########## pinot-core/src/main/java/org/apache/pinot/core/operator/combine/merger/SelectionOnlyResultBlockMerger.java: ########## @@ -0,0 +1,65 @@ +/** + * 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.merger; + +import java.util.Collection; +import org.apache.pinot.common.exception.QueryException; +import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.core.operator.blocks.results.SelectionResultsBlock; +import org.apache.pinot.core.query.request.context.QueryContext; +import org.apache.pinot.core.query.selection.SelectionOperatorUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class SelectionOnlyResultBlockMerger implements ResultBlockMerger<SelectionResultsBlock> { + private static final Logger LOGGER = LoggerFactory.getLogger(SelectionOnlyResultBlockMerger.class); + private final int _numRowsToKeep; + + public SelectionOnlyResultBlockMerger(QueryContext queryContext) { + _numRowsToKeep = queryContext.getLimit(); + } + + @Override + public boolean isQuerySatisfied(SelectionResultsBlock resultsBlock) { + return resultsBlock.getRows().size() == _numRowsToKeep; + } + + @Override + public void mergeResultsBlocks(SelectionResultsBlock mergedBlock, SelectionResultsBlock blockToMerge) { + DataSchema mergedDataSchema = mergedBlock.getDataSchema(); + DataSchema dataSchemaToMerge = blockToMerge.getDataSchema(); + assert mergedDataSchema != null && dataSchemaToMerge != null; + if (!mergedDataSchema.equals(dataSchemaToMerge)) { + String errorMessage = + String.format("Data schema mismatch between merged block: %s and block to merge: %s, drop block to merge", Review Comment: i think that's the intention. but regardless this is the same behavior before the refactor. i think we can address in separate PR? (i will add a TODO if that helps) -- 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-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org