walterddr commented on code in PR #10779:
URL: https://github.com/apache/pinot/pull/10779#discussion_r1207492947
##########
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();
+ PipelineBreakerVisitor.visitPlanRoot(distributedStagePlan.getStageRoot(),
pipelineBreakerContext);
+ if (pipelineBreakerContext.getPipelineBreakerMap().size() > 0) {
+ PlanNode stageRoot = distributedStagePlan.getStageRoot();
+ PlanRequestContext planRequestContext =
+ new PlanRequestContext(_mailboxService, requestId,
stageRoot.getPlanFragmentId(), timeoutMs, deadlineMs,
+ distributedStagePlan.getServer(),
distributedStagePlan.getStageMetadata(), null, isTraceEnabled);
+ Map<Integer, List<TransferableBlock>> resultMap =
+ PipelineBreakerExecutionUtils.execute(scheduler,
pipelineBreakerContext, planRequestContext);
+ Preconditions.checkState(!resultMap.containsKey(-1), "Pipeline Breaker
received error block!");
Review Comment:
done. throwing instead
--
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]