Re: [PR] Performance improvements to use read lock to access LRUQueryCache [lucene]

2024-04-28 Thread via GitHub


boicehuang commented on code in PR #13306:
URL: https://github.com/apache/lucene/pull/13306#discussion_r1582091772


##
lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java:
##
@@ -265,7 +269,6 @@ boolean requiresEviction() {
   }
 
   CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) {
-assert lock.isHeldByCurrentThread();
 assert key instanceof BoostQuery == false;
 assert key instanceof ConstantScoreQuery == false;
 final IndexReader.CacheKey readerKey = cacheHelper.getKey();

Review Comment:
   I drew 2 flow charts and tried to describe all the concurrency paths.
   ![LRUCache 
优化结构2](https://github.com/apache/lucene/assets/4607177/8233fd85-98a3-435f-b4ee-2214a80e4cc1)
   



-- 
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: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Performance improvements to use read lock to access LRUQueryCache [lucene]

2024-04-28 Thread via GitHub


boicehuang commented on code in PR #13306:
URL: https://github.com/apache/lucene/pull/13306#discussion_r1582091772


##
lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java:
##
@@ -265,7 +269,6 @@ boolean requiresEviction() {
   }
 
   CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) {
-assert lock.isHeldByCurrentThread();
 assert key instanceof BoostQuery == false;
 assert key instanceof ConstantScoreQuery == false;
 final IndexReader.CacheKey readerKey = cacheHelper.getKey();

Review Comment:
   I drew 2 flow charts and tried to describe all the concurrency paths.
   ![LRUCache 
优化结构2](https://github.com/apache/lucene/assets/4607177/c3c104b3-1ba8-4885-8e0e-d0d787d383f3)
   
   



-- 
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: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Performance improvements to use read lock to access LRUQueryCache [lucene]

2024-04-28 Thread via GitHub


boicehuang commented on code in PR #13306:
URL: https://github.com/apache/lucene/pull/13306#discussion_r1582091772


##
lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java:
##
@@ -265,7 +269,6 @@ boolean requiresEviction() {
   }
 
   CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) {
-assert lock.isHeldByCurrentThread();
 assert key instanceof BoostQuery == false;
 assert key instanceof ConstantScoreQuery == false;
 final IndexReader.CacheKey readerKey = cacheHelper.getKey();

Review Comment:
   I think we still need `IdentityHashMap` to clear the cache of expired 
queries. I drew 2 flow charts and tried to describe all the concurrency paths 
of the newest modification.
   ![LRUCache 
优化结构2](https://github.com/apache/lucene/assets/4607177/c3c104b3-1ba8-4885-8e0e-d0d787d383f3)
   
   



-- 
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: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Decouple within-query concurrency from the index's segment geometry [LUCENE-8675] [lucene]

2024-04-28 Thread via GitHub


jpountz commented on issue #9721:
URL: https://github.com/apache/lucene/issues/9721#issuecomment-2081643392

   > but I guess as an execution strategy it kind of made sense to me -- is it 
really necessary to clone Scorers? Could we create new ones for each 
searcher-segment or do we think that is too costly due to initialization costs?
   
   It depends on queries. For term queries, duplicating the overhead of looking 
up terms in the terms dict may be ok, but for multi-term queries and point 
queries that often compute the bit set of matches of the whole segment, this 
could significantly hurt throughput. Maybe it doesn't have to be this way for 
the first iteration (progress over perfection), but this feels important to me 
so that we don't have weird recommendations like "only enable intra-segment 
concurrency if you don't use multi-term or point queries".
   
   Related: in the long term, I'd like inter-segment search concurrency to be 
enabled by default (#11523 maybe something else we should consider for 10.0), 
and ideally intra-segment search concurrency too, which is another reason why I 
care about avoiding introducing major downsides vs. single-threaded search.


-- 
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: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Can IndexSearcher enable search concurrency by default? [LUCENE-10487] [lucene]

2024-04-28 Thread via GitHub


jpountz commented on issue #11523:
URL: https://github.com/apache/lucene/issues/11523#issuecomment-2081644183

   Optimistically adding this issue to the 10.0 milestone.


-- 
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: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Make Weight#scorerSupplier abstract, Weight#scorer final [lucene]

2024-04-28 Thread via GitHub


jpountz commented on code in PR #13319:
URL: https://github.com/apache/lucene/pull/13319#discussion_r1582371473


##
lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java:
##
@@ -136,7 +136,18 @@ public float matchCost() {
   }
 };
 
-return new ConstantScoreScorer(this, score(), scoreMode, iterator);
+final var scorer = new ConstantScoreScorer(this, score(), scoreMode, 
iterator);
+return new ScorerSupplier() {

Review Comment:
   Since we seem to be doing this in lots of places, we could introduce a 
`DefaultScorerSupplier` class that takes a `Scorer` as a ctor argument, 
alongside `DefaultBulkScorer`.



##
lucene/core/src/test/org/apache/lucene/search/TestDisjunctionScoreBlockBoundaryPropagator.java:
##
@@ -37,8 +37,18 @@ public Explanation explain(LeafReaderContext context, int 
doc) throws IOExceptio
 }
 
 @Override
-public Scorer scorer(LeafReaderContext context) throws IOException {
-  return null;
+public ScorerSupplier scorerSupplier(LeafReaderContext context) throws 
IOException {
+  return new ScorerSupplier() {
+@Override
+public Scorer get(long leadCost) throws IOException {
+  return null;
+}
+
+@Override
+public long cost() {
+  return 0;
+}
+  };
 }

Review Comment:
   Let's return a null `ScorerSupplier`?



##
lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesRangeQuery.java:
##
@@ -101,7 +102,8 @@ public boolean isCacheable(LeafReaderContext ctx) {
   }
 
   @Override
-  public Scorer scorer(LeafReaderContext context) throws IOException {
+  public ScorerSupplier scorerSupplier(LeafReaderContext context) throws 
IOException {
+final Weight weight = this;

Review Comment:
   Extracting this local variable doesn't seem to help? (maybe it did in a 
previous version of the change?)



##
lucene/core/src/java/org/apache/lucene/search/FilterWeight.java:
##
@@ -58,11 +58,6 @@ public Explanation explain(LeafReaderContext context, int 
doc) throws IOExceptio
 return in.explain(context, doc);
   }
 
-  @Override
-  public Scorer scorer(LeafReaderContext context) throws IOException {
-return in.scorer(context);
-  }

Review Comment:
   This class should delegate scorerSupplier now?



##
lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java:
##
@@ -42,8 +42,18 @@ public Explanation explain(LeafReaderContext context, int 
doc) throws IOExceptio
 }
 
 @Override
-public Scorer scorer(LeafReaderContext context) throws IOException {
-  return null;
+public ScorerSupplier scorerSupplier(LeafReaderContext context) throws 
IOException {
+  return new ScorerSupplier() {
+@Override
+public Scorer get(long leadCost) throws IOException {
+  return null;
+}
+
+@Override
+public long cost() {
+  return 0;
+}
+  };
 }

Review Comment:
   Let's return a null `ScorerSupplier`? Returning null from 
`ScorerSupplier#get` is not legal.



##
lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java:
##
@@ -98,8 +98,18 @@ public Explanation explain(LeafReaderContext context, int 
doc) throws IOExceptio
 }
 
 @Override
-public Scorer scorer(LeafReaderContext context) throws IOException {
-  return null;
+public ScorerSupplier scorerSupplier(LeafReaderContext context) throws 
IOException {
+  return new ScorerSupplier() {
+@Override
+public Scorer get(long leadCost) throws IOException {
+  return null;
+}
+
+@Override
+public long cost() {
+  return 0;
+}
+  };
 }

Review Comment:
   Let's return a null `ScorerSupplier`? Returning null from 
`ScorerSupplier#get` is not legal.



##
lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java:
##
@@ -1355,9 +1355,23 @@ protected WeightWrapper(Weight in, AtomicBoolean 
scorerCalled, AtomicBoolean bul
 }
 
 @Override
-public Scorer scorer(LeafReaderContext context) throws IOException {
-  scorerCalled.set(true);
-  return in.scorer(context);
+public ScorerSupplier scorerSupplier(LeafReaderContext context) throws 
IOException {
+  final var scorer = in.scorer(context);

Review Comment:
   let's delegate to `scorerSupplier()` not `scorer()`?



##
lucene/core/src/java/org/apache/lucene/search/Weight.java:
##
@@ -131,23 +137,7 @@ public final Query getQuery() {
*
* @see #scorer
*/
-  public ScorerSupplier scorerSupplier(LeafReaderContext context) throws 
IOException {

Review Comment:
   Can you update javadocs? E.g. it's not an optional method anymore



##
lucene/queries/src/java/org/apache/lucene/queries/spans/SpanWeight.java:
##
@@ -135,16 

Re: [I] Support for building materialized views using Lucene formats [lucene]

2024-04-28 Thread via GitHub


jpountz commented on issue #13188:
URL: https://github.com/apache/lucene/issues/13188#issuecomment-2081658301

   It's not clear to me how we'd take advantage of this information at search 
time. What changes would we make to e.g. `Collector` to allow it to take 
advantage of these new data structures?


-- 
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: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



[PR] Fix numDeletesToMerge for unchanged segments [lucene]

2024-04-28 Thread via GitHub


dnhatn opened a new pull request, #13324:
URL: https://github.com/apache/lucene/pull/13324

   (no comment)


-- 
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: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org