yashmayya commented on code in PR #15143:
URL: https://github.com/apache/pinot/pull/15143#discussion_r1973603370


##########
pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java:
##########
@@ -1065,6 +1065,9 @@ public static class SegmentCompletionProtocol {
 
     public static final String CONFIG_OF_CURRENT_DATA_TABLE_VERSION = 
"pinot.server.instance.currentDataTableVersion";
 
+    public static final String CONFIG_OF_MSE_THREADS_HARD_LIMIT = 
"pinot.server.mse.threads.hardLimit";

Review Comment:
   I'm wondering if we can re-use the cluster configuration 
`pinot.beta.multistage.engine.max.server.query.threads` instead here? Maybe we 
could use some multiple of it (2x for instance) to be safe since we'll be 
killing queries beyond that limit. 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java:
##########
@@ -177,6 +178,12 @@ public void init(PinotConfiguration config, 
InstanceDataManager instanceDataMana
     _executorService =
         ExecutorServiceUtils.create(config, 
Server.MULTISTAGE_EXECUTOR_CONFIG_PREFIX, "query-runner-on-" + port,
             Server.DEFAULT_MULTISTAGE_EXECUTOR_TYPE);
+
+    int maxThreads = 
config.getProperty(Server.CONFIG_OF_MSE_THREADS_HARD_LIMIT, 
Server.DEFAULT_MSE_THREADS_HARD_LIMIT);
+    if (maxThreads > 0) {
+      _executorService = new MaxTasksExecutor(maxThreads, _executorService);
+    }

Review Comment:
   Can't we use the existing executor service configuration mechanism (based on 
`MULTISTAGE_EXECUTOR_CONFIG_PREFIX...`) and create a new configurable executor 
instead of introducing an additional configuration type and this decorated 
executor? I can't think of any reasons we'd want both the mechanisms 
independently?



##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/MdcQueryExecutor.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.query.executor;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import org.apache.pinot.spi.executor.DecoratorExecutorService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A delegator executor service that sets MDC context for the query.
+ *
+ * By using this executor, all tasks submitted to the executor will have the 
MDC context set to the query context.
+ * This is easier and safer to apply than setting the MDC context manually for 
each task.
+ * You can read more about MDC in <a 
href="https://www.baeldung.com/mdc-in-log4j-2-logback";>this baeldung post</a>
+ *
+ * TODO: Convert this class and its usages into an Executor instead of an 
ExecutorService
+ *
+ */
+public abstract class MdcQueryExecutor extends DecoratorExecutorService {

Review Comment:
   This is a duplicate of the executor added in 
https://github.com/apache/pinot/pull/15072?



##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java:
##########
@@ -142,7 +144,33 @@ public synchronized void shutDown() {
   @Override
   public InstanceResponseBlock execute(ServerQueryRequest queryRequest, 
ExecutorService executorService,
       @Nullable ResultsBlockStreamer streamer) {
-    MdcExecutor mdcExecutor = new MdcExecutor(executorService) {
+
+    ExecutorService decoratedExecutor = getExecutorService(queryRequest, 
executorService);

Review Comment:
   This is the single-stage query engine's query executor. We shouldn't be 
using the multi-stage engine specific property and limits here. 



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