yashmayya commented on code in PR #15445:
URL: https://github.com/apache/pinot/pull/15445#discussion_r2044516915
##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java:
##########
@@ -159,37 +164,49 @@ public void submit(Worker.QueryRequest request,
StreamObserver<Worker.QueryRespo
return;
}
- try (QueryThreadContext.CloseableContext queryTlClosable =
QueryThreadContext.openFromRequestMetadata(reqMetadata);
- QueryThreadContext.CloseableContext mseTlCloseable =
MseWorkerThreadContext.open()) {
+ try (QueryThreadContext.CloseableContext qClosable =
QueryThreadContext.openFromRequestMetadata(reqMetadata);
+ QueryThreadContext.CloseableContext mseCloseable =
MseWorkerThreadContext.open()) {
+
long requestId = QueryThreadContext.getRequestId();
QueryThreadContext.setQueryEngine("mse");
- Tracing.ThreadAccountantOps.setupRunner(Long.toString(requestId),
ThreadExecutionContext.TaskType.MSE);
- ThreadExecutionContext parentContext =
Tracing.getThreadAccountant().getThreadExecutionContext();
- try {
- forEachStage(request,
- (stagePlan, workerMetadata) -> {
+ // Submit the stage for each worker
+ List<CompletableFuture<List<Object>>> futures =
forEachStageAndWorker(request,
+ (stagePlan, workerMetadata) -> {
+ Tracing.ThreadAccountantOps.setupRunner(Long.toString(requestId),
ThreadExecutionContext.TaskType.MSE);
+ ThreadExecutionContext parentContext =
Tracing.getThreadAccountant().getThreadExecutionContext();
+
+ try {
_queryRunner.processQuery(workerMetadata, stagePlan,
reqMetadata, parentContext);
- return null;
- },
- (ignored) -> {
- });
- } catch (ExecutionException | InterruptedException | TimeoutException |
RuntimeException e) {
- LOGGER.error("Caught exception while submitting request: {}",
requestId, e);
- String errorMsg = "Caught exception while submitting request: " +
e.getMessage();
- responseObserver.onNext(Worker.QueryResponse.newBuilder()
-
.putMetadata(CommonConstants.Query.Response.ServerResponseStatus.STATUS_ERROR,
errorMsg)
- .build());
- responseObserver.onCompleted();
- return;
- } finally {
- Tracing.ThreadAccountantOps.clear();
- }
- responseObserver.onNext(
- Worker.QueryResponse.newBuilder()
-
.putMetadata(CommonConstants.Query.Response.ServerResponseStatus.STATUS_OK, "")
+ } finally {
+ Tracing.ThreadAccountantOps.clear();
+ }
+ return null;
+ });
+
+ // A completable future that will finish when all submit task finish or
on timoeut
+ CompletableFuture<Void> allCompleted =
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
+ .orTimeout(QueryThreadContext.getDeadlineMs(),
TimeUnit.MILLISECONDS);
+ // When this future completes, notify the broker.
+ allCompleted.handle((result, error) -> {
Review Comment:
Why not `whenComplete` instead of `handle`?
--
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]