walterddr commented on code in PR #10332:
URL: https://github.com/apache/pinot/pull/10332#discussion_r1119547454


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -99,28 +112,43 @@ public int submit(long requestId, QueryPlan queryPlan, 
long timeoutMs, Map<Strin
           String host = serverInstance.getHostname();
           int servicePort = serverInstance.getQueryServicePort();
           DispatchClient client = getOrCreateDispatchClient(host, servicePort);
-          Worker.QueryResponse response = 
client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                  
QueryPlanSerDeUtils.serialize(constructDistributedStagePlan(queryPlan, stageId, 
serverInstance)))
-              .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, 
String.valueOf(requestId))
-              .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, 
String.valueOf(timeoutMs))
-              .putAllMetadata(queryOptions).build());
-
+          dispatchCalls++;
+          _executorService.submit(() -> {
+            client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
+                
QueryPlanSerDeUtils.serialize(constructDistributedStagePlan(queryPlan, stageId, 
serverInstance)))
+                .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, 
String.valueOf(requestId))
+                .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, 
String.valueOf(timeoutMs))
+                .putAllMetadata(queryOptions).build(), stageId, 
serverInstance, deadline, callbacks::offer);
+          });
+        }
+      }
+    }
+    int returnedDispatchCalls = 0;

Review Comment:
   ```suggestion
       int successfulDispatchCalls = 0;
   ```



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java:
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.query.service.dispatch;
+
+import io.grpc.Deadline;
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import java.util.function.Consumer;
+import org.apache.pinot.common.proto.PinotQueryWorkerGrpc;
+import org.apache.pinot.common.proto.Worker;
+import org.apache.pinot.query.routing.VirtualServer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Dispatches a query plan to given a {@link VirtualServer}. Each {@link 
DispatchClient} has its own gRPC Channel and
+ * Client Stub.
+ * TODO: It might be neater to implement pooling at the client level. Two 
options: (1) Pass a channel provider and
+ *       let that take care of pooling. (2) Create a DispatchClient interface 
and implement pooled/non-pooled versions.
+ */
+class DispatchClient {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DispatchClient.class);
+  private ManagedChannel _channel;
+  private PinotQueryWorkerGrpc.PinotQueryWorkerStub _dispatchStub;

Review Comment:
   both final?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -88,6 +98,9 @@ public ResultTable submitAndReduce(long requestId, QueryPlan 
queryPlan,
   public int submit(long requestId, QueryPlan queryPlan, long timeoutMs, 
Map<String, String> queryOptions)
       throws Exception {
     int reduceStageId = -1;
+    Deadline deadline = Deadline.after(timeoutMs, TimeUnit.MILLISECONDS);
+    BlockingQueue<AsyncQueryDispatchResponse> callbacks = new 
LinkedBlockingQueue<>();

Review Comment:
   nit:
   ```suggestion
       BlockingQueue<AsyncQueryDispatchResponse> dispatchCallbacks = new 
LinkedBlockingQueue<>();
   ```



-- 
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

Reply via email to