albertobastos commented on code in PR #15143: URL: https://github.com/apache/pinot/pull/15143#discussion_r1975237338
########## 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: Yeah, I can see that. Changed it to just reuse the same parameter adding a multiplier (for now I set it to 4x, maybe too conservative). bbb195f4b4ead02032bf4060f900f444698d1c7c ########## 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: You're right. When the SSE runs by order of a MSE leaf stage, it already receives its own executorService from it properly decorated. Removed bbb195f4b4ead02032bf4060f900f444698d1c7c ########## 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: Didn't know about that provider system, but makes total sense to keep using it. Changed bbb195f4b4ead02032bf4060f900f444698d1c7c ########## pinot-core/src/test/java/org/apache/pinot/core/query/executor/MaxTasksExecutorTest.java: ########## @@ -0,0 +1,74 @@ +/** + * 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.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import org.testng.annotations.Test; + +import static org.testng.AssertJUnit.*; + + +public class MaxTasksExecutorTest { + + private static final int MAX_TASKS = 5; + + @Test + public void testExecutor() { + MaxTasksExecutor ex = new MaxTasksExecutor(MAX_TASKS, Executors.newCachedThreadPool()); + + final Semaphore sem1 = new Semaphore(0); + final Semaphore sem2 = new Semaphore(0); + final Semaphore sem3 = new Semaphore(0); + + for (int i = 1; i <= MAX_TASKS; i++) { + ex.execute(() -> { + sem2.release(); + sem1.acquireUninterruptibly(); + sem3.release(); + }); + } + + try { + ex.execute(() -> { + }); + fail("Should not allow more than " + MAX_TASKS + " threads"); + } catch (Exception e) { + // as expected + } + + for (int i = MAX_TASKS; i > 0; i--) { + sem1.release(); + } + + try { + sem3.tryAcquire(MAX_TASKS, 10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + fail("Interrupted waiting for tasks to complete"); + } + + try { + ex.execute(() -> { + }); + } catch (Exception e) { + fail("Exception submitting task after release: " + e); + } + } Review Comment: I can't see how could we have two separate tests as the IllegalStageException is the way to verify that the executor is capping the amount of threads as expected. But yes, the test tried too hard and it is enough with having a simple hardlimit=1 for the sake of it. bbb195f4b4ead02032bf4060f900f444698d1c7c ########## pinot-core/src/test/java/org/apache/pinot/core/query/executor/MaxTasksExecutorTest.java: ########## @@ -0,0 +1,74 @@ +/** + * 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.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import org.testng.annotations.Test; + +import static org.testng.AssertJUnit.*; + + +public class MaxTasksExecutorTest { + + private static final int MAX_TASKS = 5; + + @Test + public void testExecutor() { + MaxTasksExecutor ex = new MaxTasksExecutor(MAX_TASKS, Executors.newCachedThreadPool()); Review Comment: Yup, overthought too much. bbb195f4b4ead02032bf4060f900f444698d1c7c -- 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