walterddr commented on code in PR #10779: URL: https://github.com/apache/pinot/pull/10779#discussion_r1198004291
########## pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java: ########## @@ -143,22 +148,51 @@ public void shutDown() } public void processQuery(DistributedStagePlan distributedStagePlan, Map<String, String> requestMetadataMap) { + try { long requestId = Long.parseLong(requestMetadataMap.get(QueryConfig.KEY_OF_BROKER_REQUEST_ID)); long timeoutMs = Long.parseLong(requestMetadataMap.get(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS)); boolean isTraceEnabled = Boolean.parseBoolean(requestMetadataMap.getOrDefault(CommonConstants.Broker.Request.TRACE, "false")); long deadlineMs = System.currentTimeMillis() + timeoutMs; + // run OpChain if (isLeafStage(distributedStagePlan)) { - OpChain rootOperator = compileLeafStage(distributedStagePlan, requestMetadataMap, timeoutMs, deadlineMs, - requestId); + // pre-stage execution for all pipeline breakers + // TODO: pipeline breaker is now only supported by leaf stage, to be supported by all OpChain + PipelineBreakerContext pipelineBreakerContext = executePipelineBreakers(_intermScheduler, distributedStagePlan, + timeoutMs, deadlineMs, requestId, isTraceEnabled); + OpChain rootOperator = compileLeafStage(distributedStagePlan, requestMetadataMap, pipelineBreakerContext, + timeoutMs, deadlineMs, requestId); _leafScheduler.register(rootOperator); } else { PlanNode stageRoot = distributedStagePlan.getStageRoot(); - OpChain rootOperator = PhysicalPlanVisitor.build(stageRoot, + OpChain rootOperator = PhysicalPlanVisitor.walkPlanNode(stageRoot, new PlanRequestContext(_mailboxService, requestId, stageRoot.getPlanFragmentId(), timeoutMs, deadlineMs, - distributedStagePlan.getServer(), distributedStagePlan.getStageMetadata(), isTraceEnabled)); + distributedStagePlan.getServer(), distributedStagePlan.getStageMetadata(), null, isTraceEnabled)); _intermScheduler.register(rootOperator); } + } catch (Throwable t) { + LOGGER.error("", t); + } + } + + private PipelineBreakerContext executePipelineBreakers(OpChainSchedulerService scheduler, + DistributedStagePlan distributedStagePlan, long timeoutMs, long deadlineMs, long requestId, + boolean isTraceEnabled) { + PipelineBreakerContext pipelineBreakerContext = new PipelineBreakerContext(); Review Comment: PipelineBreakerContext does a lot of things 1. capture all the pipeline breaking operators needs to be executed first - currently only mailbox receive operator can trigger this but later it should support others 2. indexes the PlanNodes with post-order traverse, so when later the PhysicalPlanVisitor hits one of these pipeline breaker PlanNode, it knows what index used to retrieve the PipelineBreaker results 3. host the pipeline breaker results in a map all 3 should be separate (especially 2, which should be done individually also for stats collecting purpose) -- 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