ankitsultana commented on code in PR #11558:
URL: https://github.com/apache/pinot/pull/11558#discussion_r1321683982


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java:
##########
@@ -109,27 +112,46 @@ public ImmutableRoaringBitmap getDictIds(String 
searchQuery) {
   @Override
   public MutableRoaringBitmap getDocIds(String searchQuery) {
     MutableRoaringBitmap docIDs = new MutableRoaringBitmap();
-    Collector docIDCollector = new RealtimeLuceneDocIdCollector(docIDs);
-    IndexSearcher indexSearcher = null;
-    try {
-      Query query = new QueryParser(_column, _analyzer).parse(searchQuery);
-      indexSearcher = _searcherManager.acquire();
-      indexSearcher.search(query, docIDCollector);
-      return getPinotDocIds(indexSearcher, docIDs);
-    } catch (Exception e) {
-      LOGGER
-          .error("Failed while searching the realtime text index for column 
{}, search query {}, exception {}", _column,
-              searchQuery, e.getMessage());
-      throw new RuntimeException(e);
-    } finally {
+    RealtimeLuceneDocIdCollector docIDCollector = new 
RealtimeLuceneDocIdCollector(docIDs);
+    // A thread interrupt during indexSearcher.search() can break the 
underlying FSDirectory used by the IndexWriter
+    // which the SearcherManager is created with. To ensure the index is never 
corrupted the search is executed
+    // in a child thread and the interrupt is handled in the current thread by 
canceling the search gracefully
+    Callable<MutableRoaringBitmap> searchCallable = () -> {
+      IndexSearcher indexSearcher = null;
       try {
-        if (indexSearcher != null) {
-          _searcherManager.release(indexSearcher);
+        Query query = new QueryParser(_column, _analyzer).parse(searchQuery);
+        indexSearcher = _searcherManager.acquire();
+        indexSearcher.search(query, docIDCollector);
+        return getPinotDocIds(indexSearcher, docIDs);
+      } finally {
+        try {
+          if (indexSearcher != null) {
+            _searcherManager.release(indexSearcher);
+          }
+        } catch (Exception e) {
+          LOGGER.error(
+              "Failed while releasing the searcher manager for realtime text 
index for column {}, exception {}",
+              _column, e.getMessage());
+        }
+      }
+    };
+    FutureTask<MutableRoaringBitmap> searchTask = new 
FutureTask<>(searchCallable);
+    Thread searcherThread = new Thread(searchTask);
+    searcherThread.start();

Review Comment:
   Starting a new thread for each query is not ideal. You can configure a 
ThreadPoolExecutor to run the Futures instead?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java:
##########
@@ -109,27 +112,46 @@ public ImmutableRoaringBitmap getDictIds(String 
searchQuery) {
   @Override
   public MutableRoaringBitmap getDocIds(String searchQuery) {
     MutableRoaringBitmap docIDs = new MutableRoaringBitmap();
-    Collector docIDCollector = new RealtimeLuceneDocIdCollector(docIDs);
-    IndexSearcher indexSearcher = null;
-    try {
-      Query query = new QueryParser(_column, _analyzer).parse(searchQuery);
-      indexSearcher = _searcherManager.acquire();
-      indexSearcher.search(query, docIDCollector);
-      return getPinotDocIds(indexSearcher, docIDs);
-    } catch (Exception e) {
-      LOGGER
-          .error("Failed while searching the realtime text index for column 
{}, search query {}, exception {}", _column,
-              searchQuery, e.getMessage());
-      throw new RuntimeException(e);
-    } finally {
+    RealtimeLuceneDocIdCollector docIDCollector = new 
RealtimeLuceneDocIdCollector(docIDs);
+    // A thread interrupt during indexSearcher.search() can break the 
underlying FSDirectory used by the IndexWriter
+    // which the SearcherManager is created with. To ensure the index is never 
corrupted the search is executed
+    // in a child thread and the interrupt is handled in the current thread by 
canceling the search gracefully
+    Callable<MutableRoaringBitmap> searchCallable = () -> {
+      IndexSearcher indexSearcher = null;
       try {
-        if (indexSearcher != null) {
-          _searcherManager.release(indexSearcher);
+        Query query = new QueryParser(_column, _analyzer).parse(searchQuery);
+        indexSearcher = _searcherManager.acquire();
+        indexSearcher.search(query, docIDCollector);
+        return getPinotDocIds(indexSearcher, docIDs);
+      } finally {
+        try {
+          if (indexSearcher != null) {
+            _searcherManager.release(indexSearcher);
+          }
+        } catch (Exception e) {
+          LOGGER.error(
+              "Failed while releasing the searcher manager for realtime text 
index for column {}, exception {}",
+              _column, e.getMessage());
+        }
+      }
+    };
+    FutureTask<MutableRoaringBitmap> searchTask = new 
FutureTask<>(searchCallable);
+    Thread searcherThread = new Thread(searchTask);
+    searcherThread.start();
+    try {
+      while (!searchTask.isDone()) {

Review Comment:
   Could the same behavior be achieved by using `Object.wait()`?
   
   Essentially you create a dummy object, say "LuceneNotifier" and pass it to 
the search callable. The search callable will call `luceneNotifier.notify()` in 
the finally block. And the while loop would be replaced with something like 
this:
   
   ```
   try {
     luceneNotifier.wait();
     // code reaches here if notify was call after success/failure
   } catch (InterruptedException e) {
     // this gets called if the thread was interrupted
     docIdCollector.markShouldCancel();
   }
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java:
##########
@@ -109,27 +112,46 @@ public ImmutableRoaringBitmap getDictIds(String 
searchQuery) {
   @Override
   public MutableRoaringBitmap getDocIds(String searchQuery) {
     MutableRoaringBitmap docIDs = new MutableRoaringBitmap();
-    Collector docIDCollector = new RealtimeLuceneDocIdCollector(docIDs);
-    IndexSearcher indexSearcher = null;
-    try {
-      Query query = new QueryParser(_column, _analyzer).parse(searchQuery);
-      indexSearcher = _searcherManager.acquire();
-      indexSearcher.search(query, docIDCollector);
-      return getPinotDocIds(indexSearcher, docIDs);
-    } catch (Exception e) {
-      LOGGER
-          .error("Failed while searching the realtime text index for column 
{}, search query {}, exception {}", _column,
-              searchQuery, e.getMessage());
-      throw new RuntimeException(e);
-    } finally {
+    RealtimeLuceneDocIdCollector docIDCollector = new 
RealtimeLuceneDocIdCollector(docIDs);
+    // A thread interrupt during indexSearcher.search() can break the 
underlying FSDirectory used by the IndexWriter
+    // which the SearcherManager is created with. To ensure the index is never 
corrupted the search is executed
+    // in a child thread and the interrupt is handled in the current thread by 
canceling the search gracefully

Review Comment:
   would be good to link this in the comments here: 
https://github.com/apache/lucene/issues/9309



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