ankitsultana commented on code in PR #11558: URL: https://github.com/apache/pinot/pull/11558#discussion_r1326589706
########## pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndexSearcherPool.java: ########## @@ -0,0 +1,56 @@ +/** + * 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.segment.local.realtime.impl.invertedindex; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + + +/** + * This class manages a thread pool used for searching over realtime Lucene segments by {@link RealtimeLuceneTextIndex}. + * The pool max size is equivalent to pinot.query.scheduler.query_worker_threads to ensure each worker thread can have + * an accompanying Lucene searcher thread if needed. init() is called in BaseServerStarter to avoid creating a Review Comment: Deadlock can very easily happen if `numConcurrentQueries = workerThreadPoolSize/2`. In that case all threads in the thread-pool will be utilized, and the new queries would get submitted again to the thread pool and they'll be at the head of the queue. These new queries will then get blocked on the future once some threads finish in the thread-pool, and the future wouldn't complete until some other query finishes. So for each such query we'll have one thread needlessly waiting on resources to be freed even though we have capacity to run them. For Multistage engine this was a critical fix (I think we had a ~5-10% reliability improvement). With shared threads we were hitting this issue once every few hours. (my memory might not be correct on the exact improvement number) -- 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]
