Jackie-Jiang commented on code in PR #10006: URL: https://github.com/apache/pinot/pull/10006#discussion_r1068832926
########## 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: When query is satisfied, we should skip processing all the following operators, instead of only breaking the current operator. -- 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