Jackie-Jiang commented on code in PR #16147: URL: https://github.com/apache/pinot/pull/16147#discussion_r2164756890
########## pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java: ########## @@ -133,6 +135,64 @@ public ImmutableRoaringBitmap getDictIds(String searchQuery) { @Override public MutableRoaringBitmap getDocIds(String searchQuery) { + return getDocIdsWithoutOptions(searchQuery); + } + + @Override + public MutableRoaringBitmap getDocIds(String searchQuery, @Nullable String optionsString) { + if (optionsString != null && !optionsString.trim().isEmpty()) { + LuceneTextIndexUtils.LuceneTextIndexOptions options = LuceneTextIndexUtils.createOptions(optionsString); + Map<String, String> optionsMap = options.getOptions(); + if (!optionsMap.isEmpty()) { + return getDocIdsWithOptions(searchQuery, options); + } + } + return getDocIdsWithoutOptions(searchQuery); + } + + private MutableRoaringBitmap getDocIdsWithOptions(String actualQuery, + LuceneTextIndexUtils.LuceneTextIndexOptions options) { + MutableRoaringBitmap docIDs = new MutableRoaringBitmap(); + 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. + // See https://github.com/apache/lucene/issues/3315 and https://github.com/apache/lucene/issues/9309 + Callable<MutableRoaringBitmap> searchCallable = () -> { + IndexSearcher indexSearcher = null; + try { + Query query = LuceneTextIndexUtils.createQueryParserWithOptions(actualQuery, options, _column, _analyzer); + 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()); + } + } + }; + Future<MutableRoaringBitmap> searchFuture = SEARCHER_POOL.getExecutorService().submit(searchCallable); + try { + return searchFuture.get(); + } catch (InterruptedException e) { + docIDCollector.markShouldCancel(); + LOGGER.warn("TEXT_MATCH query interrupted while querying the consuming segment {}, column {}, search query {}", Review Comment: Same here, and other index readers. ########## pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/MultiColumnRealtimeLuceneTextIndex.java: ########## @@ -149,8 +150,66 @@ public MutableRoaringBitmap getDocIds(String searchQuery) { throw new UnsupportedOperationException("Multi-column text index requires column name to query!"); } + @Override + public MutableRoaringBitmap getDocIds(String column, String searchQuery, @Nullable String optionsString) { Review Comment: The same logic is repeated in 4 places. Let's add a TODO to add a common base class in the future ########## pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/MultiColumnRealtimeLuceneTextIndex.java: ########## @@ -149,8 +150,66 @@ public MutableRoaringBitmap getDocIds(String searchQuery) { throw new UnsupportedOperationException("Multi-column text index requires column name to query!"); } + @Override + public MutableRoaringBitmap getDocIds(String column, String searchQuery, @Nullable String optionsString) { + if (optionsString != null && !optionsString.trim().isEmpty()) { + LuceneTextIndexUtils.LuceneTextIndexOptions options = LuceneTextIndexUtils.createOptions(optionsString); + Map<String, String> optionsMap = options.getOptions(); + if (!optionsMap.isEmpty()) { + return getDocIdsWithOptions(column, searchQuery, options); + } + } + return getDocIdsWithoutOptions(column, searchQuery); + } + @Override public MutableRoaringBitmap getDocIds(String column, String searchQuery) { + return getDocIdsWithoutOptions(column, searchQuery); + } + + private MutableRoaringBitmap getDocIdsWithOptions(String column, String actualQuery, + LuceneTextIndexUtils.LuceneTextIndexOptions options) { + MutableRoaringBitmap docIDs = new MutableRoaringBitmap(); + 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. + // See https://github.com/apache/lucene/issues/3315 and https://github.com/apache/lucene/issues/9309 + Callable<MutableRoaringBitmap> searchCallable = () -> { + IndexSearcher indexSearcher = null; + try { + Query query = LuceneTextIndexUtils.createQueryParserWithOptions(actualQuery, options, column, _analyzer); + 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 columns {}, exception {}", + _columns, e.getMessage()); + } + } + }; + Future<MutableRoaringBitmap> searchFuture = SEARCHER_POOL.getExecutorService().submit(searchCallable); + try { + return searchFuture.get(); + } catch (InterruptedException e) { + docIDCollector.markShouldCancel(); + LOGGER.warn("TEXT_MATCH query interrupted while querying the consuming segment {}, columns {}, search query {}", + _segmentName, _columns, actualQuery); + throw new RuntimeException("TEXT_MATCH query interrupted while querying the consuming segment"); + } catch (Exception e) { + LOGGER.error("Failed while searching the realtime text index for segment {}, columns {}, search query {}," Review Comment: Same here. Add the context into the `RuntimeException` instead of logging error ########## pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/MultiColumnRealtimeLuceneTextIndex.java: ########## @@ -149,8 +150,66 @@ public MutableRoaringBitmap getDocIds(String searchQuery) { throw new UnsupportedOperationException("Multi-column text index requires column name to query!"); } + @Override + public MutableRoaringBitmap getDocIds(String column, String searchQuery, @Nullable String optionsString) { + if (optionsString != null && !optionsString.trim().isEmpty()) { + LuceneTextIndexUtils.LuceneTextIndexOptions options = LuceneTextIndexUtils.createOptions(optionsString); + Map<String, String> optionsMap = options.getOptions(); + if (!optionsMap.isEmpty()) { + return getDocIdsWithOptions(column, searchQuery, options); + } + } + return getDocIdsWithoutOptions(column, searchQuery); + } + @Override public MutableRoaringBitmap getDocIds(String column, String searchQuery) { + return getDocIdsWithoutOptions(column, searchQuery); + } + + private MutableRoaringBitmap getDocIdsWithOptions(String column, String actualQuery, + LuceneTextIndexUtils.LuceneTextIndexOptions options) { + MutableRoaringBitmap docIDs = new MutableRoaringBitmap(); + 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. + // See https://github.com/apache/lucene/issues/3315 and https://github.com/apache/lucene/issues/9309 + Callable<MutableRoaringBitmap> searchCallable = () -> { + IndexSearcher indexSearcher = null; + try { + Query query = LuceneTextIndexUtils.createQueryParserWithOptions(actualQuery, options, column, _analyzer); + 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 columns {}, exception {}", + _columns, e.getMessage()); + } + } + }; + Future<MutableRoaringBitmap> searchFuture = SEARCHER_POOL.getExecutorService().submit(searchCallable); + try { + return searchFuture.get(); + } catch (InterruptedException e) { + docIDCollector.markShouldCancel(); + LOGGER.warn("TEXT_MATCH query interrupted while querying the consuming segment {}, columns {}, search query {}", Review Comment: When we throw out the exception, no need to log warning/error. We usually log the exception when catching/handling it -- 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