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/19 14:23:27 UTC

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

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