-
Notifications
You must be signed in to change notification settings - Fork 1.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Handle interrupt gracefully for Lucene mutable index #11558
Handle interrupt gracefully for Lucene mutable index #11558
Conversation
Codecov Report
@@ Coverage Diff @@
## master #11558 +/- ##
=============================================
- Coverage 63.07% 14.49% -48.58%
+ Complexity 1108 201 -907
=============================================
Files 2325 2327 +2
Lines 124772 125001 +229
Branches 19052 19079 +27
=============================================
- Hits 78694 18124 -60570
- Misses 40480 105343 +64863
+ Partials 5598 1534 -4064
Flags with carried forward coverage won't be shown. Click here to find out more.
... and 1505 files with indirect coverage changes 📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more |
...java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java
Outdated
Show resolved
Hide resolved
...java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java
Outdated
Show resolved
Hide resolved
...java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java
Outdated
Show resolved
Hide resolved
7fe7984
to
ec97d37
Compare
/** | ||
* 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 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why not just delegate to worker thread pool itself ?
I doubt if this approach actually ensures much or provides a benefit since due to context switching (and the fact that number of worker threads is 2x the number of cores based on the current code), having a searcher thread for each corresponding worker thread may not lead to the benefit we are looking for.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this might be due to my unfamiliarity with the code, but I didn't find the worker thread pool to be accessible from RealtimeLuceneTextIndex
. Could you provide some suggestion on how to delegate?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we use worker thread pool then we would have two types of threads in the thread-pool: one that waits on the future and the other that actually runs the future. (correct me if I am wrong)
That could lead to deadlock kind of scenarios when we have many concurrent queries. We ran into similar issues with Multistage engine in the past.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think this standard coding pattern of outer / parent thread waiting on future completion and child thread actually doing the future / work should lead to deadlock unless we have done something in the code for them to compete for same resource.
In any case, this isn't a blocker. My main concern was primarily that we may be starting too many threads and it might just have just general system performance impacts. If the deadlock concern is real, then may be this is fine.
Within LinkedIn, we don't currently use text search in realtime so can't speak to it for now.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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)
Edit: It's not technically a deadlock. I use the term loosely but I guess you get the idea. We would end up underutilizing CPU cores and query latency would increase/timeout.
...java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java
Outdated
Show resolved
Hide resolved
...org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneDocIdCollector.java
Outdated
Show resolved
Hide resolved
try { | ||
return searchFuture.get(); | ||
} catch (InterruptedException e) { | ||
docIDCollector.markShouldCancel(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just for clarification - does doing this prevent index corruption like mentioned in PR description ? This is still in the context of searcher thread that got interrupted right ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The IE being caught is from the parent thread getting interrupted (not the searcher thread) while waiting for the searchFuture to be done
private static ExecutorService _executorService; | ||
|
||
private RealtimeLuceneTextIndexSearcherPool(int size) { | ||
_executorService = new ThreadPoolExecutor(0, size, 500, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just to confirm, the ThreadPoolExecutor is not going to create any thread (and keep them available) before submission of search tasks ?
The reason being that if that happens then there may be some perf impact of creating 4x numCore threads (2x coming from worker pool and 2x coming from this new pool).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, exactly. Threads are created as needed (up to numWorkerThreads
), and kept alive for 500ms once idle to prevent re-instantiation overhead during for high QPS use cases
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Overall LGTM. Let's just ensure that the new pool won't spin up unnecessary threads
@siddharthteotia re: spinning up threads, see this comment Let me know if there's any other concerns, or could you help merge this? |
This addresses #11402.
When a thread is interrupted when searching the consuming segment, the underlying FSDirectory used by the IndexWriter which the SearcherManager was created with can be corrupted. To ensure the index is never corrupted this PR changes the the search to be executed in a child thread and the interrupt is handled in the current thread by canceling the search gracefully.
This was tested on an internal cluster, query timeouts no longer cause massive numbers of 'failed to index' logs and exceptions reported in the aforementioned issue are no longer present.
Some additional background on this behavior is here: apache/lucene#9309
tags:
bugfix