You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2023/06/30 13:37:34 UTC

[lucene] branch branch_9x updated (b120b80c6a4 -> de07bdd88dd)

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a change to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git


    from b120b80c6a4 Remove usage and add some legacy java.util classes to forbiddenapis (Stack, Hashtable, Vector) (#12404)
     new 5fc16bc310a Speed up NumericDocValuesWriter with index sorting (#12381)
     new 856647dde6f Add a thread safe CachingLeafSlicesSupplier to compute and cache the LeafSlices used with concurrent segment (#12374)
     new aa047475058 Assign a dummy simScorer in TermsWeight if score is not needed (#12383)
     new de07bdd88dd Add a post-collection hook to LeafCollector. (#12380)

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 lucene/CHANGES.txt                                 | 10 ++-
 .../org/apache/lucene/index/NormValuesWriter.java  |  3 +-
 .../lucene/index/NumericDocValuesWriter.java       | 54 +++++++++++++---
 .../org/apache/lucene/search/CachingCollector.java | 71 +++++++++++----------
 .../apache/lucene/search/FilterLeafCollector.java  |  5 ++
 .../org/apache/lucene/search/IndexSearcher.java    | 72 +++++++++++++++++++---
 .../org/apache/lucene/search/LeafCollector.java    |  9 +++
 .../org/apache/lucene/search/MultiCollector.java   | 10 +++
 .../java/org/apache/lucene/search/TermQuery.java   | 17 ++++-
 .../apache/lucene/search/TestCachingCollector.java |  2 +
 .../org/apache/lucene/search/TestTermQuery.java    | 51 +++++++++++++++
 .../apache/lucene/facet/DrillSidewaysScorer.java   | 17 +++++
 .../org/apache/lucene/facet/FacetsCollector.java   | 19 +++---
 .../search/grouping/BlockGroupingCollector.java    | 13 ++--
 .../search/grouping/GroupFacetCollector.java       | 11 ++--
 .../search/grouping/TermGroupFacetCollector.java   |  8 ---
 .../suggest/document/SuggestIndexSearcher.java     |  5 +-
 .../suggest/document/TopSuggestDocsCollector.java  | 19 +++---
 .../lucene/tests/search/AssertingCollector.java    | 13 +++-
 .../tests/search/AssertingIndexSearcher.java       |  4 +-
 .../tests/search/AssertingLeafCollector.java       |  8 +++
 21 files changed, 320 insertions(+), 101 deletions(-)


[lucene] 01/04: Speed up NumericDocValuesWriter with index sorting (#12381)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 5fc16bc310a017510d4800e5388cb2c80b946b70
Author: zhangchao <80...@qq.com>
AuthorDate: Fri Jun 30 20:56:56 2023 +0800

    Speed up NumericDocValuesWriter with index sorting (#12381)
---
 lucene/CHANGES.txt                                 |  2 +
 .../org/apache/lucene/index/NormValuesWriter.java  |  3 +-
 .../lucene/index/NumericDocValuesWriter.java       | 54 ++++++++++++++++++----
 3 files changed, 50 insertions(+), 9 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 0bfaa237675..e1fccc12de3 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -29,6 +29,8 @@ Optimizations
 
 * GITHUB#12385: Restore parallel knn query rewrite across segments rather than slices (Luca Cavanna)
 
+* GITHUB#12381: Speed up NumericDocValuesWriter with index sorting. (Chao Zhang)
+
 Bug Fixes
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
index 8fe73768f61..dfd98ab291e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
@@ -76,7 +76,8 @@ class NormValuesWriter {
           NumericDocValuesWriter.sortDocValues(
               state.segmentInfo.maxDoc(),
               sortMap,
-              new BufferedNorms(values, docsWithField.iterator()));
+              new BufferedNorms(values, docsWithField.iterator()),
+              sortMap.size() == docsWithField.cardinality());
     } else {
       sorted = null;
     }
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
index f4501108643..09bef657b2d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
@@ -78,9 +78,14 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
     return new BufferedNumericDocValues(finalValues, docsWithField.iterator());
   }
 
-  static NumericDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues)
+  static NumericDVs sortDocValues(
+      int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues, boolean dense)
       throws IOException {
-    FixedBitSet docsWithField = new FixedBitSet(maxDoc);
+    FixedBitSet docsWithField = null;
+    if (dense == false) {
+      docsWithField = new FixedBitSet(maxDoc);
+    }
+
     long[] values = new long[maxDoc];
     while (true) {
       int docID = oldDocValues.nextDoc();
@@ -88,7 +93,9 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
         break;
       }
       int newDocID = sortMap.oldToNew(docID);
-      docsWithField.set(newDocID);
+      if (docsWithField != null) {
+        docsWithField.set(newDocID);
+      }
       values[newDocID] = oldDocValues.longValue();
     }
     return new NumericDVs(values, docsWithField);
@@ -114,7 +121,9 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
     final NumericDVs sorted;
     if (sortMap != null) {
       NumericDocValues oldValues = new BufferedNumericDocValues(values, docsWithField.iterator());
-      sorted = sortDocValues(sortMap.size(), sortMap, oldValues);
+      sorted =
+          sortDocValues(
+              sortMap.size(), sortMap, oldValues, sortMap.size() == docsWithField.cardinality());
     } else {
       sorted = null;
     }
@@ -197,10 +206,10 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
 
     @Override
     public int nextDoc() {
-      if (docID + 1 == dvs.docsWithField.length()) {
+      if (docID + 1 == dvs.maxDoc()) {
         docID = NO_MORE_DOCS;
       } else {
-        docID = dvs.docsWithField.nextSetBit(docID + 1);
+        docID = dvs.advance(docID + 1);
       }
       return docID;
     }
@@ -214,7 +223,7 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
     public boolean advanceExact(int target) throws IOException {
       // needed in IndexSorter#{Long|Int|Double|Float}Sorter
       docID = target;
-      return dvs.docsWithField.get(target);
+      return dvs.advanceExact(target);
     }
 
     @Override
@@ -225,7 +234,7 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
     @Override
     public long cost() {
       if (cost == -1) {
-        cost = dvs.docsWithField.cardinality();
+        cost = dvs.cost();
       }
       return cost;
     }
@@ -234,10 +243,39 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
   static class NumericDVs {
     private final long[] values;
     private final BitSet docsWithField;
+    private final int maxDoc;
 
     NumericDVs(long[] values, BitSet docsWithField) {
       this.values = values;
       this.docsWithField = docsWithField;
+      this.maxDoc = values.length;
+    }
+
+    int maxDoc() {
+      return maxDoc;
+    }
+
+    private boolean advanceExact(int target) {
+      if (docsWithField != null) {
+        return docsWithField.get(target);
+      }
+      return true;
+    }
+
+    private int advance(int target) {
+      if (docsWithField != null) {
+        return docsWithField.nextSetBit(target);
+      }
+
+      // Only called when target is less than maxDoc
+      return target;
+    }
+
+    private long cost() {
+      if (docsWithField != null) {
+        return docsWithField.cardinality();
+      }
+      return maxDoc;
     }
   }
 }


[lucene] 04/04: Add a post-collection hook to LeafCollector. (#12380)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit de07bdd88dd124da1da8709e337f2b5839f6207d
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Fri Jun 30 15:19:35 2023 +0200

    Add a post-collection hook to LeafCollector. (#12380)
    
    This adds `LeafCollector#finish` as a per-segment post-collection hook. While
    it was already possible to do this sort of things on top of the collector API
    before, a downside is that the last leaf would need to be post-collected in the
    current thread instead of using the executor, which is a missed opportunity for
    making queries concurrent.
---
 lucene/CHANGES.txt                                 |  4 +-
 .../org/apache/lucene/search/CachingCollector.java | 71 +++++++++++-----------
 .../apache/lucene/search/FilterLeafCollector.java  |  5 ++
 .../org/apache/lucene/search/IndexSearcher.java    |  3 +
 .../org/apache/lucene/search/LeafCollector.java    |  9 +++
 .../org/apache/lucene/search/MultiCollector.java   | 10 +++
 .../apache/lucene/search/TestCachingCollector.java |  2 +
 .../apache/lucene/facet/DrillSidewaysScorer.java   | 17 ++++++
 .../org/apache/lucene/facet/FacetsCollector.java   | 19 +++---
 .../search/grouping/BlockGroupingCollector.java    | 13 ++--
 .../search/grouping/GroupFacetCollector.java       | 11 ++--
 .../search/grouping/TermGroupFacetCollector.java   |  8 ---
 .../suggest/document/SuggestIndexSearcher.java     |  5 +-
 .../suggest/document/TopSuggestDocsCollector.java  | 19 +++---
 .../lucene/tests/search/AssertingCollector.java    | 13 +++-
 .../tests/search/AssertingIndexSearcher.java       |  4 +-
 .../tests/search/AssertingLeafCollector.java       |  8 +++
 17 files changed, 141 insertions(+), 80 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ef5c9e496a1..b931b32eda2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -11,7 +11,9 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+
+* GITHUB#12383: Introduced LeafCollector#finish, a hook that runs after
+  collection has finished running on a leaf. (Adrien Grand)
 
 Improvements
 ---------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java b/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
index d065efd3406..fd45666d33a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
@@ -66,7 +66,6 @@ public abstract class CachingCollector extends FilterCollector {
     List<LeafReaderContext> contexts;
     List<int[]> docs;
     int maxDocsToCache;
-    NoScoreCachingLeafCollector lastCollector;
 
     NoScoreCachingCollector(Collector in, int maxDocsToCache) {
       super(in);
@@ -76,7 +75,7 @@ public abstract class CachingCollector extends FilterCollector {
     }
 
     protected NoScoreCachingLeafCollector wrap(LeafCollector in, int maxDocsToCache) {
-      return new NoScoreCachingLeafCollector(in, maxDocsToCache);
+      return new NoScoreCachingLeafCollector(in, maxDocsToCache, this);
     }
 
     // note: do *not* override needScore to say false. Just because we aren't caching the score
@@ -85,13 +84,12 @@ public abstract class CachingCollector extends FilterCollector {
 
     @Override
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
-      postCollection();
       final LeafCollector in = this.in.getLeafCollector(context);
-      if (contexts != null) {
-        contexts.add(context);
-      }
       if (maxDocsToCache >= 0) {
-        return lastCollector = wrap(in, maxDocsToCache);
+        if (contexts != null) {
+          contexts.add(context);
+        }
+        return wrap(in, maxDocsToCache);
       } else {
         return in;
       }
@@ -103,33 +101,16 @@ public abstract class CachingCollector extends FilterCollector {
       this.docs = null;
     }
 
-    protected void postCollect(NoScoreCachingLeafCollector collector) {
-      final int[] docs = collector.cachedDocs();
-      maxDocsToCache -= docs.length;
-      this.docs.add(docs);
-    }
-
-    private void postCollection() {
-      if (lastCollector != null) {
-        if (!lastCollector.hasCache()) {
-          invalidate();
-        } else {
-          postCollect(lastCollector);
-        }
-        lastCollector = null;
-      }
-    }
-
     protected void collect(LeafCollector collector, int i) throws IOException {
       final int[] docs = this.docs.get(i);
       for (int doc : docs) {
         collector.collect(doc);
       }
+      collector.finish();
     }
 
     @Override
     public void replay(Collector other) throws IOException {
-      postCollection();
       if (!isCached()) {
         throw new IllegalStateException(
             "cannot replay: cache was cleared because too much RAM was required");
@@ -154,14 +135,7 @@ public abstract class CachingCollector extends FilterCollector {
 
     @Override
     protected NoScoreCachingLeafCollector wrap(LeafCollector in, int maxDocsToCache) {
-      return new ScoreCachingLeafCollector(in, maxDocsToCache);
-    }
-
-    @Override
-    protected void postCollect(NoScoreCachingLeafCollector collector) {
-      final ScoreCachingLeafCollector coll = (ScoreCachingLeafCollector) collector;
-      super.postCollect(coll);
-      scores.add(coll.cachedScores());
+      return new ScoreCachingLeafCollector(in, maxDocsToCache, this);
     }
 
     /**
@@ -191,12 +165,15 @@ public abstract class CachingCollector extends FilterCollector {
   private class NoScoreCachingLeafCollector extends FilterLeafCollector {
 
     final int maxDocsToCache;
+    final NoScoreCachingCollector collector;
     int[] docs;
     int docCount;
 
-    NoScoreCachingLeafCollector(LeafCollector in, int maxDocsToCache) {
+    NoScoreCachingLeafCollector(
+        LeafCollector in, int maxDocsToCache, NoScoreCachingCollector collector) {
       super(in);
       this.maxDocsToCache = maxDocsToCache;
+      this.collector = collector;
       docs = new int[Math.min(maxDocsToCache, INITIAL_ARRAY_SIZE)];
       docCount = 0;
     }
@@ -235,6 +212,21 @@ public abstract class CachingCollector extends FilterCollector {
       super.collect(doc);
     }
 
+    protected void postCollect() {
+      final int[] docs = cachedDocs();
+      collector.maxDocsToCache -= docs.length;
+      collector.docs.add(docs);
+    }
+
+    @Override
+    public void finish() {
+      if (!hasCache()) {
+        collector.invalidate();
+      } else {
+        postCollect();
+      }
+    }
+
     boolean hasCache() {
       return docs != null;
     }
@@ -249,8 +241,9 @@ public abstract class CachingCollector extends FilterCollector {
     Scorable scorer;
     float[] scores;
 
-    ScoreCachingLeafCollector(LeafCollector in, int maxDocsToCache) {
-      super(in, maxDocsToCache);
+    ScoreCachingLeafCollector(
+        LeafCollector in, int maxDocsToCache, ScoreCachingCollector collector) {
+      super(in, maxDocsToCache, collector);
       scores = new float[docs.length];
     }
 
@@ -281,6 +274,12 @@ public abstract class CachingCollector extends FilterCollector {
     float[] cachedScores() {
       return docs == null ? null : ArrayUtil.copyOfSubArray(scores, 0, docCount);
     }
+
+    @Override
+    protected void postCollect() {
+      super.postCollect();
+      ((ScoreCachingCollector) collector).scores.add(cachedScores());
+    }
   }
 
   /**
diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java b/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java
index 24733668ff6..d9bc671fd42 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java
@@ -42,6 +42,11 @@ public abstract class FilterLeafCollector implements LeafCollector {
     in.collect(doc);
   }
 
+  @Override
+  public void finish() throws IOException {
+    in.finish();
+  }
+
   @Override
   public String toString() {
     String name = getClass().getSimpleName();
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index 9c0a29c052f..b76bf3161d0 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -782,6 +782,9 @@ public class IndexSearcher {
           partialResult = true;
         }
       }
+      // Note: this is called if collection ran successfully, including the above special cases of
+      // CollectionTerminatedException and TimeExceededException, but no other exception.
+      leafCollector.finish();
     }
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java b/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
index a42d531c2b2..334afc798ca 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
@@ -95,4 +95,13 @@ public interface LeafCollector {
   default DocIdSetIterator competitiveIterator() throws IOException {
     return null;
   }
+
+  /**
+   * Hook that gets called once the leaf that is associated with this collector has finished
+   * collecting successfully, including when a {@link CollectionTerminatedException} is thrown. This
+   * is typically useful to compile data that has been collected on this leaf, e.g. to convert facet
+   * counts on leaf ordinals to facet counts on global ordinals. The default implementation does
+   * nothing.
+   */
+  default void finish() throws IOException {}
 }
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java b/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
index 7f8e8121734..ff6a6a97ba5 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
@@ -223,6 +223,7 @@ public class MultiCollector implements Collector {
           } catch (
               @SuppressWarnings("unused")
               CollectionTerminatedException e) {
+            collectors[i].finish();
             collectors[i] = null;
             if (allCollectorsTerminated()) {
               throw new CollectionTerminatedException();
@@ -232,6 +233,15 @@ public class MultiCollector implements Collector {
       }
     }
 
+    @Override
+    public void finish() throws IOException {
+      for (LeafCollector collector : collectors) {
+        if (collector != null) {
+          collector.finish();
+        }
+      }
+    }
+
     private boolean allCollectorsTerminated() {
       for (int i = 0; i < collectors.length; i++) {
         if (collectors[i] != null) {
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java
index 4b07e06dd0a..425c33a34e7 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java
@@ -57,6 +57,7 @@ public class TestCachingCollector extends LuceneTestCase {
       for (int i = 0; i < 1000; i++) {
         acc.collect(i);
       }
+      acc.finish();
 
       // now replay them
       cc.replay(
@@ -127,6 +128,7 @@ public class TestCachingCollector extends LuceneTestCase {
       acc.collect(0);
 
       assertTrue(cc.isCached());
+      acc.finish();
       cc.replay(new NoOpCollector());
     }
   }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
index c86432b30f7..ad82594f41d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
@@ -18,6 +18,7 @@ package org.apache.lucene.facet;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -198,6 +199,7 @@ class DrillSidewaysScorer extends BulkScorer {
 
       docID = baseApproximation.nextDoc();
     }
+    finish(collector, Collections.singleton(dim));
   }
 
   /**
@@ -334,6 +336,8 @@ class DrillSidewaysScorer extends BulkScorer {
 
       docID = baseApproximation.nextDoc();
     }
+
+    finish(collector, sidewaysDims);
   }
 
   private static int advanceIfBehind(int docID, DocIdSetIterator iterator) throws IOException {
@@ -552,6 +556,7 @@ class DrillSidewaysScorer extends BulkScorer {
 
       nextChunkStart += CHUNK;
     }
+    finish(collector, Arrays.asList(dims));
   }
 
   private void doUnionScoring(Bits acceptDocs, LeafCollector collector, DocsAndCost[] dims)
@@ -706,6 +711,8 @@ class DrillSidewaysScorer extends BulkScorer {
 
       nextChunkStart += CHUNK;
     }
+
+    finish(collector, Arrays.asList(dims));
   }
 
   private void collectHit(LeafCollector collector, DocsAndCost[] dims) throws IOException {
@@ -757,6 +764,16 @@ class DrillSidewaysScorer extends BulkScorer {
     sidewaysCollector.collect(collectDocID);
   }
 
+  private void finish(LeafCollector collector, Collection<DocsAndCost> dims) throws IOException {
+    collector.finish();
+    if (drillDownLeafCollector != null) {
+      drillDownLeafCollector.finish();
+    }
+    for (DocsAndCost dim : dims) {
+      dim.sidewaysLeafCollector.finish();
+    }
+  }
+
   private void setScorer(LeafCollector mainCollector, Scorable scorer) throws IOException {
     mainCollector.setScorer(scorer);
     if (drillDownLeafCollector != null) {
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
index cf49aef39ee..2bce78e22b3 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
@@ -103,13 +103,6 @@ public class FacetsCollector extends SimpleCollector {
 
   /** Returns the documents matched by the query, one {@link MatchingDocs} per visited segment. */
   public List<MatchingDocs> getMatchingDocs() {
-    if (docsBuilder != null) {
-      matchingDocs.add(new MatchingDocs(this.context, docsBuilder.build(), totalHits, scores));
-      docsBuilder = null;
-      scores = null;
-      context = null;
-    }
-
     return matchingDocs;
   }
 
@@ -139,9 +132,7 @@ public class FacetsCollector extends SimpleCollector {
 
   @Override
   protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    if (docsBuilder != null) {
-      matchingDocs.add(new MatchingDocs(this.context, docsBuilder.build(), totalHits, scores));
-    }
+    assert docsBuilder == null;
     docsBuilder = new DocIdSetBuilder(context.reader().maxDoc());
     totalHits = 0;
     if (keepScores) {
@@ -150,6 +141,14 @@ public class FacetsCollector extends SimpleCollector {
     this.context = context;
   }
 
+  @Override
+  public void finish() throws IOException {
+    matchingDocs.add(new MatchingDocs(this.context, docsBuilder.build(), totalHits, scores));
+    docsBuilder = null;
+    scores = null;
+    context = null;
+  }
+
   /** Utility method, to search and also collect all hits into the provided {@link Collector}. */
   public static TopDocs search(IndexSearcher searcher, Query q, int n, Collector fc)
       throws IOException {
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
index 9ead686831e..26c3c915dd3 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
@@ -270,9 +270,6 @@ public class BlockGroupingCollector extends SimpleCollector {
     // if (queueFull) {
     // System.out.println("getTopGroups groupOffset=" + groupOffset + " topNGroups=" + topNGroups);
     // }
-    if (subDocUpto != 0) {
-      processGroup();
-    }
     if (groupOffset >= groupQueue.size()) {
       return null;
     }
@@ -472,9 +469,6 @@ public class BlockGroupingCollector extends SimpleCollector {
 
   @Override
   protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    if (subDocUpto != 0) {
-      processGroup();
-    }
     subDocUpto = 0;
     docBase = readerContext.docBase;
     // System.out.println("setNextReader base=" + docBase + " r=" + readerContext.reader);
@@ -492,6 +486,13 @@ public class BlockGroupingCollector extends SimpleCollector {
     }
   }
 
+  @Override
+  public void finish() throws IOException {
+    if (subDocUpto != 0) {
+      processGroup();
+    }
+  }
+
   @Override
   public ScoreMode scoreMode() {
     return needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
index 74a957d809d..4e56a12c902 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
@@ -67,11 +67,6 @@ public abstract class GroupFacetCollector extends SimpleCollector {
    */
   public GroupedFacetResult mergeSegmentResults(int size, int minCount, boolean orderByCount)
       throws IOException {
-    if (segmentFacetCounts != null) {
-      segmentResults.add(createSegmentResult());
-      segmentFacetCounts = null; // reset
-    }
-
     int totalCount = 0;
     int missingCount = 0;
     SegmentResultPriorityQueue segments = new SegmentResultPriorityQueue(segmentResults.size());
@@ -109,6 +104,12 @@ public abstract class GroupFacetCollector extends SimpleCollector {
     return facetResult;
   }
 
+  @Override
+  public void finish() throws IOException {
+    segmentResults.add(createSegmentResult());
+    segmentFacetCounts = null;
+  }
+
   protected abstract SegmentResult createSegmentResult() throws IOException;
 
   @Override
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
index c1b49758b13..e49e517faa8 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
@@ -141,10 +141,6 @@ public abstract class TermGroupFacetCollector extends GroupFacetCollector {
 
     @Override
     protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      if (segmentFacetCounts != null) {
-        segmentResults.add(createSegmentResult());
-      }
-
       groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
       facetFieldTermsIndex = DocValues.getSorted(context.reader(), facetField);
 
@@ -321,10 +317,6 @@ public abstract class TermGroupFacetCollector extends GroupFacetCollector {
 
     @Override
     protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      if (segmentFacetCounts != null) {
-        segmentResults.add(createSegmentResult());
-      }
-
       groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
       facetFieldDocTermOrds = DocValues.getSortedSet(context.reader(), facetField);
       facetFieldNumTerms = (int) facetFieldDocTermOrds.getValueCount();
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
index e46e73bb1aa..0c88359029b 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
@@ -22,6 +22,7 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BulkScorer;
 import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Weight;
 
 /**
@@ -67,14 +68,16 @@ public class SuggestIndexSearcher extends IndexSearcher {
     for (LeafReaderContext context : getIndexReader().leaves()) {
       BulkScorer scorer = weight.bulkScorer(context);
       if (scorer != null) {
+        LeafCollector leafCollector = collector.getLeafCollector(context);
         try {
-          scorer.score(collector.getLeafCollector(context), context.reader().getLiveDocs());
+          scorer.score(leafCollector, context.reader().getLiveDocs());
         } catch (
             @SuppressWarnings("unused")
             CollectionTerminatedException e) {
           // collection was terminated prematurely
           // continue with the following leaf
         }
+        leafCollector.finish();
       }
     }
   }
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
index e8f66c87b3b..3cfc8e2582d 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
@@ -100,12 +100,19 @@ public class TopSuggestDocsCollector extends SimpleCollector {
   @Override
   protected void doSetNextReader(LeafReaderContext context) throws IOException {
     docBase = context.docBase;
+  }
+
+  @Override
+  public void finish() throws IOException {
     if (seenSurfaceForms != null) {
-      seenSurfaceForms.clear();
       // NOTE: this also clears the priorityQueue:
       for (SuggestScoreDoc hit : priorityQueue.getResults()) {
         pendingResults.add(hit);
       }
+
+      // Deduplicate all hits: we already dedup'd efficiently within each segment by
+      // truncating the FST top paths search, but across segments there may still be dups:
+      seenSurfaceForms.clear();
     }
   }
 
@@ -136,15 +143,7 @@ public class TopSuggestDocsCollector extends SimpleCollector {
     SuggestScoreDoc[] suggestScoreDocs;
 
     if (seenSurfaceForms != null) {
-      // NOTE: this also clears the priorityQueue:
-      for (SuggestScoreDoc hit : priorityQueue.getResults()) {
-        pendingResults.add(hit);
-      }
-
-      // Deduplicate all hits: we already dedup'd efficiently within each segment by
-      // truncating the FST top paths search, but across segments there may still be dups:
-      seenSurfaceForms.clear();
-
+      assert seenSurfaceForms.isEmpty();
       // TODO: we could use a priority queue here to make cost O(N * log(num)) instead of O(N *
       // log(N)), where N = O(num *
       // numSegments), but typically numSegments is smallish and num is smallish so this won't
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingCollector.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingCollector.java
index cf2c2732614..af0df8cdc3f 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingCollector.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingCollector.java
@@ -30,11 +30,12 @@ class AssertingCollector extends FilterCollector {
   private boolean weightSet = false;
   private int maxDoc = -1;
   private int previousLeafMaxDoc = 0;
+  boolean hasFinishedCollectingPreviousLeaf = true;
 
   /** Wrap the given collector in order to add assertions. */
-  public static Collector wrap(Collector in) {
+  public static AssertingCollector wrap(Collector in) {
     if (in instanceof AssertingCollector) {
-      return in;
+      return (AssertingCollector) in;
     }
     return new AssertingCollector(in);
   }
@@ -49,7 +50,9 @@ class AssertingCollector extends FilterCollector {
     assert context.docBase >= previousLeafMaxDoc;
     previousLeafMaxDoc = context.docBase + context.reader().maxDoc();
 
+    assert hasFinishedCollectingPreviousLeaf;
     final LeafCollector in = super.getLeafCollector(context);
+    hasFinishedCollectingPreviousLeaf = false;
     final int docBase = context.docBase;
     return new AssertingLeafCollector(in, 0, DocIdSetIterator.NO_MORE_DOCS) {
       @Override
@@ -66,6 +69,12 @@ class AssertingCollector extends FilterCollector {
         super.collect(doc);
         maxDoc = docBase + doc;
       }
+
+      @Override
+      public void finish() throws IOException {
+        hasFinishedCollectingPreviousLeaf = true;
+        super.finish();
+      }
     };
   }
 
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingIndexSearcher.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingIndexSearcher.java
index 83abd788309..f5fa29b1494 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingIndexSearcher.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingIndexSearcher.java
@@ -75,7 +75,9 @@ public class AssertingIndexSearcher extends IndexSearcher {
   protected void search(List<LeafReaderContext> leaves, Weight weight, Collector collector)
       throws IOException {
     assert weight instanceof AssertingWeight;
-    super.search(leaves, weight, AssertingCollector.wrap(collector));
+    AssertingCollector assertingCollector = AssertingCollector.wrap(collector);
+    super.search(leaves, weight, assertingCollector);
+    assert assertingCollector.hasFinishedCollectingPreviousLeaf;
   }
 
   @Override
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingLeafCollector.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingLeafCollector.java
index 5c7801e5122..bcf3c4b1098 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingLeafCollector.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingLeafCollector.java
@@ -30,6 +30,7 @@ class AssertingLeafCollector extends FilterLeafCollector {
 
   private Scorable scorer;
   private int lastCollected = -1;
+  private boolean finishCalled;
 
   AssertingLeafCollector(LeafCollector collector, int min, int max) {
     super(collector);
@@ -57,4 +58,11 @@ class AssertingLeafCollector extends FilterLeafCollector {
   public DocIdSetIterator competitiveIterator() throws IOException {
     return in.competitiveIterator();
   }
+
+  @Override
+  public void finish() throws IOException {
+    assert finishCalled == false;
+    finishCalled = true;
+    super.finish();
+  }
 }


[lucene] 02/04: Add a thread safe CachingLeafSlicesSupplier to compute and cache the LeafSlices used with concurrent segment (#12374)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 856647dde6f4d2ac4aae412b4877809d8437c6cd
Author: Sorabh <so...@apache.org>
AuthorDate: Fri Jun 30 05:57:34 2023 -0700

    Add a thread safe CachingLeafSlicesSupplier to compute and cache the LeafSlices used with concurrent segment (#12374)
    
    search. It uses the protected method `slices` by default to compute the slices which can be
    overriden by the sub classes of IndexSearcher
---
 lucene/CHANGES.txt                                 |  2 +-
 .../org/apache/lucene/search/IndexSearcher.java    | 69 ++++++++++++++++++----
 2 files changed, 60 insertions(+), 11 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index e1fccc12de3..09e7b4dfe3f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -15,7 +15,7 @@ New Features
 
 Improvements
 ---------------------
-(No changes)
+* GITHUB#12374: Add CachingLeafSlicesSupplier to compute the LeafSlices for concurrent segment search (Sorabh Hamirwasia)
 
 Optimizations
 ---------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index 529f8326ccb..9c0a29c052f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -29,6 +29,7 @@ import java.util.concurrent.Executor;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.RunnableFuture;
 import java.util.concurrent.ThreadPoolExecutor;
+import java.util.function.Function;
 import java.util.function.Supplier;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
@@ -117,8 +118,13 @@ public class IndexSearcher {
   protected final IndexReaderContext readerContext;
   protected final List<LeafReaderContext> leafContexts;
 
-  /** used with executor - each slice holds a set of leafs executed within one thread */
-  private final LeafSlice[] leafSlices;
+  /**
+   * used with executor - LeafSlice supplier where each slice holds a set of leafs executed within
+   * one thread. We are caching it instead of creating it eagerly to avoid calling a protected
+   * method from constructor, which is a bad practice. This is {@code null} if no executor is
+   * provided
+   */
+  private final CachingLeafSlicesSupplier leafSlicesSupplier;
 
   // These are only used for multi-threaded search
   private final Executor executor;
@@ -237,7 +243,8 @@ public class IndexSearcher {
     this.taskExecutor = taskExecutor;
     this.readerContext = context;
     leafContexts = context.leaves();
-    this.leafSlices = executor == null ? null : slices(leafContexts);
+    leafSlicesSupplier =
+        (executor == null) ? null : new CachingLeafSlicesSupplier(this::slices, leafContexts);
   }
 
   /**
@@ -465,7 +472,7 @@ public class IndexSearcher {
    * @lucene.experimental
    */
   public LeafSlice[] getSlices() {
-    return leafSlices;
+    return (executor == null) ? null : leafSlicesSupplier.get();
   }
 
   /**
@@ -490,16 +497,17 @@ public class IndexSearcher {
 
     final int cappedNumHits = Math.min(numHits, limit);
 
+    final LeafSlice[] leafSlices = getSlices();
     final CollectorManager<TopScoreDocCollector, TopDocs> manager =
         new CollectorManager<TopScoreDocCollector, TopDocs>() {
 
           private final HitsThresholdChecker hitsThresholdChecker =
-              (executor == null || leafSlices.length <= 1)
+              (leafSlices == null || leafSlices.length <= 1)
                   ? HitsThresholdChecker.create(Math.max(TOTAL_HITS_THRESHOLD, numHits))
                   : HitsThresholdChecker.createShared(Math.max(TOTAL_HITS_THRESHOLD, numHits));
 
           private final MaxScoreAccumulator minScoreAcc =
-              (executor == null || leafSlices.length <= 1) ? null : new MaxScoreAccumulator();
+              (leafSlices == null || leafSlices.length <= 1) ? null : new MaxScoreAccumulator();
 
           @Override
           public TopScoreDocCollector newCollector() throws IOException {
@@ -633,17 +641,18 @@ public class IndexSearcher {
     }
     final int cappedNumHits = Math.min(numHits, limit);
     final Sort rewrittenSort = sort.rewrite(this);
+    final LeafSlice[] leafSlices = getSlices();
 
     final CollectorManager<TopFieldCollector, TopFieldDocs> manager =
         new CollectorManager<>() {
 
           private final HitsThresholdChecker hitsThresholdChecker =
-              (executor == null || leafSlices.length <= 1)
+              (leafSlices == null || leafSlices.length <= 1)
                   ? HitsThresholdChecker.create(Math.max(TOTAL_HITS_THRESHOLD, numHits))
                   : HitsThresholdChecker.createShared(Math.max(TOTAL_HITS_THRESHOLD, numHits));
 
           private final MaxScoreAccumulator minScoreAcc =
-              (executor == null || leafSlices.length <= 1) ? null : new MaxScoreAccumulator();
+              (leafSlices == null || leafSlices.length <= 1) ? null : new MaxScoreAccumulator();
 
           @Override
           public TopFieldCollector newCollector() throws IOException {
@@ -673,7 +682,7 @@ public class IndexSearcher {
   /**
    * Lower-level search API. Search all leaves using the given {@link CollectorManager}. In contrast
    * to {@link #search(Query, Collector)}, this method will use the searcher's {@link Executor} in
-   * order to parallelize execution of the collection on the configured {@link #leafSlices}.
+   * order to parallelize execution of the collection on the configured {@link #getSlices()}.
    *
    * @see CollectorManager
    * @lucene.experimental
@@ -688,7 +697,8 @@ public class IndexSearcher {
 
   private <C extends Collector, T> T search(
       Weight weight, CollectorManager<C, T> collectorManager, C firstCollector) throws IOException {
-    if (executor == null || leafSlices.length <= 1) {
+    final LeafSlice[] leafSlices = getSlices();
+    if (leafSlices == null || leafSlices.length <= 1) {
       search(leafContexts, weight, firstCollector);
       return collectorManager.reduce(Collections.singletonList(firstCollector));
     } else {
@@ -1037,4 +1047,43 @@ public class IndexSearcher {
 
     return new TaskExecutor(executor);
   }
+
+  /**
+   * Supplier for {@link LeafSlice} slices which computes and caches the value on first invocation
+   * and returns cached value on subsequent invocation. If the passed in provider for slice
+   * computation throws exception then same will be passed to the caller of this supplier on each
+   * invocation. If the provider returns null then {@link NullPointerException} will be thrown to
+   * the caller.
+   *
+   * <p>NOTE: To provide thread safe caching mechanism this class is implementing the (subtle) <a
+   * href="https://shipilev.net/blog/2014/safe-public-construction/">double-checked locking
+   * idiom</a>
+   */
+  private static class CachingLeafSlicesSupplier implements Supplier<LeafSlice[]> {
+    private volatile LeafSlice[] leafSlices;
+
+    private final Function<List<LeafReaderContext>, LeafSlice[]> sliceProvider;
+
+    private final List<LeafReaderContext> leaves;
+
+    private CachingLeafSlicesSupplier(
+        Function<List<LeafReaderContext>, LeafSlice[]> provider, List<LeafReaderContext> leaves) {
+      this.sliceProvider = Objects.requireNonNull(provider, "leaf slice provider cannot be null");
+      this.leaves = Objects.requireNonNull(leaves, "list of LeafReaderContext cannot be null");
+    }
+
+    @Override
+    public LeafSlice[] get() {
+      if (leafSlices == null) {
+        synchronized (this) {
+          if (leafSlices == null) {
+            leafSlices =
+                Objects.requireNonNull(
+                    sliceProvider.apply(leaves), "slices computed by the provider is null");
+          }
+        }
+      }
+      return leafSlices;
+    }
+  }
 }


[lucene] 03/04: Assign a dummy simScorer in TermsWeight if score is not needed (#12383)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit aa0474750584c3949ac9d2a59d8ce383fd26c723
Author: Sagar <99...@users.noreply.github.com>
AuthorDate: Fri Jun 30 06:14:33 2023 -0700

    Assign a dummy simScorer in TermsWeight if score is not needed (#12383)
---
 lucene/CHANGES.txt                                 |  2 +
 .../java/org/apache/lucene/search/TermQuery.java   | 17 +++++++-
 .../org/apache/lucene/search/TestTermQuery.java    | 51 ++++++++++++++++++++++
 3 files changed, 69 insertions(+), 1 deletion(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 09e7b4dfe3f..ef5c9e496a1 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -25,6 +25,8 @@ Optimizations
 * GITHUB#12361: Faster top-level disjunctions sorted by descending score.
   (Adrien Grand)
 
+* GITHUB#12383: Assign a dummy simScorer in TermsWeight if score is not needed. (Sagar Upadhyaya)
+
 * GITHUB#12372: Reduce allocation during HNSW construction (Jonathan Ellis)
 
 * GITHUB#12385: Restore parallel knn query rewrite across segments rather than slices (Luca Cavanna)
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
index f08855abe5c..c7d5ac325b9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
@@ -72,7 +72,22 @@ public class TermQuery extends Query {
       if (termStats == null) {
         this.simScorer = null; // term doesn't exist in any segment, we won't use similarity at all
       } else {
-        this.simScorer = similarity.scorer(boost, collectionStats, termStats);
+        // Assigning a dummy simScorer in case score is not needed to avoid unnecessary float[]
+        // allocations in case default BM25Scorer is used.
+        // See: https://github.com/apache/lucene/issues/12297
+        if (scoreMode.needsScores()) {
+          this.simScorer = similarity.scorer(boost, collectionStats, termStats);
+        } else {
+          // Assigning a dummy scorer as this is not expected to be called since scores are not
+          // needed.
+          this.simScorer =
+              new Similarity.SimScorer() {
+                @Override
+                public float score(float freq, long norm) {
+                  return 0f;
+                }
+              };
+        }
       }
     }
 
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java
index 9a6f39de726..473f6c9a8f0 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java
@@ -17,11 +17,14 @@
 package org.apache.lucene.search;
 
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.CompositeReaderContext;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.FilterDirectoryReader;
 import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.LeafReader;
@@ -32,6 +35,7 @@ import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermStates;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.tests.index.RandomIndexWriter;
 import org.apache.lucene.tests.search.DummyTotalHitCountCollector;
@@ -164,6 +168,53 @@ public class TestTermQuery extends LuceneTestCase {
     IOUtils.close(reader, w, dir);
   }
 
+  public void testWithWithDifferentScoreModes() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w =
+        new RandomIndexWriter(
+            random(), dir, newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE));
+    // segment that contains the term
+    Document doc = new Document();
+    doc.add(new StringField("foo", "bar", Store.NO));
+    w.addDocument(doc);
+    w.getReader().close();
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = new IndexSearcher(reader);
+    Similarity existingSimilarity = searcher.getSimilarity();
+
+    for (ScoreMode scoreMode : ScoreMode.values()) {
+      final AtomicReference<ScoreMode> scoreModeInWeight = new AtomicReference<ScoreMode>();
+      final AtomicBoolean scorerCalled = new AtomicBoolean();
+      searcher.setSimilarity(
+          new Similarity() { // Wrapping existing similarity for testing
+            @Override
+            public long computeNorm(FieldInvertState state) {
+              return existingSimilarity.computeNorm(state);
+            }
+
+            @Override
+            public SimScorer scorer(
+                float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+              scorerCalled.set(true);
+              return existingSimilarity.scorer(boost, collectionStats, termStats);
+            }
+          });
+      TermQuery termQuery =
+          new TermQuery(new Term("foo", "bar")) {
+            @Override
+            public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
+                throws IOException {
+              scoreModeInWeight.set(scoreMode);
+              return super.createWeight(searcher, scoreMode, boost);
+            }
+          };
+      termQuery.createWeight(searcher, scoreMode, 1f);
+      assertEquals(scoreMode, scoreModeInWeight.get());
+      assertEquals(scoreMode.needsScores(), scorerCalled.get());
+    }
+    IOUtils.close(reader, w, dir);
+  }
+
   private static class NoSeekDirectoryReader extends FilterDirectoryReader {
 
     public NoSeekDirectoryReader(DirectoryReader in) throws IOException {