xiangfu0 commented on code in PR #18047: URL: https://github.com/apache/pinot/pull/18047#discussion_r3042923916
########## pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QuerySchedulerThreadPoolConfigChangeListener.java: ########## @@ -0,0 +1,84 @@ +/** + * 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.scheduler; + +import java.util.Map; +import java.util.Set; +import org.apache.pinot.core.query.scheduler.resources.ResourceManager; +import org.apache.pinot.spi.config.provider.PinotClusterConfigChangeListener; +import org.apache.pinot.spi.utils.CommonConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Listens for cluster config changes to dynamically resize the query scheduler thread pools + * ({@code query_runner_threads} and {@code query_worker_threads}). + */ +public class QuerySchedulerThreadPoolConfigChangeListener implements PinotClusterConfigChangeListener { + private static final Logger LOGGER = LoggerFactory.getLogger(QuerySchedulerThreadPoolConfigChangeListener.class); + + static final String QUERY_RUNNER_THREADS_KEY = + CommonConstants.PINOT_QUERY_SCHEDULER_PREFIX + "." + ResourceManager.QUERY_RUNNER_CONFIG_KEY; + static final String QUERY_WORKER_THREADS_KEY = + CommonConstants.PINOT_QUERY_SCHEDULER_PREFIX + "." + ResourceManager.QUERY_WORKER_CONFIG_KEY; + + private final ResourceManager _resourceManager; + + public QuerySchedulerThreadPoolConfigChangeListener(ResourceManager resourceManager) { + _resourceManager = resourceManager; + } + + @Override + public void onChange(Set<String> changedConfigs, Map<String, String> clusterConfigs) { + if (!changedConfigs.contains(QUERY_RUNNER_THREADS_KEY) + && !changedConfigs.contains(QUERY_WORKER_THREADS_KEY)) { + return; + } + + int newRunnerThreads = _resourceManager.getNumQueryRunnerThreads(); + int newWorkerThreads = _resourceManager.getNumQueryWorkerThreads(); + + if (changedConfigs.contains(QUERY_RUNNER_THREADS_KEY) && clusterConfigs.containsKey(QUERY_RUNNER_THREADS_KEY)) { Review Comment: DefaultClusterConfigChangeHandler reports deleted cluster keys in changedConfigs, but this listener only reparses a value when the key is still present in clusterConfigs. If an operator removes pinot.query.scheduler.query_runner_threads or pinot.query.scheduler.query_worker_threads to roll back the override (expecting the value to be rollback to the system default) resizeThreadPools is invoked with the current value instead, so the last live size becomes sticky until another explicit number is pushed. ########## pinot-server/src/main/java/org/apache/pinot/server/starter/helix/BaseServerStarter.java: ########## @@ -803,6 +804,9 @@ public void start() } _clusterConfigChangeHandler.registerClusterConfigChangeListener(_segmentOperationsThrottlerSet); _clusterConfigChangeHandler.registerClusterConfigChangeListener(keepPipelineBreakerStatsPredicate); + _clusterConfigChangeHandler.registerClusterConfigChangeListener( Review Comment: There is another usage of `query_worker_threads` at line 707-711: ``` // Create a thread pool used for mutable lucene index searches, with size based on query_worker_threads config LOGGER.info("Initializing lucene searcher thread pool"); int queryWorkerThreads = _serverConf.getProperty(ResourceManager.QUERY_WORKER_CONFIG_KEY, ResourceManager.DEFAULT_QUERY_WORKER_THREADS); _realtimeLuceneTextIndexSearcherPool = RealtimeLuceneTextIndexSearcherPool.init(queryWorkerThreads); ``` -- 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]
