You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/08/12 05:30:22 UTC

[GitHub] [lucene] zacharymorn opened a new pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

zacharymorn opened a new pull request #240:
URL: https://github.com/apache/lucene/pull/240


   # Description / Solution
   Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager), with the following changes:
   1. Refactor out TopScoreDocCollectorManager, TopFieldCollectorManager, TotalHitCountCollectorManager and FixedBitSetCollector
   2. Refactor some tests to use the above collector manager
   3. Refactor DrillSideways to use extracted out collector managers
   
   # Tests
   Passed updated tests with `./gradlew clean; ./gradlew check -Ptests.nightly=true` (currently there are nocommit messages that are failing precommit though)
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/lucene/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [x] I have given Lucene maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [x] I have run `./gradlew check`.
   - [ ] I have added tests for my changes.
   


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r692722475



##########
File path: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
##########
@@ -180,6 +185,7 @@ protected int withTopDocs(IndexSearcher searcher, Query q, TopDocs hits) throws
     return res;
   }
 
+  @Deprecated

Review comment:
       > I wonder if anyone out there has sub-classed this to provide their own Collector through overriding this method? It's possible right?
   
   Hmm you are right. I reverted some changes in this class to still use the `ReadTask#createCollector` method, in case there's overriding from users. 
   
   > Would we want to support this with a CollectorManager now instead? You might consider creating a new protected method--createCollectorManager--that sub-classes could override if they want, then add javadoc here with a @lucene.deprecated tag pointing users to the new method.
   
   I think this `createCollector` method was originally added (11 years ago) to support benchmarking,  and not sure how widely this is used actually? So hopefully marking it as deprecated for release 9.0 and removing it entirely once we release 10.0 should be good enough?

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
##########
@@ -527,7 +503,10 @@ public TopDocs search(Query query, int n) throws IOException {
    *
    * @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()}
    *     clauses.
