Re: [PR] Add support for intra-segment search concurrency [lucene]
javanna commented on code in PR #13542: URL: https://github.com/apache/lucene/pull/13542#discussion_r1749750973 ## lucene/core/src/java/org/apache/lucene/search/BulkScorer.java: ## @@ -27,18 +27,6 @@ */ public abstract class BulkScorer { - /** - * Scores and collects all matching documents. - * - * @param collector The collector to which all matching documents are passed. - * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null} - * if they are all allowed to match. - */ - public void score(LeafCollector collector, Bits acceptDocs) throws IOException { Review Comment: Actually, I am not too sure it makes sense to deprecate in 9.x, because it is still heavily used and despite there is a variant of it that takes min and max, it is not a replacement yet until intra-segment concurrency is introduced with this PR. -- 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] Add support for intra-segment search concurrency [lucene]
javanna commented on code in PR #13542: URL: https://github.com/apache/lucene/pull/13542#discussion_r1749979372 ## lucene/core/src/test/org/apache/lucene/index/TestForTooMuchCloning.java: ## @@ -80,7 +80,7 @@ public void test() throws Exception { // System.out.println("query clone count=" + queryCloneCount); assertTrue( "too many calls to IndexInput.clone during TermRangeQuery: " + queryCloneCount, -queryCloneCount < 50); +queryCloneCount <= Math.max(s.getLeafContexts().size(), s.getSlices().length) * 4); Review Comment: I did more digging on this, things were much more predictable before because we only have 20 docs and a couple of segments. `50` was then high enough for all cases. With intra-segment slicing, we end up with more slices, and the number of clone calls is indeed a factor of the number of slices, but there is more to it which depends on the collected terms, and I am not sure how to calculate that exactly or make it predictable. The test does not seem to do that either so far, it just has a high enough value. I see two clone calls per partition done when `Weight#scorerSupplier` is called. They are from two different places in the `IntersectTermsEnum` constructor (line 89 and 98). I see other two clone calls per partition done when `ScorerSupplier#bulkScorer` is called. That is where my current factor of 4 came from. It seems to work in the vast majority of the cases. I have a seed (`F02FB6F046EE8C80`) where I have 9 partitions, but a total of 40 clone calls as part of search. Those additional 4 calls are rather unpredictable, I think that it depends on whether the query is rewritten as a boolean query or not. After all this analysis though, I realize that I came up with a formula that is more restrictive than the previous fixed value. Here we are searching 9 partitions, and the total becomes `36` (against `40` real value), while previously the fixed limit was `50`. I think I can increase the factor further without worrying too much. Key is that we are currently limiting the max number of partitions per segments to 5. We will need to adapt things in tests if we remove that limit in the future, which is likely. -- 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] Add deprecated complement (~) operator to RegExp [lucene]
rmuir merged PR #13739: URL: https://github.com/apache/lucene/pull/13739 -- 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] Add Bulk Scorer For ToParentBlockJoinQuery [lucene]
Mikep86 commented on code in PR #13697: URL: https://github.com/apache/lucene/pull/13697#discussion_r1750179324 ## lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinBulkScorer.java: ## @@ -0,0 +1,454 @@ +/* + * 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.lucene.search.join; + +import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.BoostQuery; +import org.apache.lucene.search.BulkScorer; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.search.ScorerSupplier; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.Weight; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; + +public class TestBlockJoinBulkScorer extends LuceneTestCase { + private static final String TYPE_FIELD_NAME = "type"; + private static final String VALUE_FIELD_NAME = "value"; + private static final String PARENT_FILTER_VALUE = "parent"; + private static final String CHILD_FILTER_VALUE = "child"; + + private enum MatchValue { +MATCH_A("A", 1), +MATCH_B("B", 2), +MATCH_C("C", 3), +MATCH_D("D", 4); + +private static final List VALUES = List.of(values()); + +private final String text; +private final int score; + +MatchValue(String text, int score) { + this.text = text; + this.score = score; +} + +public String getText() { + return text; +} + +public int getScore() { + return score; +} + +@Override +public String toString() { + return text; +} + +public static MatchValue random() { + return RandomPicks.randomFrom(LuceneTestCase.random(), VALUES); +} + } + + private record ChildDocMatch(int docId, List matches) { +public ChildDocMatch(int docId, List matches) { + this.docId = docId; + this.matches = Collections.unmodifiableList(matches); +} + } + + private static Map> populateRandomIndex( + RandomIndexWriter writer, int maxParentDocCount, int maxChildDocCount, int maxChildDocMatches) + throws IOException { +Map> expectedMatches = new HashMap<>(); + +final int parentDocCount = random().nextInt(1, maxParentDocCount + 1); +int currentDocId = 0; +for (int i = 0; i < parentDocCount; i++) { + final int childDocCount = random().nextInt(maxChildDocCount + 1); + List docs = new ArrayList<>(childDocCount); + List childDocMatches = new ArrayList<>(childDocCount); + + for (int j = 0; j < childDocCount; j++) { +// Build a child doc +Document childDoc = new Document(); +childDoc.add(newStringField(TYPE_FIELD_NAME, CHILD_FILTER_VALUE, Field.Store.NO)); + +final int matchCount = random().nextInt(maxChildDocMatches + 1); +List matchValues = new ArrayList<>(matchCount); +for (int k = 0; k < matchCount; k++) { + // Add a match to the child doc + MatchValue matchValue = MatchValue.random(); + matchValues.add(matchValue); + childDoc.add(newStringField(VALUE_FIELD_NAME, matchValue.getText(), Field.Store.NO)); +} + +docs.add(childDoc); +childDocMatches.add(new ChildDocMatch(currentDocId++, matchValues)); + } + + // Build a parent doc + Document parentDoc = new Document(); + parentDoc.add(newStringField(TYPE_FIELD_NAME, PARENT_FILTER_VALUE, Fi
Re: [PR] Add unit-of-least-precision float comparison [lucene]
mikemccand commented on PR #13723: URL: https://github.com/apache/lucene/pull/13723#issuecomment-2338279298 > Should we add the "double" variant, too? +1, maybe as follow-on. -- 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] Remove usage of IndexSearcher#Search(Query, Collector) from monitor package [lucene]
javanna commented on code in PR #13735: URL: https://github.com/apache/lucene/pull/13735#discussion_r1750433222 ## lucene/core/src/java/org/apache/lucene/search/CollectorManager.java: ## @@ -46,4 +48,28 @@ public interface CollectorManager { * called after collection is finished on all provided collectors. */ T reduce(Collection collectors) throws IOException; + + /** + * Wrap a provided {@link Collector} with a thin {@code CollectorManager} wrapper for use with + * {@link IndexSearcher#search(Query, CollectorManager)}. The wrapping {@code CollectorManager} + * provides no {@link CollectorManager#reduce(Collection)} implementation, so the wrapped {@code + * Collector} needs to do all relevant work while collecting. + * + * Note: This is only safe to use when {@code IndexSearcher} is created with no executor (see: + * {@link IndexSearcher#IndexSearcher(IndexReader, Executor)}), or the provided collector is + * threadsafe. + */ + static CollectorManager wrap(C in) { +return new CollectorManager() { + @Override + public C newCollector() { +return in; + } + + @Override + public Void reduce(Collection collectors) { +return null; Review Comment: assert that this is called providing a collection of size 1 that contains exactly the wrapped collector? Actually I am wondering if these should be IllegalStateExceptions instead of assert ## lucene/core/src/java/org/apache/lucene/search/CollectorManager.java: ## @@ -46,4 +48,28 @@ public interface CollectorManager { * called after collection is finished on all provided collectors. */ T reduce(Collection collectors) throws IOException; + + /** + * Wrap a provided {@link Collector} with a thin {@code CollectorManager} wrapper for use with + * {@link IndexSearcher#search(Query, CollectorManager)}. The wrapping {@code CollectorManager} + * provides no {@link CollectorManager#reduce(Collection)} implementation, so the wrapped {@code + * Collector} needs to do all relevant work while collecting. + * + * Note: This is only safe to use when {@code IndexSearcher} is created with no executor (see: + * {@link IndexSearcher#IndexSearcher(IndexReader, Executor)}), or the provided collector is + * threadsafe. + */ + static CollectorManager wrap(C in) { Review Comment: This is not fantastic but I suspect we may lean on this solution for other leftover usages of Collector vs CollectorManager. Shall we clarify in the name that this is for sequential execution / single threaded? ## lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java: ## @@ -37,7 +38,7 @@ abstract class CollectingMatcher extends CandidateMatcher< @Override public void matchQuery(final String queryId, Query matchQuery, Map metadata) throws IOException { -searcher.search(matchQuery, new MatchCollector(queryId, scoreMode)); +searcher.search(matchQuery, CollectorManager.wrap(new MatchCollector(queryId, scoreMode))); Review Comment: I left a comment about whether this is entirely safe, I was under the impression that one can provide an external searcher but I am not familiar with monitor code ;) ## lucene/core/src/java/org/apache/lucene/search/CollectorManager.java: ## @@ -46,4 +48,28 @@ public interface CollectorManager { * called after collection is finished on all provided collectors. */ T reduce(Collection collectors) throws IOException; + + /** + * Wrap a provided {@link Collector} with a thin {@code CollectorManager} wrapper for use with + * {@link IndexSearcher#search(Query, CollectorManager)}. The wrapping {@code CollectorManager} + * provides no {@link CollectorManager#reduce(Collection)} implementation, so the wrapped {@code + * Collector} needs to do all relevant work while collecting. + * + * Note: This is only safe to use when {@code IndexSearcher} is created with no executor (see: + * {@link IndexSearcher#IndexSearcher(IndexReader, Executor)}), or the provided collector is + * threadsafe. + */ + static CollectorManager wrap(C in) { +return new CollectorManager() { + @Override + public C newCollector() { +return in; Review Comment: should we assert that this is called once? -- 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] Remove usage of IndexSearcher#Search(Query, Collector) from monitor package [lucene]
romseygeek commented on code in PR #13735: URL: https://github.com/apache/lucene/pull/13735#discussion_r1750455113 ## lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java: ## @@ -37,7 +38,7 @@ abstract class CollectingMatcher extends CandidateMatcher< @Override public void matchQuery(final String queryId, Query matchQuery, Map metadata) throws IOException { -searcher.search(matchQuery, new MatchCollector(queryId, scoreMode)); +searcher.search(matchQuery, CollectorManager.wrap(new MatchCollector(queryId, scoreMode))); Review Comment: I guess you *could* call it with an externally built searcher, but it wouldn't be very useful as the whole point is that it's a searcher over a batch of documents submitted to the monitor for checking. It's public to allow clients to build wrapping matchers. There may be a nicer way to do this though, let me experiment. -- 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] Add support for intra-segment search concurrency [lucene]
mikemccand commented on code in PR #13542: URL: https://github.com/apache/lucene/pull/13542#discussion_r1750413448 ## lucene/MIGRATE.md: ## @@ -802,5 +802,35 @@ both `TopDocs` as well as facets results included in a reduced `FacetsCollector` ### `SearchWithCollectorTask` no longer supports the `collector.class` config parameter -`collector.class` used to allow users to load a custom collector implementation. `collector.manager.class` -replaces it by allowing users to load a custom collector manager instead. (Luca Cavanna) \ No newline at end of file +`collector.class` used to allow users to load a custom collector implementation. `collector.manager.class` +replaces it by allowing users to load a custom collector manager instead. (Luca Cavanna) + +### BulkScorer#score(LeafCollector collector, Bits acceptDocs) removed + +Use `BulkScorer#score(LeafCollector collector, Bits acceptDocs, int min, int max)` instead. In order to score the +entire leaf, provide `0` as min and `DocIdSetIterator.NO_MORE_DOCS` as max. `BulkScorer` subclasses that override +such method need to instead override the method variant that takes the range of doc ids as well as arguments. + +### CollectorManager#newCollector and Collector#getLeafCollector contract + +With the introduction of intra-segment query concurrency support, a `LeafCollector` for a given `LeafReaderContext` +instance may be requested multiple times as part of a single search call. This may only happen across separate +`Collector` instances returned by `CollectorManager#newCollector`, and never for the same `Collector` instance, given +that a slice can only ever hold a single partition of a given segment. Any logic or computation that needs to happen +once per segment requires specific handling in the collector manager implementation. +See `TotalHitCountCollectorManager` as an example. + +### Weight#scorer, Weight#bulkScorer and Weight#scorerSupplier contract + +With the introduction of intra-segment query concurrency support, a `Scorer`, `ScorerSupplier` or `BulkScorer` may +be requested multiple times for the same `LeafReaderContext` instance as part of a single search call. That +may happen concurrently from separate threads each searching a specific doc id range of the segment. +`Weight` implementations that rely on the assumption that a scorer, bulk scorer or scorer supplier for a given +`LeafReaderContext` is requested once per search need updating. + +### Signature of IndexSearcher#searchLeaf changed + +With the introduction of intra-segment query concurrency support, the `IndexSearcher#searchLeaf(LeafReaderContext, Weight, Collector)` +method accepts now two additional int arguments to identify the min and max doc id of the range that the leaf partition Review Comment: `accepts now` -> `now accepts`? ## lucene/core/src/java/org/apache/lucene/search/CollectorManager.java: ## @@ -31,6 +32,13 @@ * fully collected. * * + * Note: Separate {@link Collector} instances returned by {@link Review Comment: Maybe `Multiple LeafCollectors may be requested for the same LeafReaderContext via getLeafCollector` (with the nice `link`s included)? ## lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java: ## @@ -363,7 +374,88 @@ public static LeafSlice[] slices( LeafSlice[] slices = new LeafSlice[groupedLeaves.size()]; int upto = 0; for (List currentLeaf : groupedLeaves) { - slices[upto] = new LeafSlice(currentLeaf); + slices[upto] = + new LeafSlice( + new ArrayList<>( + currentLeaf.stream() + .map(LeafReaderContextPartition::createForEntireSegment) + .toList())); + ++upto; +} + +return slices; + } + + /** + * Creates leaf slices that leverage intra-segment concurrency by splitting segments into multiple + * partitions according to the provided max number of documents per slice and maximum number of + * segments per slice. If a segment holds more documents than the provided max per slice, it gets + * split into equal size partitions that each gets its own slice assigned. Review Comment: Can a slice contain multiple partitions? (This impl doesn't do so right now, but it is allowed/possible?) Can a slice mix partitions and whole leaves? (I guess one could always take a whole leaf and switch it to a single partition `0..maxDoc`). ## lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java: ## @@ -363,7 +374,88 @@ public static LeafSlice[] slices( LeafSlice[] slices = new LeafSlice[groupedLeaves.size()]; int upto = 0; for (List currentLeaf : groupedLeaves) { - slices[upto] = new LeafSlice(currentLeaf); + slices[upto] = + new LeafSlice( + new ArrayList<>( + currentLeaf.stream() + .map(LeafReaderContextPartition
Re: [PR] Add support for intra-segment search concurrency [lucene]
javanna commented on code in PR #13542: URL: https://github.com/apache/lucene/pull/13542#discussion_r1750500023 ## lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java: ## @@ -363,7 +374,88 @@ public static LeafSlice[] slices( LeafSlice[] slices = new LeafSlice[groupedLeaves.size()]; int upto = 0; for (List currentLeaf : groupedLeaves) { - slices[upto] = new LeafSlice(currentLeaf); + slices[upto] = + new LeafSlice( + new ArrayList<>( + currentLeaf.stream() + .map(LeafReaderContextPartition::createForEntireSegment) + .toList())); + ++upto; +} + +return slices; + } + + /** + * Creates leaf slices that leverage intra-segment concurrency by splitting segments into multiple + * partitions according to the provided max number of documents per slice and maximum number of + * segments per slice. If a segment holds more documents than the provided max per slice, it gets + * split into equal size partitions that each gets its own slice assigned. Review Comment: yes it can be a mix of entire segments and partitions of different segments. I am only enforcing that different partitions of the same segment need to go to different slices, one each. That is because it makes little sense to partition and then use a single thread. Also, supporting that scenario would require to further clarify that the getLeafCollector semantics need handling within each Collector implementation, as opposed to only across multiple collectors returned by the collector manager. -- 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] Add support for intra-segment search concurrency [lucene]
javanna commented on code in PR #13542: URL: https://github.com/apache/lucene/pull/13542#discussion_r1750502348 ## lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java: ## @@ -363,7 +374,88 @@ public static LeafSlice[] slices( LeafSlice[] slices = new LeafSlice[groupedLeaves.size()]; int upto = 0; for (List currentLeaf : groupedLeaves) { - slices[upto] = new LeafSlice(currentLeaf); + slices[upto] = + new LeafSlice( + new ArrayList<>( + currentLeaf.stream() + .map(LeafReaderContextPartition::createForEntireSegment) + .toList())); + ++upto; +} + +return slices; + } + + /** + * Creates leaf slices that leverage intra-segment concurrency by splitting segments into multiple + * partitions according to the provided max number of documents per slice and maximum number of + * segments per slice. If a segment holds more documents than the provided max per slice, it gets + * split into equal size partitions that each gets its own slice assigned. + * + * Note: this method is not yet called by the default slicing implementation {@link + * #slices(List)}. Certain queries that require segment-level computation ahead of time duplicate + * this effort across segment partitions. Once that can be shared across partitions we can safely + * create partitions by default and perhaps refine the slicing approach implemented in this + * method. For this reason segment partitions are currently only created in tests. Users can call + * this method at their own risk. + * + * @lucene.experimental + */ + public static LeafSlice[] slicesWithPartitions( Review Comment: I would think so too. I don't know if we'll want to take the desired amount of slices as an argument. Using max docs per slice is not flexible enough it seems. That is kind of why I left this method experimental. I think adding a boolean to the existing method makes a lot of sense though! -- 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] Reorder checks in LRUQueryCache#count [lucene]
shubhamvishu commented on PR #13742: URL: https://github.com/apache/lucene/pull/13742#issuecomment-2338468843 @jpountz Added a CHANGES entry now. Thanks! -- 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] Remove usage of IndexSearcher#Search(Query, Collector) from monitor package [lucene]
romseygeek commented on code in PR #13735: URL: https://github.com/apache/lucene/pull/13735#discussion_r1750548459 ## lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java: ## @@ -37,7 +38,7 @@ abstract class CollectingMatcher extends CandidateMatcher< @Override public void matchQuery(final String queryId, Query matchQuery, Map metadata) throws IOException { -searcher.search(matchQuery, new MatchCollector(queryId, scoreMode)); +searcher.search(matchQuery, CollectorManager.wrap(new MatchCollector(queryId, scoreMode))); Review Comment: I think we can make the public-facing methods all take `DocumentBatch` here rather than `IndexSearcher`, which will make building the index searcher an implementation detail. I'll put together a PR. -- 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] Add Bulk Scorer For ToParentBlockJoinQuery [lucene]
jpountz commented on code in PR #13697: URL: https://github.com/apache/lucene/pull/13697#discussion_r1750571840 ## lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinBulkScorer.java: ## @@ -0,0 +1,454 @@ +/* + * 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.lucene.search.join; + +import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.BoostQuery; +import org.apache.lucene.search.BulkScorer; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.search.ScorerSupplier; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.Weight; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; + +public class TestBlockJoinBulkScorer extends LuceneTestCase { + private static final String TYPE_FIELD_NAME = "type"; + private static final String VALUE_FIELD_NAME = "value"; + private static final String PARENT_FILTER_VALUE = "parent"; + private static final String CHILD_FILTER_VALUE = "child"; + + private enum MatchValue { +MATCH_A("A", 1), +MATCH_B("B", 2), +MATCH_C("C", 3), +MATCH_D("D", 4); + +private static final List VALUES = List.of(values()); + +private final String text; +private final int score; + +MatchValue(String text, int score) { + this.text = text; + this.score = score; +} + +public String getText() { + return text; +} + +public int getScore() { + return score; +} + +@Override +public String toString() { + return text; +} + +public static MatchValue random() { + return RandomPicks.randomFrom(LuceneTestCase.random(), VALUES); +} + } + + private record ChildDocMatch(int docId, List matches) { +public ChildDocMatch(int docId, List matches) { + this.docId = docId; + this.matches = Collections.unmodifiableList(matches); +} + } + + private static Map> populateRandomIndex( + RandomIndexWriter writer, int maxParentDocCount, int maxChildDocCount, int maxChildDocMatches) + throws IOException { +Map> expectedMatches = new HashMap<>(); + +final int parentDocCount = random().nextInt(1, maxParentDocCount + 1); +int currentDocId = 0; +for (int i = 0; i < parentDocCount; i++) { + final int childDocCount = random().nextInt(maxChildDocCount + 1); + List docs = new ArrayList<>(childDocCount); + List childDocMatches = new ArrayList<>(childDocCount); + + for (int j = 0; j < childDocCount; j++) { +// Build a child doc +Document childDoc = new Document(); +childDoc.add(newStringField(TYPE_FIELD_NAME, CHILD_FILTER_VALUE, Field.Store.NO)); + +final int matchCount = random().nextInt(maxChildDocMatches + 1); +List matchValues = new ArrayList<>(matchCount); +for (int k = 0; k < matchCount; k++) { + // Add a match to the child doc + MatchValue matchValue = MatchValue.random(); + matchValues.add(matchValue); + childDoc.add(newStringField(VALUE_FIELD_NAME, matchValue.getText(), Field.Store.NO)); +} + +docs.add(childDoc); +childDocMatches.add(new ChildDocMatch(currentDocId++, matchValues)); + } + + // Build a parent doc + Document parentDoc = new Document(); + parentDoc.add(newStringField(TYPE_FIELD_NAME, PARENT_FILTER_VALUE, Fi
Re: [PR] Add support for intra-segment search concurrency [lucene]
javanna commented on code in PR #13542: URL: https://github.com/apache/lucene/pull/13542#discussion_r1750711025 ## lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java: ## @@ -363,7 +374,88 @@ public static LeafSlice[] slices( LeafSlice[] slices = new LeafSlice[groupedLeaves.size()]; int upto = 0; for (List currentLeaf : groupedLeaves) { - slices[upto] = new LeafSlice(currentLeaf); + slices[upto] = + new LeafSlice( + new ArrayList<>( + currentLeaf.stream() + .map(LeafReaderContextPartition::createForEntireSegment) + .toList())); + ++upto; +} + +return slices; + } + + /** + * Creates leaf slices that leverage intra-segment concurrency by splitting segments into multiple + * partitions according to the provided max number of documents per slice and maximum number of + * segments per slice. If a segment holds more documents than the provided max per slice, it gets + * split into equal size partitions that each gets its own slice assigned. + * + * Note: this method is not yet called by the default slicing implementation {@link + * #slices(List)}. Certain queries that require segment-level computation ahead of time duplicate Review Comment: Will apply this suggestion. I am removing the explicit mention of KNN queries, because they are a bit of a different world that needs specific treatment, in that the bulk of the work is done as part of query rewrite that parallelizes across segments bypassing the slicing mechanism. -- 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] Add support for intra-segment search concurrency [lucene]
javanna commented on code in PR #13542: URL: https://github.com/apache/lucene/pull/13542#discussion_r1750743766 ## lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java: ## @@ -363,7 +374,88 @@ public static LeafSlice[] slices( LeafSlice[] slices = new LeafSlice[groupedLeaves.size()]; int upto = 0; for (List currentLeaf : groupedLeaves) { - slices[upto] = new LeafSlice(currentLeaf); + slices[upto] = + new LeafSlice( + new ArrayList<>( + currentLeaf.stream() + .map(LeafReaderContextPartition::createForEntireSegment) + .toList())); + ++upto; +} + +return slices; + } + + /** + * Creates leaf slices that leverage intra-segment concurrency by splitting segments into multiple + * partitions according to the provided max number of documents per slice and maximum number of + * segments per slice. If a segment holds more documents than the provided max per slice, it gets + * split into equal size partitions that each gets its own slice assigned. + * + * Note: this method is not yet called by the default slicing implementation {@link + * #slices(List)}. Certain queries that require segment-level computation ahead of time duplicate + * this effort across segment partitions. Once that can be shared across partitions we can safely + * create partitions by default and perhaps refine the slicing approach implemented in this + * method. For this reason segment partitions are currently only created in tests. Users can call + * this method at their own risk. + * + * @lucene.experimental + */ + public static LeafSlice[] slicesWithPartitions( Review Comment: Looks like the existing IndexSearcher#slices (that public static one) does not have a lucene.experimental tag. That's odd, because the rest of the search concurrency code is marked experimental. On the other hand, it should be fine adding a last argument boolean to the existing method, that's the direction I am taking. I added that to the MIGRATE file too. -- 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] Add migration note about the deprecation of optional RegExp complement syntax [lucene]
ChrisHegarty merged PR #13732: URL: https://github.com/apache/lucene/pull/13732 -- 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] Remove usage of IndexSearcher#Search(Query, Collector) from monitor package [lucene]
gsmiller commented on code in PR #13735: URL: https://github.com/apache/lucene/pull/13735#discussion_r1750877300 ## lucene/core/src/java/org/apache/lucene/search/CollectorManager.java: ## @@ -46,4 +48,28 @@ public interface CollectorManager { * called after collection is finished on all provided collectors. */ T reduce(Collection collectors) throws IOException; + + /** + * Wrap a provided {@link Collector} with a thin {@code CollectorManager} wrapper for use with + * {@link IndexSearcher#search(Query, CollectorManager)}. The wrapping {@code CollectorManager} + * provides no {@link CollectorManager#reduce(Collection)} implementation, so the wrapped {@code + * Collector} needs to do all relevant work while collecting. + * + * Note: This is only safe to use when {@code IndexSearcher} is created with no executor (see: + * {@link IndexSearcher#IndexSearcher(IndexReader, Executor)}), or the provided collector is + * threadsafe. + */ + static CollectorManager wrap(C in) { +return new CollectorManager() { + @Override + public C newCollector() { +return in; Review Comment: Oh you're right. I'd overlooked the logic for setting up `leafSlicesSupplier`. -- 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] Remove usage of IndexSearcher#Search(Query, Collector) from monitor package [lucene]
gsmiller commented on code in PR #13735: URL: https://github.com/apache/lucene/pull/13735#discussion_r1750884436 ## lucene/core/src/java/org/apache/lucene/search/CollectorManager.java: ## @@ -46,4 +48,28 @@ public interface CollectorManager { * called after collection is finished on all provided collectors. */ T reduce(Collection collectors) throws IOException; + + /** + * Wrap a provided {@link Collector} with a thin {@code CollectorManager} wrapper for use with + * {@link IndexSearcher#search(Query, CollectorManager)}. The wrapping {@code CollectorManager} + * provides no {@link CollectorManager#reduce(Collection)} implementation, so the wrapped {@code + * Collector} needs to do all relevant work while collecting. + * + * Note: This is only safe to use when {@code IndexSearcher} is created with no executor (see: + * {@link IndexSearcher#IndexSearcher(IndexReader, Executor)}), or the provided collector is + * threadsafe. + */ + static CollectorManager wrap(C in) { Review Comment: Makes sense. I'll revise the name (but if you have suggestions, I'm open to them; naming can be tricky) -- 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] Add Bulk Scorer For ToParentBlockJoinQuery [lucene]
gsmiller commented on code in PR #13697: URL: https://github.com/apache/lucene/pull/13697#discussion_r1750898154 ## lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java: ## @@ -348,34 +410,23 @@ public void setMinCompetitiveScore(float minScore) throws IOException { } } -private void setScoreAndFreq() throws IOException { +private float scoreChildDocs() throws IOException { if (childApproximation.docID() >= parentApproximation.docID()) { -return; +return parentScore.score(); } - double score = scoreMode == ScoreMode.None ? 0 : childScorer.score(); - int freq = 1; - while (childApproximation.nextDoc() < parentApproximation.docID()) { -if (childTwoPhase == null || childTwoPhase.matches()) { - final float childScore = scoreMode == ScoreMode.None ? 0 : childScorer.score(); - freq += 1; - switch (scoreMode) { -case Total: -case Avg: - score += childScore; - break; -case Min: - score = Math.min(score, childScore); - break; -case Max: - score = Math.max(score, childScore); - break; -case None: - break; -default: - throw new AssertionError(); + + float score = 0; + if (scoreMode != ScoreMode.None) { Review Comment: I'll have to look at the test case a little closer, but I'd rather we didn't unnecessarily advance to make a test case happy. If there's a much sparser query clause leading a conjunctive iteration, the eager advancing of the child iterator to the next parent could add meaningful overhead. I think anyway? I'm also not totally sure why we even have that check in the TPBJQ logic. It seems like a slightly odd place to do that check? Maybe there's a different place we could do that check if necessary? Maybe we could do a check in `ParentApproximation#advance` to validate that the child iterator doesn't provide a doc that's present in the parent bitset? Another alternative might be an `assert` that eagerly advances to the next parent (like you had)? With an `assert`, we could bypass the potential performance drain while still getting some validation coverage? -- 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] Remove usage of IndexSearcher#Search(Query, Collector) from monitor package [lucene]
msfroh commented on code in PR #13735: URL: https://github.com/apache/lucene/pull/13735#discussion_r1750940757 ## lucene/core/src/java/org/apache/lucene/search/CollectorManager.java: ## @@ -46,4 +48,28 @@ public interface CollectorManager { * called after collection is finished on all provided collectors. */ T reduce(Collection collectors) throws IOException; + + /** + * Wrap a provided {@link Collector} with a thin {@code CollectorManager} wrapper for use with + * {@link IndexSearcher#search(Query, CollectorManager)}. The wrapping {@code CollectorManager} + * provides no {@link CollectorManager#reduce(Collection)} implementation, so the wrapped {@code + * Collector} needs to do all relevant work while collecting. + * + * Note: This is only safe to use when {@code IndexSearcher} is created with no executor (see: + * {@link IndexSearcher#IndexSearcher(IndexReader, Executor)}), or the provided collector is + * threadsafe. + */ + static CollectorManager wrap(C in) { +return new CollectorManager() { + @Override + public C newCollector() { +return in; + } + + @Override + public Void reduce(Collection collectors) { +return null; Review Comment: Can we assume that the `Collector`s passed to `reduce` were produced by calling `newCollector` on this collector? If so, the assertion that `newCollector` was called at most once should be sufficient. -- 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] Remove usage of IndexSearcher#search(Query, Collector) from join package [lucene]
msfroh opened a new pull request, #13747: URL: https://github.com/apache/lucene/pull/13747 ### Description Relates to https://github.com/apache/lucene/issues/12892 For global ordinal-based join, we can support concurrent search. For numeric and term-based joins, we fail if we're called from a multithreaded searcher. I can implement concurrent versions of the other join collectors, but wanted to get a first pass that removes the uses of `IndexSearcher#search(Query, Collector)`. Note that for the cases that still assume a single-threaded searcher, I used a version of `CollectorManager#wrap(Collector)` from https://github.com/apache/lucene/pull/13735, with my guess for where it will end up based on feedback so far. -- 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] nocommit: demonstrate how a minor change in IndexSearcher can have an inexplicable performance impact [lucene]
github-actions[bot] commented on PR #13657: URL: https://github.com/apache/lucene/pull/13657#issuecomment-2339375272 This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the d...@lucene.apache.org list. Thank you for your contribution! -- 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] Update QueryUtils to use CollectorManager [lucene]
msfroh commented on code in PR #13748: URL: https://github.com/apache/lucene/pull/13748#discussion_r1751112687 ## lucene/test-framework/src/java/org/apache/lucene/tests/search/QueryUtils.java: ## @@ -345,175 +348,199 @@ public static void checkSkipTo(final Query q, final IndexSearcher s) throws IOEx // FUTURE: ensure scorer.doc()==-1 final float maxDiff = 1e-5f; - final LeafReader[] lastReader = {null}; - s.search( - q, - new SimpleCollector() { -private Scorable sc; -private Scorer scorer; -private DocIdSetIterator iterator; -private int leafPtr; + List lastReaders = + s.search( + q, + new CollectorManager>() { +@Override +public SimpleCollectorWithLastReader newCollector() { + return new SimpleCollectorWithLastReader() { +LeafReader lastReader = null; +private Scorable sc; +private Scorer scorer; +private DocIdSetIterator iterator; +private int leafPtr; + +@Override +public void setScorer(Scorable scorer) { + this.sc = scorer; +} -@Override -public void setScorer(Scorable scorer) { - this.sc = scorer; -} +@Override +public void collect(int doc) throws IOException { + float score = sc.score(); + lastDoc[0] = doc; + try { +if (scorer == null) { + Query rewritten = s.rewrite(q); + Weight w = s.createWeight(rewritten, ScoreMode.COMPLETE, 1); + LeafReaderContext context = readerContextArray.get(leafPtr); + scorer = w.scorer(context); + iterator = scorer.iterator(); +} + +int op = order[(opidx[0]++) % order.length]; +// System.out.println(op==skip_op ? +// "skip("+(sdoc[0]+1)+")":"next()"); +boolean more = +op == skip_op +? iterator.advance(scorer.docID() + 1) +!= DocIdSetIterator.NO_MORE_DOCS +: iterator.nextDoc() != DocIdSetIterator.NO_MORE_DOCS; +int scorerDoc = scorer.docID(); +float scorerScore = scorer.score(); +float scorerScore2 = scorer.score(); +float scoreDiff = Math.abs(score - scorerScore); +float scorerDiff = Math.abs(scorerScore2 - scorerScore); + +boolean success = false; +try { + assertTrue(more); + assertEquals("scorerDoc=" + scorerDoc + ",doc=" + doc, scorerDoc, doc); + assertTrue( + "score=" + score + ", scorerScore=" + scorerScore, + scoreDiff <= maxDiff); + assertTrue( + "scorerScorer=" + scorerScore + ", scorerScore2=" + scorerScore2, + scorerDiff <= maxDiff); + success = true; +} finally { + if (!success) { +if (LuceneTestCase.VERBOSE) { + StringBuilder sbord = new StringBuilder(); + for (int i = 0; i < order.length; i++) { +sbord.append(order[i] == skip_op ? " skip()" : " next()"); + } + System.out.println( + "ERROR matching docs:" + + "\n\t" + + (doc != scorerDoc ? "--> " : "") + + "doc=" + + doc + + ", scorerDoc=" + + scorerDoc + + "\n\t" + + (!more ? "--> " : "") + + "tscorer.more=" + + more + + "\n\t" + + (scoreDiff > maxDiff ? "--> " : "") + + "scorerScore=" + + scorerScore + + " scoreDiff=" + + scoreDiff +
Re: [PR] Update QueryUtils to use CollectorManager [lucene]
msfroh commented on PR #13748: URL: https://github.com/apache/lucene/pull/13748#issuecomment-2339574918 I had another thought while walking home. Both of the anonymous CollectorManagers are the same (but produce slightly different collectors). What if you split it out into an inner class that takes a `Supplier` as a constructor arg? That way the two checks can each have their anonymous implementations, but the anonymous nesting won't be as deep. -- 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