+   * @deprecated This method is being deprecated in favor of {@link IndexSearcher#search(Query,

Review comment:
       > I think we like the @lucene.deprecated tag?
   Do you mean `@lucene.deprecated` should be created and used instead of using `@deprecated`? I can't seems to find that tag is being used in lucene actually...I think `@lucene.deprecated` most likely won't be recognized by IDE or build tool to signal / warn the use of deprecated methods though.
   
   > Is your reasoning behind keeping this in there for now that all of our uses of this (internal to Lucene) haven't yet migrated as part of this change? Would the plan me to migrate all usages off of this internally and then actually remove it on main/9.0, or are you thinking of keeping it around until 10.0? I think our backwards compatibility policy is such that we could just directly remove this on main/9.0, but then leave it like you have it (marked deprecated) if you choose to backport this to 8x. Since this method is so fundamental though, I could easily see an argument to keep it around for an extra major release to give users more time to migrate. Then again, the migration path seems pretty straight-forward. What do you think?
   
   As explained in https://issues.apache.org/jira/browse/LUCENE-10002?focusedCommentId=17397827&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17397827, I use deprecation instead of direct removal in this PR exactly for the reasons you mentioned:
   1. Not all internal uses have been migrated in this PR (which may require another few thousand lines of changes).
   1. I personally feel that we should give users more time for migration and testing out the changes, so 10.0 release might be a good timing for complete removal. 
   
   I would be interested in seeing how folks feel about the timing of removal, but after #1  is completed, complete removal of IndexSearcher#search(Query, Collector) in lucene codebase should require only straightforward deletion changes.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r693358901



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       Thanks for the details! I'm still a bit confused though. I'm not totally clear on where the problem with `acceptDocs` comes into play (but there are a lot of code layers to follow here and I'm still trying to wrap my head around it). The `DrillSidewaysScorer` (which implements `BulkScorer`) would get used to populate the cached bitset, and it checks against `acceptDocs` when scoring. So wouldn't the cached docs already take into account the deleted doc filtering? 
   
   I think there _is_ at least one other problem with caching though. It's important that the `DrillSidewaysScorer` actually runs during query evaluation every time since it's where the "sideways" `FacetsCollectors` get populated with the "near miss" docs. So I think there's at least another critical caching issue here where only the "true hit" docs would get cached, then on a cache hit would get reused without the `DrillSidewaysScorer` getting a chance to populate the near misses (so all those FCs would be empty).
   
   So yeah, I'm very much convinced this hack is needed (and I see where it's done already for the base `Collector` implementation), but I want to make sure I fully understand the caching concerns around `acceptDoc`. Apologies if I'm overlooking 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: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #240:
URL: https://github.com/apache/lucene/pull/240#issuecomment-916729951


   Hi @jpountz @gsmiller , just want to check back on this PR to see if you have any further feedback?


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #240:
URL: https://github.com/apache/lucene/pull/240#issuecomment-920178625


   Sorry @zacharymorn I have not forgotten about it but the change is very large and it's hard to find enough adjacent time to review it. I'll do my best to find time in the coming week.


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r692722544



##########
File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
##########
@@ -527,7 +503,10 @@ public TopDocs search(Query query, int n) throws IOException {
    *
    * @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()}
    *     clauses.
+   * @deprecated This method is being deprecated in favor of {@link IndexSearcher#search(Query,

Review comment:
       > I think we like the @lucene.deprecated tag?
   
   Do you mean `@lucene.deprecated` should be created and used instead of using `@deprecated`? I can't seems to find that tag is being used in lucene actually...I think `@lucene.deprecated` most likely won't be recognized by IDE or build tool to signal / warn the use of deprecated methods though.
   
   > Is your reasoning behind keeping this in there for now that all of our uses of this (internal to Lucene) haven't yet migrated as part of this change? Would the plan me to migrate all usages off of this internally and then actually remove it on main/9.0, or are you thinking of keeping it around until 10.0? I think our backwards compatibility policy is such that we could just directly remove this on main/9.0, but then leave it like you have it (marked deprecated) if you choose to backport this to 8x. Since this method is so fundamental though, I could easily see an argument to keep it around for an extra major release to give users more time to migrate. Then again, the migration path seems pretty straight-forward. What do you think?
   
   As explained in https://issues.apache.org/jira/browse/LUCENE-10002?focusedCommentId=17397827&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17397827, I use deprecation instead of direct removal in this PR exactly for the reasons you mentioned:
   1. Not all internal uses have been migrated in this PR (which may require another few thousand lines of changes).
   1. I personally feel that we should give users more time for migration and testing out the changes, so 10.0 release might be a good timing for complete removal. 
   
   I would be interested in seeing how folks feel about the timing of removal, but after #1  is completed, complete removal of IndexSearcher#search(Query, Collector) in lucene codebase should require only straightforward deletion changes.

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
##########
@@ -659,9 +614,12 @@ public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOEx
    */
   public <C extends Collector, T> T search(Query query, CollectorManager<C, T> collectorManager)
       throws IOException {
-    if (executor == null || leafSlices.length <= 1) {
+    if (executor == null || leafSlices.length == 0) {

Review comment:
       Technically speaking in this block, `collectorManager#newCollector` & `collectorManager#reduce` etc are still used. But I couldn't remember why I made this change exactly (maybe was just trying to still use the thread pool executor to process this case as well), so I reverted this change.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -383,9 +382,13 @@ protected void updateMinCompetitiveScore(Scorable scorer) throws IOException {
    *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
    *     count accurate, but this will also make query processing slower.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated

Review comment:
       Please see above.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -407,97 +410,14 @@ public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThre
    *     field is indexed both with doc values and points. In this case, there is an assumption that
    *     the same data is stored in these points and doc values.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated

Review comment:
       Please see above.

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FixedBitSetCollector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectorManager;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+
+/** Test utility collector that uses FixedBitSet to record hits. */
+public class FixedBitSetCollector extends SimpleCollector {
+  private FixedBitSet hits;

Review comment:
       Updated.

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FixedBitSetCollector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectorManager;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+
+/** Test utility collector that uses FixedBitSet to record hits. */
+public class FixedBitSetCollector extends SimpleCollector {
+  private FixedBitSet hits;
+  private int docBase;
+
+  public FixedBitSetCollector(int maxDoc) {
+    hits = new FixedBitSet(maxDoc);
+  }
+
+  @Override
+  public ScoreMode scoreMode() {
+    return ScoreMode.COMPLETE_NO_SCORES;
+  }
+
+  @Override
+  protected void doSetNextReader(LeafReaderContext context) {
+    docBase = context.docBase;
+  }
+
+  @Override
+  public void collect(int doc) {
+    hits.set(docBase + doc);
+  }
+
+  public FixedBitSet getHits() {
+    return hits;
+  }
+
+  public static CollectorManager<FixedBitSetCollector, FixedBitSet> create(int maxDoc) {
+    return new CollectorManager<>() {
+      @Override
+      public FixedBitSetCollector newCollector() {
+        return new FixedBitSetCollector(maxDoc);
+      }
+
+      @Override
+      public FixedBitSet reduce(Collection<FixedBitSetCollector> collectors) {
+        FixedBitSet result = new FixedBitSet(maxDoc);
+        collectors.stream().forEach(c -> result.or(c.getHits()));

Review comment:
       You are right! Updated.

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FixedBitSetCollector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectorManager;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+
+/** Test utility collector that uses FixedBitSet to record hits. */
+public class FixedBitSetCollector extends SimpleCollector {
+  private FixedBitSet hits;
+  private int docBase;
+
+  public FixedBitSetCollector(int maxDoc) {
+    hits = new FixedBitSet(maxDoc);
+  }
+
+  @Override
+  public ScoreMode scoreMode() {
+    return ScoreMode.COMPLETE_NO_SCORES;
+  }
+
+  @Override
+  protected void doSetNextReader(LeafReaderContext context) {
+    docBase = context.docBase;
+  }
+
+  @Override
+  public void collect(int doc) {
+    hits.set(docBase + doc);
+  }
+
+  public FixedBitSet getHits() {
+    return hits;
+  }
+
+  public static CollectorManager<FixedBitSetCollector, FixedBitSet> create(int maxDoc) {

Review comment:
       Makes sense. Updated.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Create a TopScoreDocCollectorManager which uses a shared hit counter to maintain number of hits
+ * and a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopScoreDocCollectorManager
+    implements CollectorManager<TopScoreDocCollector, TopDocs> {
+  private final int numHits;
+  private final ScoreDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+
+  /**
+   * Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the
+   * number of hits to count accurately.
+   *
+   * <p><b>NOTE</b>: If the total hit count of the top docs is less than or exactly {@code
+   * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link
+   * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower
+   * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate
+   * but will also likely make query processing slower.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopScoreDocCollectorManager(
+      int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {

Review comment:
       Yes the concern is indeed valid. Technically speaking this is probably an existing issue, as you can accidentally pass in not thread-safe internals to collectorManager to be used in a concurrent index searcher (the example below is thread-safe and internal to lucene's method, but the user can implement similar logic and accidentally pass in not thread-safe `HitsThresholdChecker` and nothing would check for thread-safety afterward) :
   
   https://github.com/apache/lucene/blob/efb7b2a5e8c1bdc19dfd65f7095f70a142343472/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java#L482-L510
   
   On the other hand, not all collectorManager requires concurrency support, for example: 
   
   https://github.com/apache/lucene/blob/efb7b2a5e8c1bdc19dfd65f7095f70a142343472/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java#L430-L447
   
   However, given we are trying to promote IndexSearch#search(Query, CollectorManager) to be the main method for collection, and to guard against hard to debug concurrency related errors, I feel we should definitely do what you suggested. Maybe we can add a new method `supportsConcurrency` to CollectorManager's API like below to "force" the user to specify it, and to allow `IndexSearcher` to check for it:
   
   ```
   public interface CollectorManager<C extends Collector, T> {
   
     C newCollector() throws IOException;
   
     T reduce(Collection<C> collectors) throws IOException;
   
     boolean supportsConcurrency();
   }
   ```
   
   What do you think?

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       Sorry about the typo there, I meant `Without this hack`. The caching decision happens here 
   
   https://github.com/apache/lucene/blob/5896e5389a83f657781875a852120615ba4763dc/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java#L863-L866
   
   so the hack of returning `ScoreMode.COMPLETE` basically would cause `scoreMode.needsScores() == true` in this logic. 
   
   Without this hack, eventually this logic will be invoked, passing `null` for `acceptDocs`, causing some deleted docs to be collected as well.
   
   https://github.com/apache/lucene/blob/5896e5389a83f657781875a852120615ba4763dc/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java#L509-L521




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r697047554



##########
File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
##########
@@ -527,7 +503,10 @@ public TopDocs search(Query query, int n) throws IOException {
    *
    * @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()}
    *     clauses.
+   * @deprecated This method is being deprecated in favor of {@link IndexSearcher#search(Query,

Review comment:
       +1 makes sense to keep it in 9.0. As for my comment about `@lucene.deprecated`, I was getting this mixed up with something else. Apologies for any confusion!




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r691795187



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+
+  public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    /*
+    nocommit
+    Should the following two be passed in instead? Possible custom initialization based on executor status and slices?
+    On the other hand, in a single-threaded environment, shared HitsThresholdChecker and MaxScoreAccumulator should be fast without lock contention anyway?

Review comment:
       Hi @jpountz, for now I've implemented the above approach in https://github.com/apache/lucene/pull/240/commits/4af77405fcb914d497d82eb60f0011f93079ec7b, and the lucenenutil benchmark shows there's no obvious impact to performance now:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
                MedSpanNear        6.59      (5.6%)        6.30     (10.4%)   -4.5% ( -19% -   12%) 0.091
   BrowseDayOfYearTaxoFacets        0.45      (6.5%)        0.43     (10.0%)   -4.3% ( -19% -   13%) 0.106
               OrNotHighLow      321.23      (8.0%)      308.14      (7.5%)   -4.1% ( -18% -   12%) 0.097
               HighSpanNear        2.96      (6.4%)        2.84      (9.5%)   -3.9% ( -18% -   12%) 0.129
       BrowseDateTaxoFacets        0.45      (6.9%)        0.43      (9.6%)   -3.9% ( -19% -   13%) 0.142
       HighIntervalsOrdered        4.12      (4.7%)        3.97      (8.7%)   -3.6% ( -16% -   10%) 0.102
      BrowseMonthSSDVFacets        1.68      (6.6%)        1.63     (11.7%)   -3.5% ( -20% -   15%) 0.239
           HighSloppyPhrase        6.41      (8.6%)        6.18     (10.0%)   -3.5% ( -20% -   16%) 0.233
                     IntNRQ       14.11      (6.2%)       13.62     (11.2%)   -3.5% ( -19% -   14%) 0.225
                   Wildcard       14.90      (7.5%)       14.39      (8.2%)   -3.4% ( -17% -   13%) 0.170
                LowSpanNear        4.53      (5.7%)        4.38      (8.1%)   -3.3% ( -16% -   11%) 0.135
                  OrHighLow      164.11      (6.2%)      158.84      (8.5%)   -3.2% ( -16% -   12%) 0.174
      BrowseMonthTaxoFacets        0.47      (6.1%)        0.45      (9.2%)   -3.2% ( -17% -   12%) 0.193
        LowIntervalsOrdered        3.02      (5.5%)        2.93      (9.1%)   -3.2% ( -16% -   12%) 0.185
            LowSloppyPhrase        8.80      (8.7%)        8.53      (9.7%)   -3.0% ( -19% -   16%) 0.298
        MedIntervalsOrdered        6.46      (5.8%)        6.26     (10.1%)   -3.0% ( -17% -   13%) 0.248
                 AndHighMed       41.11      (6.6%)       39.91      (8.0%)   -2.9% ( -16% -   12%) 0.205
                     Fuzzy1       30.54     (13.2%)       29.75     (14.1%)   -2.6% ( -26% -   28%) 0.550
   BrowseDayOfYearSSDVFacets        1.46      (6.1%)        1.43     (10.6%)   -2.6% ( -18% -   15%) 0.350
                 HighPhrase      104.16      (5.5%)      101.53      (8.9%)   -2.5% ( -16% -   12%) 0.284
               OrHighNotLow      394.47      (6.8%)      385.27      (8.3%)   -2.3% ( -16% -   13%) 0.329
                     Fuzzy2       25.00      (8.9%)       24.42      (9.8%)   -2.3% ( -19% -   18%) 0.440
            MedSloppyPhrase        9.24      (8.3%)        9.03      (9.5%)   -2.3% ( -18% -   16%) 0.421
                 OrHighHigh        4.58      (5.9%)        4.47      (7.7%)   -2.2% ( -14% -   12%) 0.304
               OrNotHighMed      326.95      (6.6%)      319.95      (7.1%)   -2.1% ( -14% -   12%) 0.322
                  LowPhrase      128.75      (5.9%)      126.06      (8.9%)   -2.1% ( -15% -   13%) 0.382
                  MedPhrase       34.71      (5.7%)       34.01      (8.0%)   -2.0% ( -14% -   12%) 0.354
              OrNotHighHigh      303.59      (6.7%)      297.55      (8.0%)   -2.0% ( -15% -   13%) 0.394
               OrHighNotMed      304.72      (5.9%)      299.56      (7.5%)   -1.7% ( -14% -   12%) 0.429
                    Prefix3       62.22      (5.1%)       61.22      (7.5%)   -1.6% ( -13% -   11%) 0.431
                  OrHighMed       32.41      (6.8%)       32.02      (6.8%)   -1.2% ( -13% -   13%) 0.579
       HighTermTitleBDVSort       24.77     (20.2%)       24.53     (18.1%)   -1.0% ( -32% -   46%) 0.871
                    Respell       20.64      (5.3%)       20.47      (7.1%)   -0.8% ( -12% -   12%) 0.682
                AndHighHigh       23.51      (6.0%)       23.34      (6.7%)   -0.7% ( -12% -   12%) 0.727
                    LowTerm      724.06      (7.1%)      719.45      (5.6%)   -0.6% ( -12% -   12%) 0.751
                 AndHighLow      272.87      (5.9%)      271.24      (7.3%)   -0.6% ( -13% -   13%) 0.777
                   PKLookup       90.10      (5.5%)       89.65      (8.8%)   -0.5% ( -14% -   14%) 0.831
                   HighTerm      876.85      (7.3%)      875.67      (7.8%)   -0.1% ( -14% -   16%) 0.955
              OrHighNotHigh      291.95      (8.8%)      291.77      (8.8%)   -0.1% ( -16% -   19%) 0.982
      HighTermDayOfYearSort       37.60     (16.6%)       37.65     (17.9%)    0.1% ( -29% -   41%) 0.981
          HighTermMonthSort       18.43     (18.6%)       18.50     (20.3%)    0.4% ( -32% -   48%) 0.954
                 TermDTSort       41.45     (15.4%)       41.99     (15.1%)    1.3% ( -25% -   37%) 0.786
                    MedTerm      700.62      (6.8%)      711.10      (6.0%)    1.5% ( -10% -   15%) 0.461
   
   ```
   
   Please let me know how this looks to you.
   




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r697049547



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Create a TopScoreDocCollectorManager which uses a shared hit counter to maintain number of hits
+ * and a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopScoreDocCollectorManager
+    implements CollectorManager<TopScoreDocCollector, TopDocs> {
+  private final int numHits;
+  private final ScoreDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+
+  /**
+   * Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the
+   * number of hits to count accurately.
+   *
+   * <p><b>NOTE</b>: If the total hit count of the top docs is less than or exactly {@code
+   * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link
+   * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower
+   * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate
+   * but will also likely make query processing slower.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopScoreDocCollectorManager(
+      int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {

Review comment:
       I need to think about it a little more, but I'm not sure putting the responsibility of guarding against multiple `Collector` creation in the `CollectorManager` feels quite right to me. Technically, there would be nothing wrong with creating multiple collectors to run sequentially if you wanted to (although I'm not sure why you would), so it feels a little over-restrictive maybe? But I really can't think of any reason why someone would do that, and it's simpler than exposing the information to `IndexSearcher` and less error-prone than relying on `IndexSearcher` to check/enforce it. So I think this is probably the best way to go, it just feels a little funny to me for some reason.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r695367943



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       No problem and sounds good on the plan!




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r694981562



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopFieldCollectorManager(
+      Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    this.hitsThresholdChecker =
+        supportsConcurrency
+            ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits))
+            : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));

Review comment:
       let's also store `supportsConcurrency` in a member of this class so that we can check that users never create more than one collector when it is false?

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
##########
@@ -527,7 +503,10 @@ public TopDocs search(Query query, int n) throws IOException {
    *
    * @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()}
    *     clauses.
+   * @deprecated This method is being deprecated in favor of {@link IndexSearcher#search(Query,

Review comment:
       I think we should try to keep `IndexSearcher#search(Query, Collector)` and the `TopXXXCollector#create` factory methods during the 9.x series, as I expect it to be quite commonly used. However in my opinion it would be fine to cut over expert classes like those in the benchmark module without providing such long bw compatibility.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r699834141



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -407,97 +410,14 @@ public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThre
    *     field is indexed both with doc values and points. In this case, there is an assumption that
    *     the same data is stored in these points and doc values.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopFieldCollector create(
       Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
-    if (totalHitsThreshold < 0) {
-      throw new IllegalArgumentException(
-          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
-    }
-
-    return create(
-        sort,
-        numHits,
-        after,
-        HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)),
-        null /* bottomValueChecker */);
-  }
-
-  /**
-   * Same as above with additional parameters to allow passing in the threshold checker and the max
-   * score accumulator.
-   */
-  static TopFieldCollector create(

Review comment:
       Sorry just realized I missed a few comments earlier (they were collapsed on github UI). I think this one and the one from `TopScoreDocCollector.java` should be safe to remove directly even when backporting to 8x, as they are package private and hence ideally no users should be using them (and assuming we don't support hacky access to these methods such as reflection or byte code manipulation) ?

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -192,9 +191,13 @@ public void collect(int doc) throws IOException {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated

Review comment:
       Resolved from the discussion in https://github.com/apache/lucene/pull/240#discussion_r692101093. 




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r697046771



##########
File path: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
##########
@@ -180,6 +185,7 @@ protected int withTopDocs(IndexSearcher searcher, Query q, TopDocs hits) throws
     return res;
   }
 
+  @Deprecated

Review comment:
       I was more thinking along the lines of keeping the change you had but providing a protected method that would allow sub-classes to provide their own `CollectorManager` if they wanted to do so (which would give users a migration path if they were previously providing their own `Collector`). So in 9.0, I think you'd move completely to the `CollectorManager` approach like you had. What do you think of that approach?




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r695366656



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopFieldCollectorManager(
+      Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    this.hitsThresholdChecker =
+        supportsConcurrency
+            ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits))
+            : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));

Review comment:
       Ah I like this idea better than the approach described in https://github.com/apache/lucene/pull/240#discussion_r692722877, since it doesn't require an additional API to be added to `CollectorManager` for thread-safety! I've implemented this in the latest commit. 




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r693359352



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       Also, @zacharymorn, since this appears to be a pre-existing bug that's separate from the change you're working on here, I went ahead and cut an issue for it: [LUCENE-10060](https://issues.apache.org/jira/browse/LUCENE-10060). Let's take our conversation over there so your PR doesn't get too overwhelmed by 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: issues-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r695368539



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Create a TopScoreDocCollectorManager which uses a shared hit counter to maintain number of hits
+ * and a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopScoreDocCollectorManager
+    implements CollectorManager<TopScoreDocCollector, TopDocs> {
+  private final int numHits;
+  private final ScoreDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+
+  /**
+   * Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the
+   * number of hits to count accurately.
+   *
+   * <p><b>NOTE</b>: If the total hit count of the top docs is less than or exactly {@code
+   * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link
+   * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower
+   * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate
+   * but will also likely make query processing slower.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopScoreDocCollectorManager(
+      int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {

Review comment:
       I've implemented the other approach suggested here https://github.com/apache/lucene/pull/240#discussion_r694981562 in commit https://github.com/apache/lucene/pull/240/commits/d21e5f9447523405db1cad2ad9357cabfc260180 for thread-safety. Please let me know how it looks to you.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r699834141



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -407,97 +410,14 @@ public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThre
    *     field is indexed both with doc values and points. In this case, there is an assumption that
    *     the same data is stored in these points and doc values.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopFieldCollector create(
       Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
-    if (totalHitsThreshold < 0) {
-      throw new IllegalArgumentException(
-          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
-    }
-
-    return create(
-        sort,
-        numHits,
-        after,
-        HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)),
-        null /* bottomValueChecker */);
-  }
-
-  /**
-   * Same as above with additional parameters to allow passing in the threshold checker and the max
-   * score accumulator.
-   */
-  static TopFieldCollector create(

Review comment:
       Sorry just realized I missed a few comments earlier (they were collapsed on github UI). I think this one and the one from `TopScoreDocCollector.java` should be safe to remove directly even when backporting to 8x, as they are package private and hence ideally no users should be using them (and assuming we don't support hacky access to these methods such as reflection or byte code manipulation) ?

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -192,9 +191,13 @@ public void collect(int doc) throws IOException {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated

Review comment:
       Resolved from the discussion in https://github.com/apache/lucene/pull/240#discussion_r692101093. 

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -209,61 +212,14 @@ public static TopScoreDocCollector create(int numHits, int totalHitsThreshold) {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopScoreDocCollector create(int numHits, ScoreDoc after, int totalHitsThreshold) {
-    return create(
-        numHits, after, HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)), null);
-  }
-
-  static TopScoreDocCollector create(

Review comment:
       Please see rely in  https://github.com/apache/lucene/pull/240#discussion_r699834141.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -209,61 +212,14 @@ public static TopScoreDocCollector create(int numHits, int totalHitsThreshold) {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopScoreDocCollector create(int numHits, ScoreDoc after, int totalHitsThreshold) {
-    return create(
-        numHits, after, HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)), null);
-  }
-
-  static TopScoreDocCollector create(

Review comment:
       Please see reply in  https://github.com/apache/lucene/pull/240#discussion_r699834141.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r688276995



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+
+  public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    /*
+    nocommit
+    Should the following two be passed in instead? Possible custom initialization based on executor status and slices?
+    On the other hand, in a single-threaded environment, shared HitsThresholdChecker and MaxScoreAccumulator should be fast without lock contention anyway?

Review comment:
       Can you check if luceneutil benchmarks become slower with this change? Hopefully they're not slower and we can keep this logic simple.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r709829843



##########
File path: lucene/classification/src/java/org/apache/lucene/classification/CachingNaiveBayesClassifier.java
##########
@@ -179,10 +179,8 @@ public CachingNaiveBayesClassifier(
         if (query != null) {
           booleanQuery.add(query, BooleanClause.Occur.MUST);
         }
-        TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
-        indexSearcher.search(booleanQuery.build(), totalHitCountCollector);
 
-        int ret = totalHitCountCollector.getTotalHits();
+        int ret = indexSearcher.search(booleanQuery.build(), new TotalHitCountCollectorManager());

Review comment:
       We could make it even better by using `IndexSearcher#count`.

##########
File path: lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java
##########
@@ -276,9 +274,7 @@ private int getWordFreqForClass(String word, Term term) throws IOException {
     if (query != null) {
       booleanQuery.add(query, BooleanClause.Occur.MUST);
     }
-    TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
-    indexSearcher.search(booleanQuery.build(), totalHitCountCollector);
-    return totalHitCountCollector.getTotalHits();
+    return indexSearcher.search(booleanQuery.build(), new TotalHitCountCollectorManager());

Review comment:
       likewise here

##########
File path: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
##########
@@ -180,6 +185,7 @@ protected int withTopDocs(IndexSearcher searcher, Query q, TopDocs hits) throws
     return res;
   }
 
+  @Deprecated

Review comment:
       I suspect it would still be useful to assess the performance of a collector but you probably have a point that if performance is the only thing that the benchmark module reports, then there's a chance that it's not actually used much.
   
   I did a quick GitHub search to see how much code I could find that extends this code:
    - [`"extends ReadTask" createCollector`](https://github.com/search?q=%22extends+ReadTask%22+createCollector&type=code): no hits
    - [`"extends SearchTask" createCollector -SearchWithCollectorTask`](https://github.com/search?q=%22extends+SearchTask%22+createCollector+-SearchWithCollectorTask&type=code): no hits (I had to exclude `SearchWithCollectorTask` or all Lucene forks would be returned).
   
   Given that the benchmark module is quite expert, I'd be ok with breaking hard here.

##########
File path: lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestLargeNumHitsTopDocsCollector.java
##########
@@ -159,19 +159,18 @@ public void testNoPQHitsOrder() throws IOException {
   private void runNumHits(int numHits) throws IOException {
     IndexSearcher searcher = newSearcher(reader);
     LargeNumHitsTopDocsCollector largeCollector = new LargeNumHitsTopDocsCollector(numHits);
-    TopScoreDocCollector regularCollector =
-        TopScoreDocCollector.create(numHits, null, Integer.MAX_VALUE);
+    TopScoreDocCollectorManager regularCollector =
+        new TopScoreDocCollectorManager(numHits, Integer.MAX_VALUE);
 
     searcher.search(testQuery, largeCollector);
-    searcher.search(testQuery, regularCollector);
+    TopDocs topDocs = searcher.search(testQuery, regularCollector);

Review comment:
       keep it called `secondTopDocs`, I think it helped readability

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -407,97 +410,14 @@ public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThre
    *     field is indexed both with doc values and points. In this case, there is an assumption that
    *     the same data is stored in these points and doc values.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopFieldCollector create(
       Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
-    if (totalHitsThreshold < 0) {
-      throw new IllegalArgumentException(
-          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
-    }
-
-    return create(
-        sort,
-        numHits,
-        after,
-        HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)),
-        null /* bottomValueChecker */);
-  }
-
-  /**
-   * Same as above with additional parameters to allow passing in the threshold checker and the max
-   * score accumulator.
-   */
-  static TopFieldCollector create(

Review comment:
       Indeed we don't support access to pkg-private code.

##########
File path: lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java
##########
@@ -179,8 +178,7 @@ protected int countDocsWithClass() throws IOException {
       if (query != null) {
         q.add(query, BooleanClause.Occur.MUST);
       }
-      indexSearcher.search(q.build(), classQueryCountCollector);
-      docCount = classQueryCountCollector.getTotalHits();
+      docCount = indexSearcher.search(q.build(), new TotalHitCountCollectorManager());

Review comment:
       let's simplify by using `IndexSearcher#count`?

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+  private final boolean supportsConcurrency;
+  private boolean collectorCreated;
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopFieldCollectorManager(
+      Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    this.supportsConcurrency = supportsConcurrency;
+    this.hitsThresholdChecker =
+        supportsConcurrency
+            ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits))
+            : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));
+    this.minScoreAcc = supportsConcurrency ? new MaxScoreAccumulator() : null;
+    this.collectors = new ArrayList<>();
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
+    this(sort, numHits, after, totalHitsThreshold, true);
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, int totalHitsThreshold) {
+    this(sort, numHits, null, totalHitsThreshold, true);
+  }
+
+  @Override
+  public TopFieldCollector newCollector() {
+    if (collectorCreated && supportsConcurrency == false) {
+      throw new IllegalStateException(
+          "The instantiated TopFieldCollectorManager does not support concurrency, but multiple collectors are being created");
+    } else {
+      collectorCreated = true;
+    }
+
+    if (sort.fields.length == 0) {
+      throw new IllegalArgumentException("Sort must contain at least one field");
+    }

Review comment:
       Can we move as much validation as possible to the constructor, so that we would detect inconsistencies earlier: when users create the collector manager rather than when Lucene starts collecting hits?

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+  private final boolean supportsConcurrency;
+  private boolean collectorCreated;
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopFieldCollectorManager(
+      Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    this.supportsConcurrency = supportsConcurrency;
+    this.hitsThresholdChecker =
+        supportsConcurrency
+            ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits))
+            : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));
+    this.minScoreAcc = supportsConcurrency ? new MaxScoreAccumulator() : null;
+    this.collectors = new ArrayList<>();
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
+    this(sort, numHits, after, totalHitsThreshold, true);
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, int totalHitsThreshold) {
+    this(sort, numHits, null, totalHitsThreshold, true);
+  }
+
+  @Override
+  public TopFieldCollector newCollector() {
+    if (collectorCreated && supportsConcurrency == false) {
+      throw new IllegalStateException(
+          "The instantiated TopFieldCollectorManager does not support concurrency, but multiple collectors are being created");
+    } else {
+      collectorCreated = true;
+    }
+
+    if (sort.fields.length == 0) {
+      throw new IllegalArgumentException("Sort must contain at least one field");
+    }
+
+    if (numHits <= 0) {
+      throw new IllegalArgumentException(
+          "numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
+    }
+
+    FieldValueHitQueue<FieldValueHitQueue.Entry> queue =
+        FieldValueHitQueue.create(sort.fields, numHits);
+
+    TopFieldCollector collector;
+    if (after == null) {
+      collector =
+          new TopFieldCollector.SimpleFieldCollector(
+              sort, queue, numHits, hitsThresholdChecker, minScoreAcc);
+    } else {
+      if (after.fields == null) {
+        throw new IllegalArgumentException(
+            "after.fields wasn't set; you must pass fillFields=true for the previous search");
+      }
+
+      if (after.fields.length != sort.getSort().length) {
+        throw new IllegalArgumentException(
+            "after.fields has "
+                + after.fields.length
+                + " values but sort has "
+                + sort.getSort().length);
+      }
+
+      collector =
+          new TopFieldCollector.PagingFieldCollector(
+              sort, queue, after, numHits, hitsThresholdChecker, minScoreAcc);
+    }
+
+    collectors.add(collector);
+    return collector;
+  }
+
+  @Override
+  public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
+    final TopFieldDocs[] topDocs = new TopFieldDocs[collectors.size()];
+    int i = 0;
+    for (TopFieldCollector collector : collectors) {
+      topDocs[i++] = collector.topDocs();
+    }
+    return TopDocs.merge(sort, 0, numHits, topDocs);
+  }
+
+  public List<TopFieldCollector> getCollectors() {

Review comment:
       Can we make it pkg-private?




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] jpountz commented on pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #240:
URL: https://github.com/apache/lucene/pull/240#issuecomment-920663920


   @zacharymorn @gsmiller If we try to do everything in a single PR, I worry that this will become very hard to review. I wonder if we should split by replacing the deprecation warnings of `IndexSearcher#search(Query,Collector)` and `TopXXXCollector#create` by some words about how the CollectorManager variants are recommended as they make it possible to search using multiple threads. Then when this PR is merged we can work more iteratively on replacing usage of Collector with CollectorManager, and we could add the deprecation warnings in the end once we have migrated all our code?


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r710741708



##########
File path: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
##########
@@ -180,6 +185,7 @@ protected int withTopDocs(IndexSearcher searcher, Query q, TopDocs hits) throws
     return res;
   }
 
+  @Deprecated

Review comment:
       Sounds good! I've removed this method, and also fixed the commented out code.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r689035631



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+
+  public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    /*
+    nocommit
+    Should the following two be passed in instead? Possible custom initialization based on executor status and slices?
+    On the other hand, in a single-threaded environment, shared HitsThresholdChecker and MaxScoreAccumulator should be fast without lock contention anyway?

Review comment:
       Thanks for the great suggestion! I just ran luceneutil, and it turns out that the changes do get slower due to this thread-safe `HitsThresholdChecker` (confirmed by playing with the atomic long inside a bit). Here is the full benchmark result: 
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
       HighTermTitleBDVSort       52.86     (14.0%)       42.72      (6.6%)  -19.2% ( -34% -    1%) 0.000
          HighTermMonthSort       31.66     (10.7%)       25.90      (6.3%)  -18.2% ( -31% -   -1%) 0.000
      HighTermDayOfYearSort       24.00     (15.7%)       20.02      (8.5%)  -16.6% ( -35% -    9%) 0.000
                 TermDTSort       20.22      (9.0%)       17.13      (6.6%)  -15.3% ( -28% -    0%) 0.000
                    MedTerm     1270.02      (3.9%)     1208.29      (5.1%)   -4.9% ( -13% -    4%) 0.001
                   HighTerm      941.14      (4.0%)      900.05      (5.9%)   -4.4% ( -13% -    5%) 0.006
               OrHighNotMed      531.63      (5.6%)      514.68      (6.2%)   -3.2% ( -14% -    9%) 0.089
                    LowTerm     1112.50      (5.1%)     1081.48      (6.2%)   -2.8% ( -13% -    8%) 0.118
                MedSpanNear       12.50      (2.2%)       12.18      (2.2%)   -2.5% (  -6% -    1%) 0.000
               HighSpanNear       12.82      (1.7%)       12.52      (1.7%)   -2.4% (  -5% -    1%) 0.000
               OrHighNotLow      562.50      (6.9%)      550.15      (5.6%)   -2.2% ( -13% -   11%) 0.269
              OrHighNotHigh      577.10      (5.6%)      565.51      (6.7%)   -2.0% ( -13% -   10%) 0.303
              OrNotHighHigh      633.91      (6.7%)      621.85      (6.0%)   -1.9% ( -13% -   11%) 0.345
       HighIntervalsOrdered        6.94      (3.1%)        6.81      (3.2%)   -1.9% (  -7% -    4%) 0.061
                   PKLookup      174.06      (4.2%)      171.17      (4.5%)   -1.7% (  -9% -    7%) 0.230
                  OrHighLow      283.56      (5.3%)      279.05      (5.8%)   -1.6% ( -12% -   10%) 0.365
        MedIntervalsOrdered       19.00      (3.7%)       18.74      (3.6%)   -1.4% (  -8% -    6%) 0.237
                     Fuzzy1       59.38     (11.6%)       58.65     (12.0%)   -1.2% ( -22% -   25%) 0.740
           HighSloppyPhrase       12.68      (2.7%)       12.53      (2.1%)   -1.1% (  -5% -    3%) 0.135
                     Fuzzy2       63.95      (8.6%)       63.40      (9.0%)   -0.9% ( -17% -   18%) 0.759
            MedSloppyPhrase       17.53      (2.7%)       17.38      (2.2%)   -0.8% (  -5% -    4%) 0.279
            LowSloppyPhrase       23.90      (3.0%)       23.70      (2.8%)   -0.8% (  -6% -    5%) 0.354
                    Respell       39.23      (3.1%)       38.90      (3.1%)   -0.8% (  -6% -    5%) 0.403
                 AndHighMed       61.17      (3.8%)       60.72      (3.8%)   -0.7% (  -8% -    7%) 0.542
                  OrHighMed       48.81      (3.0%)       48.46      (3.0%)   -0.7% (  -6% -    5%) 0.444
        LowIntervalsOrdered       14.64      (2.3%)       14.55      (2.3%)   -0.6% (  -5% -    4%) 0.380
                LowSpanNear       11.65      (1.6%)       11.57      (1.6%)   -0.6% (  -3% -    2%) 0.223
                AndHighHigh       16.57      (3.6%)       16.46      (4.0%)   -0.6% (  -7% -    7%) 0.608
                     IntNRQ       17.25     (28.0%)       17.17     (27.3%)   -0.5% ( -43% -   76%) 0.959
                 OrHighHigh        9.28      (2.5%)        9.24      (2.5%)   -0.4% (  -5% -    4%) 0.635
                  MedPhrase       72.99      (3.3%)       72.75      (3.7%)   -0.3% (  -7% -    6%) 0.763
                  LowPhrase       78.02      (3.5%)       77.77      (4.0%)   -0.3% (  -7% -    7%) 0.786
      BrowseMonthSSDVFacets        3.19      (1.8%)        3.19      (1.7%)   -0.2% (  -3% -    3%) 0.653
   BrowseDayOfYearSSDVFacets        3.01      (1.5%)        3.01      (1.2%)   -0.0% (  -2% -    2%) 0.991
                   Wildcard       27.98      (2.8%)       27.98      (2.7%)    0.0% (  -5% -    5%) 0.981
                 AndHighLow      454.42      (5.4%)      455.10      (4.6%)    0.1% (  -9% -   10%) 0.925
                 HighPhrase      289.77      (5.1%)      290.54      (4.0%)    0.3% (  -8% -    9%) 0.854
               OrNotHighLow      521.43      (6.5%)      523.38      (5.7%)    0.4% ( -11% -   13%) 0.846
                    Prefix3       39.82      (4.8%)       40.02      (3.1%)    0.5% (  -7% -    8%) 0.707
               OrNotHighMed      560.49      (5.8%)      563.51      (5.5%)    0.5% ( -10% -   12%) 0.762
      BrowseMonthTaxoFacets        0.99      (4.8%)        1.00      (4.6%)    0.6% (  -8% -   10%) 0.702
       BrowseDateTaxoFacets        0.94      (4.6%)        0.94      (4.6%)    0.6% (  -8% -   10%) 0.683
   BrowseDayOfYearTaxoFacets        0.94      (4.5%)        0.94      (4.5%)    0.7% (  -7% -   10%) 0.638
   
   ```
   
   To fix the performance regression, but still try to keep the logic simple then, I'm considering adding an additional flag `supportsConcurrency` to the collector managers' constructor like such:
   
   ```
   public TopScoreDocCollectorManager(int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
       ...
       this.hitsThresholdChecker = supportsConcurrency ? 
           HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits)) : 
           HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));
       this.minScoreAcc = supportsConcurrency ? new MaxScoreAccumulator() : null;
     }
   
   public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
      ...
       this.hitsThresholdChecker = supportsConcurrency ? 
           HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits)) : 
           HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));
       this.minScoreAcc = supportsConcurrency ? new MaxScoreAccumulator() : null;
     }
   ```
   
   What do you think about this approach?
   
   




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r699834253



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -209,61 +212,14 @@ public static TopScoreDocCollector create(int numHits, int totalHitsThreshold) {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopScoreDocCollector create(int numHits, ScoreDoc after, int totalHitsThreshold) {
-    return create(
-        numHits, after, HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)), null);
-  }
-
-  static TopScoreDocCollector create(

Review comment:
       Please see rely in  https://github.com/apache/lucene/pull/240#discussion_r699834141.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -209,61 +212,14 @@ public static TopScoreDocCollector create(int numHits, int totalHitsThreshold) {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopScoreDocCollector create(int numHits, ScoreDoc after, int totalHitsThreshold) {
-    return create(
-        numHits, after, HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)), null);
-  }
-
-  static TopScoreDocCollector create(

Review comment:
       Please see reply in  https://github.com/apache/lucene/pull/240#discussion_r699834141.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r695278754



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       Ah, I see the problem. Thanks again for the details! I was overlooking L:521 in `LRUQueryCache` where it always provides a `null` `acceptDocs`. I think the solution to both this issue and the one I detailed in LUCENE-10060 is the same (and is the one you've included in this PR).
   
   If I have some time this week, I'll see if I can write a test case that reliably repro's the issue. If I can, I'll bundle the test and the fix you've written here into a separate PR against LUCENE-10060 that can just get rebased into your change (should be an easy conflict to resolve). If you beat me to it with pushing this PR, I'll just work on the test case and confirm it's fixed.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] javanna commented on pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
javanna commented on pull request #240:
URL: https://github.com/apache/lucene/pull/240#issuecomment-1008729436


   hi @zacharymorn I looked at this issue a long while ago, before you started working on it, and I am now catching up. I see you made great progress on it! I also see that reviewing it as a single change is a bit of a challenge due to the size of the PR. Could I help out splitting this in smaller PRs so that we make progress towards getting the change in, step by step?


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r710741902



##########
File path: lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java
##########
@@ -276,9 +274,7 @@ private int getWordFreqForClass(String word, Term term) throws IOException {
     if (query != null) {
       booleanQuery.add(query, BooleanClause.Occur.MUST);
     }
-    TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
-    indexSearcher.search(booleanQuery.build(), totalHitCountCollector);
-    return totalHitCountCollector.getTotalHits();
+    return indexSearcher.search(booleanQuery.build(), new TotalHitCountCollectorManager());

Review comment:
       Done. 

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+  private final boolean supportsConcurrency;
+  private boolean collectorCreated;
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopFieldCollectorManager(
+      Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    this.supportsConcurrency = supportsConcurrency;
+    this.hitsThresholdChecker =
+        supportsConcurrency
+            ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits))
+            : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));
+    this.minScoreAcc = supportsConcurrency ? new MaxScoreAccumulator() : null;
+    this.collectors = new ArrayList<>();
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
+    this(sort, numHits, after, totalHitsThreshold, true);
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, int totalHitsThreshold) {
+    this(sort, numHits, null, totalHitsThreshold, true);
+  }
+
+  @Override
+  public TopFieldCollector newCollector() {
+    if (collectorCreated && supportsConcurrency == false) {
+      throw new IllegalStateException(
+          "The instantiated TopFieldCollectorManager does not support concurrency, but multiple collectors are being created");
+    } else {
+      collectorCreated = true;
+    }
+
+    if (sort.fields.length == 0) {
+      throw new IllegalArgumentException("Sort must contain at least one field");
+    }

Review comment:
       Ahh good catch! Updated.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #240:
URL: https://github.com/apache/lucene/pull/240#issuecomment-902489306


   > Left a few comments. Looks good overall. Lots of little changes! Those test cases... :)
   
   Thanks @gsmiller for the review! Yes there are indeed many small changes, and more will come :D


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r697155622



##########
File path: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
##########
@@ -180,6 +185,7 @@ protected int withTopDocs(IndexSearcher searcher, Query q, TopDocs hits) throws
     return res;
   }
 
+  @Deprecated

Review comment:
       I see where you are coming from. However, upon looking at this method more closely, I'm afraid this method is effectively not useful, since the result of using this collector was commented out :D :
   
   https://github.com/apache/lucene/blob/3b3f9600c2ea6023f5400a364c0921ba29667584/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java#L119-L121
   
   So I'm now leaning more towards just removing this method altogether if no users ever noticed / complaint about this.  What do you think @gsmiller @jpountz ?




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r710742109



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and
+ * a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments if the
+ * primary sort is by relevancy.
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final FieldDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+  private final List<TopFieldCollector> collectors;
+  private final boolean supportsConcurrency;
+  private boolean collectorCreated;
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopFieldCollectorManager(
+      Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
+    if (totalHitsThreshold < 0) {
+      throw new IllegalArgumentException(
+          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
+    }
+
+    this.sort = sort;
+    this.numHits = numHits;
+    this.after = after;
+    this.supportsConcurrency = supportsConcurrency;
+    this.hitsThresholdChecker =
+        supportsConcurrency
+            ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits))
+            : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits));
+    this.minScoreAcc = supportsConcurrency ? new MaxScoreAccumulator() : null;
+    this.collectors = new ArrayList<>();
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
+    this(sort, numHits, after, totalHitsThreshold, true);
+  }
+
+  /**
+   * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
+   * internal states.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>.
+   *
+   * @param sort the sort criteria (SortFields).
+   * @param numHits the number of results to collect.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   */
+  public TopFieldCollectorManager(Sort sort, int numHits, int totalHitsThreshold) {
+    this(sort, numHits, null, totalHitsThreshold, true);
+  }
+
+  @Override
+  public TopFieldCollector newCollector() {
+    if (collectorCreated && supportsConcurrency == false) {
+      throw new IllegalStateException(
+          "The instantiated TopFieldCollectorManager does not support concurrency, but multiple collectors are being created");
+    } else {
+      collectorCreated = true;
+    }
+
+    if (sort.fields.length == 0) {
+      throw new IllegalArgumentException("Sort must contain at least one field");
+    }
+
+    if (numHits <= 0) {
+      throw new IllegalArgumentException(
+          "numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
+    }
+
+    FieldValueHitQueue<FieldValueHitQueue.Entry> queue =
+        FieldValueHitQueue.create(sort.fields, numHits);
+
+    TopFieldCollector collector;
+    if (after == null) {
+      collector =
+          new TopFieldCollector.SimpleFieldCollector(
+              sort, queue, numHits, hitsThresholdChecker, minScoreAcc);
+    } else {
+      if (after.fields == null) {
+        throw new IllegalArgumentException(
+            "after.fields wasn't set; you must pass fillFields=true for the previous search");
+      }
+
+      if (after.fields.length != sort.getSort().length) {
+        throw new IllegalArgumentException(
+            "after.fields has "
+                + after.fields.length
+                + " values but sort has "
+                + sort.getSort().length);
+      }
+
+      collector =
+          new TopFieldCollector.PagingFieldCollector(
+              sort, queue, after, numHits, hitsThresholdChecker, minScoreAcc);
+    }
+
+    collectors.add(collector);
+    return collector;
+  }
+
+  @Override
+  public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
+    final TopFieldDocs[] topDocs = new TopFieldDocs[collectors.size()];
+    int i = 0;
+    for (TopFieldCollector collector : collectors) {
+      topDocs[i++] = collector.topDocs();
+    }
+    return TopDocs.merge(sort, 0, numHits, topDocs);
+  }
+
+  public List<TopFieldCollector> getCollectors() {

Review comment:
       So this method was added to support early termination detection in tests like the following:
   
   ```
   collectorManager.getCollectors().stream().anyMatch(TopFieldCollector::isEarlyTerminated)
   ```
   
   I think Lucene users may have the need for this check as well? 

##########
File path: lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestLargeNumHitsTopDocsCollector.java
##########
@@ -159,19 +159,18 @@ public void testNoPQHitsOrder() throws IOException {
   private void runNumHits(int numHits) throws IOException {
     IndexSearcher searcher = newSearcher(reader);
     LargeNumHitsTopDocsCollector largeCollector = new LargeNumHitsTopDocsCollector(numHits);
-    TopScoreDocCollector regularCollector =
-        TopScoreDocCollector.create(numHits, null, Integer.MAX_VALUE);
+    TopScoreDocCollectorManager regularCollector =
+        new TopScoreDocCollectorManager(numHits, Integer.MAX_VALUE);
 
     searcher.search(testQuery, largeCollector);
-    searcher.search(testQuery, regularCollector);
+    TopDocs topDocs = searcher.search(testQuery, regularCollector);

Review comment:
       Updated. 




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r696243140



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       Thanks @gsmiller for the fix! I've approved that PR and reverted the hack done in this PR. Will pull in your change once it's merged into `main`. 




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r697150848



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Create a TopScoreDocCollectorManager which uses a shared hit counter to maintain number of hits
+ * and a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopScoreDocCollectorManager
+    implements CollectorManager<TopScoreDocCollector, TopDocs> {
+  private final int numHits;
+  private final ScoreDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+
+  /**
+   * Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the
+   * number of hits to count accurately.
+   *
+   * <p><b>NOTE</b>: If the total hit count of the top docs is less than or exactly {@code
+   * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link
+   * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower
+   * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate
+   * but will also likely make query processing slower.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopScoreDocCollectorManager(
+      int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {

Review comment:
       I think for the purpose of ensuring correct / thread-safe usage of Lucene provided `TopXXXCollectorManager` classes & `IndexSearch#search` API, this would probably be the simplest and yet effective solution, without any special change needed from users. 
   
   The other scenarios you are considering, such as creating multiple collectors to run sequentially, will probably happen when users need to do something special about these classes and API. For that, users can still achieve it via various means, such as providing their custom implementations of `CollectorManager`, overriding the existing method from one of the `TopXXXCollectorManager` to remove the check, or simply just using `TopXXXCollector` instead of `CollectorManager` and also skipping the `reduce` step. So users should still have the freedom to customize these APIs to tailor their special needs?




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r710741804



##########
File path: lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java
##########
@@ -179,8 +178,7 @@ protected int countDocsWithClass() throws IOException {
       if (query != null) {
         q.add(query, BooleanClause.Occur.MUST);
       }
-      indexSearcher.search(q.build(), classQueryCountCollector);
-      docCount = classQueryCountCollector.getTotalHits();
+      docCount = indexSearcher.search(q.build(), new TotalHitCountCollectorManager());

Review comment:
       Done. Also updated the implementation of `IndexSearcher#count` to use `TotalHitCountCollectorManager`. 

##########
File path: lucene/classification/src/java/org/apache/lucene/classification/CachingNaiveBayesClassifier.java
##########
@@ -179,10 +179,8 @@ public CachingNaiveBayesClassifier(
         if (query != null) {
           booleanQuery.add(query, BooleanClause.Occur.MUST);
         }
-        TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
-        indexSearcher.search(booleanQuery.build(), totalHitCountCollector);
 
-        int ret = totalHitCountCollector.getTotalHits();
+        int ret = indexSearcher.search(booleanQuery.build(), new TotalHitCountCollectorManager());

Review comment:
       Done.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #240:
URL: https://github.com/apache/lucene/pull/240#issuecomment-920606951


   > Sorry @zacharymorn I have not forgotten about it but the change is very large and it's hard to find enough adjacent time to review it. I'll do my best to find time in the coming week.
   
   No worry @jpountz , this PR is indeed large and time consuming! Please take your time and I appreciate your help on the review!


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r694404654



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       Thanks @gsmiller for the additional context & a new dedicated ticket. Just to provide some additional context below to wrap up the discussion in this PR before we cut over.
   
   This issue was discovered during my nightly test run with the following seed `./gradlew test --tests TestDrillSideways.testRandom -Dtests.seed=69A3EF02D8E3465E -Dtests.nightly=true` on commit https://github.com/apache/lucene/pull/240/commits/4af77405fcb914d497d82eb60f0011f93079ec7b . The failing stack trace looks something like this:
   
   ```
   org.apache.lucene.facet.TestDrillSideways > test suite's output saved to /Users/xichen/IdeaProjects/lucene/lucene/facet/build/test-results/test/outputs/OUTPUT-org.apache.lucene.facet.TestDrillSideways.txt, copied below:
      >     java.lang.AssertionError: expected:<56> but was:<59>
      >         at __randomizedtesting.SeedInfo.seed([69A3EF02D8E3465E:1BEFCA0D6983F02D]:0)
      >         at org.junit.Assert.fail(Assert.java:89)
      >         at org.junit.Assert.failNotEquals(Assert.java:835)
      >         at org.junit.Assert.assertEquals(Assert.java:647)
      >         at org.junit.Assert.assertEquals(Assert.java:633)
      >         at org.apache.lucene.facet.TestDrillSideways.verifyEquals(TestDrillSideways.java:1607)
      >         at org.apache.lucene.facet.TestDrillSideways.testRandom(TestDrillSideways.java:1185)
   ```
   
   and the following difference with verbose output
   
   ```
     1>     dim0 topN=2 (vs 2 unique values)
     1>       actual
     1>         0: [e1 9d 80 e1 9d 97 e1 9d 82 e1 9d 92]: 74
     1>         1: [ea a5 aa ea a5 a2 ea a5 a0]: 59
     1>       expected (unsorted)
     1>         0: [ea a5 aa ea a5 a2 ea a5 a0]: 56
     1>         1: [e1 9d 80 e1 9d 97 e1 9d 82 e1 9d 92]: 73
      >     java.lang.AssertionError: expected:<56> but was:<59>
   ```
   
   The failing test complained there are more docs collected than expected.
   
   As I have limited context on drillsideway / drilldown logic, I ended up spending quite a few hours on debugging this, and found the issue ultimately came from this caching issue. Specifically, here are the main key difference in the call chain that contributed to this difference:
   
   #### Expected / existing entry path from `DrillSideways#search(Query, Collector)` with the hack
   ```
   DrillSideways#search(Query, Collector) -> IndexSearcher#createWeight -> NOT cached weight -> ... -> DrillSidewaysScorer#doUnionScoring with INSTANTIATED acceptDocs
   ```
   With this path, some docs will not be collected as `acceptDocs.get(docID) == false`. The hit collector also uses the same `acceptDocs` so the number of matches between hit collector and drillsideway / drilldown collector are also the same.
   
   #### With this change, using the entry path from `DrillSideways#searchSequentially(Query, CollectorManager)` and without the hack: 
   
   ```
   DrillSideways#search(Query, Collector) -> IndexSearcher#createWeight -> cached weight -> ... -> DrillSidewaysScorer#doUnionScoring with NULL acceptDocs (NULL was passed in via the LRUQueryCache code path above)
   ```
   
   With this path, some deleted docs will be collected as `acceptDocs == null`.  As the hit collector uses an instantiated `acceptDocs`, here drillsideway / drilldown collector will also collect more docs compared with hit collector.
   
   Hope this clarifies. 
   
   ----
   
   I think for the purpose of this PR, we may still need to leave this hack in since otherwise this may break nightly tests once merged. As it sounds like you discovered another issue related to this, and that needs to be addressed as part of  https://issues.apache.org/jira/browse/LUCENE-10060 as well, please feel free to take on that one independently from this, and I can resolve any merge conflict in `DrillSideways` had you merged that change earlier than this one.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] zacharymorn commented on pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #240:
URL: https://github.com/apache/lucene/pull/240#issuecomment-921456717


   > I left some comments but the approach that you took so far looks good to me. I see that we still need to migrate some collectors like LargeNumHitsTopDocsCollector or the internal JoinUtil collectors to a CollectorManager?
   
   Thanks @jpountz for the review! Indeed there are more that need to be migrated as well. In fact, I have been working on the 2nd PR to migrate more "easy" ones, and after that, we still have the following: 
   
   #### Explanation related
   `ExplanationAsserter`
   `MatchesAsserter`
   
   #### Facet related
   `FacetsCollector`
   `RandomSamplingFacetsCollector`
   
   #### Grouping related
   `GroupFacetCollector`
   `TopGroupsCollector`
   `BlockGroupingCollector`
   `FirstPassGroupingCollector`
   `AllGroupHeadsCollector`
   `AllGroupsCollector`
   
   #### Global ordinal related
   `GlobalOrdinalsCollector`
   `GlobalOrdinalsWithScoreCollector`
   
   #### Anonymous classes
   Anonymous `Collector` used in `JoinUtil`
   Anonymous `Collector` used in `TestJoinUtil`
   Anonymous `Collector` used in `QueryUtils`
   
   #### Profiling related
   `ProfilerCollector`
   `MemoryAccountingBitsetCollector`
   `MonitorQueryCollector`
   
   #### Counting related
   `DocValuesStatsCollector`
   `MyHitCollector`
   `MatchCollector`
   
   #### Result diversification related
   `DistinctValuesCollector`
   `DiversifiedTopDocsCollector`
   
   #### Other
   `CachingCollector`
   `TerminateAfterCollector`
   `TimeLimitingCollector`
   `LargeNumHitsTopDocsCollector`
   
   As each of these categories / collectors will likely require more thoughts / time to understand the current logic, and come up with `CollectorManager` implementation that's also thread-safe, I'm thinking to create follow-up Jira sub-tickets for each of these categories to track them. What do you think?
   
   > I wonder if we should split by replacing the deprecation warnings of IndexSearcher#search(Query,Collector) and TopXXXCollector#create by some words about how the CollectorManager variants are recommended as they make it possible to search using multiple threads. Then when this PR is merged we can work more iteratively on replacing usage of Collector with CollectorManager, and we could add the deprecation warnings in the end once we have migrated all our code?
   
   Yeah I can definitely follow this strategy as well. The only concern I may have is that, as can be seen above, it may still take some time to migrate all existing collectors to collectorManagers. So using suggestion comment for now and adding  deprecation warning at the end may likely see a few more collectors without collectorManagers added accidentally (i.e. folks may not notice the comment when they just code against the `IndexSearch#search(Query, Collector)` API)?
   


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r697667815



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Create a TopScoreDocCollectorManager which uses a shared hit counter to maintain number of hits
+ * and a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopScoreDocCollectorManager
+    implements CollectorManager<TopScoreDocCollector, TopDocs> {
+  private final int numHits;
+  private final ScoreDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+
+  /**
+   * Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the
+   * number of hits to count accurately.
+   *
+   * <p><b>NOTE</b>: If the total hit count of the top docs is less than or exactly {@code
+   * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link
+   * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower
+   * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate
+   * but will also likely make query processing slower.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopScoreDocCollectorManager(
+      int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {

Review comment:
       I think that all makes sense. +1 to this approach




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r695909915



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       @zacharymorn FYI, I repro'd the bug with a unit test and I have a [PR](https://github.com/apache/lucene/pull/261) out to fix this. I requested you as a reviewer. I _think_ with the fix in my PR, this hack will no longer be necessary.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] gsmiller commented on a change in pull request #240: LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager)

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #240:
URL: https://github.com/apache/lucene/pull/240#discussion_r692105614



##########
File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
##########
@@ -659,9 +614,12 @@ public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOEx
    */
   public <C extends Collector, T> T search(Query query, CollectorManager<C, T> collectorManager)
       throws IOException {
-    if (executor == null || leafSlices.length <= 1) {
+    if (executor == null || leafSlices.length == 0) {

Review comment:
       I'm a little confused why this condition changed. Wouldn't we still want to execute this logic if there is a single slice (avoiding the `reduce` step, etc.)? Wouldn't `leafSlices == 0` indicate there's nothing to actually search over?

##########
File path: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
##########
@@ -180,6 +185,7 @@ protected int withTopDocs(IndexSearcher searcher, Query q, TopDocs hits) throws
     return res;
   }
 
+  @Deprecated

Review comment:
       I wonder if anyone out there has sub-classed this to provide their own `Collector` through overriding this method? It's possible right? Would we want to support this with a `CollectorManager` now instead? You might consider creating a new protected method--`createCollectorManager`--that sub-classes could override if they want, then add javadoc here with a `@lucene.deprecated` tag pointing users to the new method.
   
   Also, if this change is going onto `main`, I might kill this method entirely and then add it back in as deprecated if you choose to backport to 8x.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -383,9 +382,13 @@ protected void updateMinCompetitiveScore(Scorable scorer) throws IOException {
    *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
    *     count accurate, but this will also make query processing slower.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated

Review comment:
       Same question/comment about the need to maintain this deprecated method on `main` as above.

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
##########
@@ -527,7 +503,10 @@ public TopDocs search(Query query, int n) throws IOException {
    *
    * @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()}
    *     clauses.
+   * @deprecated This method is being deprecated in favor of {@link IndexSearcher#search(Query,

Review comment:
       I think we like the `@lucene.deprecated` tag?
   
   Is your reasoning behind keeping this in there for now that all of our uses of this (internal to Lucene) haven't yet migrated as part of this change? Would the plan me to migrate all usages off of this internally and then actually remove it on `main`/9.0, or are you thinking of keeping it around until 10.0? I think our backwards compatibility [policy](https://cwiki.apache.org/confluence/display/LUCENE/BackwardsCompatibility) is such that we could just directly remove this on `main`/9.0, but then leave it like you have it (marked deprecated) if you choose to backport this to 8x. Since this method is so fundamental though, I could easily see an argument to keep it around for an extra major release to give users more time to migrate. Then again, the migration path seems pretty straight-forward. What do you think?

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -407,97 +410,14 @@ public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThre
    *     field is indexed both with doc values and points. In this case, there is an assumption that
    *     the same data is stored in these points and doc values.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated

Review comment:
       Same question/comment about the need to maintain this deprecated method on `main` as above.

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FixedBitSetCollector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectorManager;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+
+/** Test utility collector that uses FixedBitSet to record hits. */
+public class FixedBitSetCollector extends SimpleCollector {
+  private FixedBitSet hits;

Review comment:
       This could be `final`

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -192,9 +191,13 @@ public void collect(int doc) throws IOException {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated

Review comment:
       Same question/comment about the need to maintain this deprecated method on `main` as above.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
##########
@@ -209,61 +212,14 @@ public static TopScoreDocCollector create(int numHits, int totalHitsThreshold) {
    *
    * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
    * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopScoreDocCollector create(int numHits, ScoreDoc after, int totalHitsThreshold) {
-    return create(
-        numHits, after, HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)), null);
-  }
-
-  static TopScoreDocCollector create(

Review comment:
       Similar comment as above regarding whether or not your intention is to not be backwards compatible on 9.0.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -407,97 +410,14 @@ public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThre
    *     field is indexed both with doc values and points. In this case, there is an assumption that
    *     the same data is stored in these points and doc values.
    * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
+   * @deprecated This method is being deprecated in favor of using the constructor of {@link
+   *     TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
    */
+  @Deprecated
   public static TopFieldCollector create(
       Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
-    if (totalHitsThreshold < 0) {
-      throw new IllegalArgumentException(
-          "totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
-    }
-
-    return create(
-        sort,
-        numHits,
-        after,
-        HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)),
-        null /* bottomValueChecker */);
-  }
-
-  /**
-   * Same as above with additional parameters to allow passing in the threshold checker and the max
-   * score accumulator.
-   */
-  static TopFieldCollector create(

Review comment:
       Since you're fully killing two methods here, does that mean that you're not intending to maintain back compat on 9.0? I assume these would get added back in and marked deprecated if backporting to 8x? I think it's reasonable, just checking what your intention is.

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FixedBitSetCollector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectorManager;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+
+/** Test utility collector that uses FixedBitSet to record hits. */
+public class FixedBitSetCollector extends SimpleCollector {
+  private FixedBitSet hits;
+  private int docBase;
+
+  public FixedBitSetCollector(int maxDoc) {
+    hits = new FixedBitSet(maxDoc);
+  }
+
+  @Override
+  public ScoreMode scoreMode() {
+    return ScoreMode.COMPLETE_NO_SCORES;
+  }
+
+  @Override
+  protected void doSetNextReader(LeafReaderContext context) {
+    docBase = context.docBase;
+  }
+
+  @Override
+  public void collect(int doc) {
+    hits.set(docBase + doc);
+  }
+
+  public FixedBitSet getHits() {
+    return hits;
+  }
+
+  public static CollectorManager<FixedBitSetCollector, FixedBitSet> create(int maxDoc) {

Review comment:
       nit: Maybe name this something like `createManager`? I would expect a method named `create` to create an instance of `FixedBitSetCollector` here, so this made me do a double-take.

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Create a TopScoreDocCollectorManager which uses a shared hit counter to maintain number of hits
+ * and a shared {@link MaxScoreAccumulator} to propagate the minimum score across segments
+ *
+ * <p>Note that a new collectorManager should be created for each search due to its internal states.
+ */
+public class TopScoreDocCollectorManager
+    implements CollectorManager<TopScoreDocCollector, TopDocs> {
+  private final int numHits;
+  private final ScoreDoc after;
+  private final HitsThresholdChecker hitsThresholdChecker;
+  private final MaxScoreAccumulator minScoreAcc;
+
+  /**
+   * Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the
+   * number of hits to count accurately.
+   *
+   * <p><b>NOTE</b>: If the total hit count of the top docs is less than or exactly {@code
+   * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link
+   * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower
+   * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate
+   * but will also likely make query processing slower.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
+   * <code>numHits</code>, and fill the array with sentinel objects.
+   *
+   * @param numHits the number of results to collect.
+   * @param after the previous doc after which matching docs will be collected.
+   * @param totalHitsThreshold the number of docs to count accurately. If the query matches more
+   *     than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
+   *     hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
+   *     count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
+   *     count accurate, but this will also make query processing slower.
+   * @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
+   */
+  public TopScoreDocCollectorManager(
+      int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {

Review comment:
       I'm not totally sure what I think of this yet (although I understand the desire to have it). It makes me a little nervous since a user could create one of these with `supportsConcurrency == false` then pass it to an `IndexSearcher` with an `executor` set. That would break in interesting ways right? The search would run concurrently but would be using non-threadsafe structures under the hood? Maybe you've thought about this more and it's actually safe? Is it worth having `IndexSearcher` "check" that this was setup to support concurrency before executing a concurrent search (e.g., assert that it was setup with `supportsConcurrency == true`)?

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FixedBitSetCollector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectorManager;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+
+/** Test utility collector that uses FixedBitSet to record hits. */
+public class FixedBitSetCollector extends SimpleCollector {
+  private FixedBitSet hits;
+  private int docBase;
+
+  public FixedBitSetCollector(int maxDoc) {
+    hits = new FixedBitSet(maxDoc);
+  }
+
+  @Override
+  public ScoreMode scoreMode() {
+    return ScoreMode.COMPLETE_NO_SCORES;
+  }
+
+  @Override
+  protected void doSetNextReader(LeafReaderContext context) {
+    docBase = context.docBase;
+  }
+
+  @Override
+  public void collect(int doc) {
+    hits.set(docBase + doc);
+  }
+
+  public FixedBitSet getHits() {
+    return hits;
+  }
+
+  public static CollectorManager<FixedBitSetCollector, FixedBitSet> create(int maxDoc) {
+    return new CollectorManager<>() {
+      @Override
+      public FixedBitSetCollector newCollector() {
+        return new FixedBitSetCollector(maxDoc);
+      }
+
+      @Override
+      public FixedBitSet reduce(Collection<FixedBitSetCollector> collectors) {
+        FixedBitSet result = new FixedBitSet(maxDoc);
+        collectors.stream().forEach(c -> result.or(c.getHits()));

Review comment:
       nit: I don't think you need the `.stream()` in here. You should be able to call `.forEach` directly on the `Collection` if you care to do so.

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
##########
@@ -520,10 +458,39 @@ private DrillDownQuery getDrillDownQuery(
   }
 
   @SuppressWarnings("unchecked")
-  private <R> ConcurrentDrillSidewaysResult<R> searchSequentially(
-      final DrillDownQuery query, final CollectorManager<?, R> hitCollectorManager)
+  private <C extends Collector, R> ConcurrentDrillSidewaysResult<R> searchSequentially(
+      final DrillDownQuery query, final CollectorManager<C, R> hitCollectorManager)
       throws IOException {
 
+    // This mirrors a similar hack from DrillSideways#search(query, collector).
+    // Without this cache, LRU cache will be used, causing acceptDocs to be null during collection

Review comment:
       Good find. I think you mean "With this hack" not "Without this cache"? Also, I'm not very familiar with where this caching happens. Could you give me a pointer so I can understand this a bit better?




-- 
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-unsubscribe@lucene.apache.org

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



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