mirror of https://github.com/apache/lucene.git
LUCENE-8242: Deprecate createNormalizedWeight
This commit is contained in:
parent
f83a8da05e
commit
798d351034
|
@ -95,6 +95,12 @@ Optimizations
|
|||
|
||||
======================= Lucene 7.4.0 =======================
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-8242: IndexSearcher.createNormalizedWeight() has been deprecated.
|
||||
Instead use IndexSearcher.createWeight(), rewriting the query first.
|
||||
(Alan Woodward)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-8200: Allow doc-values to be updated atomically together
|
||||
|
|
|
@ -685,7 +685,8 @@ class FrozenBufferedUpdates {
|
|||
}
|
||||
final IndexSearcher searcher = new IndexSearcher(readerContext.reader());
|
||||
searcher.setQueryCache(null);
|
||||
final Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
query = searcher.rewrite(query);
|
||||
final Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
final Scorer scorer = weight.scorer(readerContext);
|
||||
if (scorer != null) {
|
||||
final DocIdSetIterator it = scorer.iterator();
|
||||
|
|
|
@ -414,7 +414,8 @@ public class IndexSearcher {
|
|||
*/
|
||||
public void search(Query query, Collector results)
|
||||
throws IOException {
|
||||
search(leafContexts, createNormalizedWeight(query, results.scoreMode()), results);
|
||||
query = rewrite(query);
|
||||
search(leafContexts, createWeight(query, results.scoreMode(), 1), results);
|
||||
}
|
||||
|
||||
/** Search implementation with arbitrary sorting, plus
|
||||
|
@ -553,8 +554,8 @@ public class IndexSearcher {
|
|||
// no segments
|
||||
scoreMode = ScoreMode.COMPLETE;
|
||||
}
|
||||
|
||||
final Weight weight = createNormalizedWeight(query, scoreMode);
|
||||
query = rewrite(query);
|
||||
final Weight weight = createWeight(query, scoreMode, 1);
|
||||
final List<Future<C>> topDocsFutures = new ArrayList<>(leafSlices.length);
|
||||
for (int i = 0; i < leafSlices.length; ++i) {
|
||||
final LeafReaderContext[] leaves = leafSlices[i].leaves;
|
||||
|
@ -651,7 +652,8 @@ public class IndexSearcher {
|
|||
* entire index.
|
||||
*/
|
||||
public Explanation explain(Query query, int doc) throws IOException {
|
||||
return explain(createNormalizedWeight(query, ScoreMode.COMPLETE), doc);
|
||||
query = rewrite(query);
|
||||
return explain(createWeight(query, ScoreMode.COMPLETE, 1), doc);
|
||||
}
|
||||
|
||||
/** Expert: low-level implementation method
|
||||
|
@ -683,7 +685,11 @@ public class IndexSearcher {
|
|||
* afterwards the {@link Weight} is normalized. The returned {@code Weight}
|
||||
* can then directly be used to get a {@link Scorer}.
|
||||
* @lucene.internal
|
||||
*
|
||||
* @deprecated Clients should rewrite the query and then call {@link #createWeight(Query, ScoreMode, float)}
|
||||
* with a boost value of 1f
|
||||
*/
|
||||
@Deprecated
|
||||
public Weight createNormalizedWeight(Query query, ScoreMode scoreMode) throws IOException {
|
||||
query = rewrite(query);
|
||||
return createWeight(query, scoreMode, 1f);
|
||||
|
|
|
@ -60,7 +60,8 @@ public abstract class QueryRescorer extends Rescorer {
|
|||
|
||||
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
Query rewritten = searcher.rewrite(query);
|
||||
Weight weight = searcher.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
|
||||
// Now merge sort docIDs from hits, with reader's leaves:
|
||||
int hitUpto = 0;
|
||||
|
|
|
@ -453,8 +453,8 @@
|
|||
* <p>Assuming we are not sorting (since sorting doesn't affect the raw Lucene score),
|
||||
* we call one of the search methods of the IndexSearcher, passing in the
|
||||
* {@link org.apache.lucene.search.Weight Weight} object created by
|
||||
* {@link org.apache.lucene.search.IndexSearcher#createNormalizedWeight(org.apache.lucene.search.Query,ScoreMode)
|
||||
* IndexSearcher.createNormalizedWeight(Query,boolean)} and the number of results we want.
|
||||
* {@link org.apache.lucene.search.IndexSearcher#createWeight(org.apache.lucene.search.Query,ScoreMode,float)
|
||||
* IndexSearcher.createWeight(Query,ScoreMode,float)} and the number of results we want.
|
||||
* This method returns a {@link org.apache.lucene.search.TopDocs TopDocs} object,
|
||||
* which is an internal collection of search results. The IndexSearcher creates
|
||||
* a {@link org.apache.lucene.search.TopScoreDocCollector TopScoreDocCollector} and
|
||||
|
|
|
@ -186,7 +186,7 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
bq.add(new TermQuery(new Term("field", "a")), BooleanClause.Occur.SHOULD);
|
||||
bq.add(new TermQuery(new Term("field", "a")), BooleanClause.Occur.SHOULD);
|
||||
|
||||
Weight w = s.createNormalizedWeight(bq.build(), ScoreMode.COMPLETE);
|
||||
Weight w = s.createWeight(s.rewrite(bq.build()), ScoreMode.COMPLETE, 1);
|
||||
|
||||
assertEquals(1, s.getIndexReader().leaves().size());
|
||||
BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0));
|
||||
|
|
|
@ -313,7 +313,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(new BooleanClause(new TermQuery(new Term("field", term)), BooleanClause.Occur.SHOULD));
|
||||
}
|
||||
|
||||
Weight weight = s.createNormalizedWeight(q.build(), ScoreMode.COMPLETE);
|
||||
Weight weight = s.createWeight(s.rewrite(q.build()), ScoreMode.COMPLETE, 1);
|
||||
|
||||
Scorer scorer = weight.scorer(s.leafContexts.get(0));
|
||||
|
||||
|
@ -331,7 +331,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
// verify exact match:
|
||||
for(int iter2=0;iter2<10;iter2++) {
|
||||
|
||||
weight = s.createNormalizedWeight(q.build(), ScoreMode.COMPLETE);
|
||||
weight = s.createWeight(s.rewrite(q.build()), ScoreMode.COMPLETE, 1);
|
||||
scorer = weight.scorer(s.leafContexts.get(0));
|
||||
|
||||
if (VERBOSE) {
|
||||
|
@ -617,7 +617,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(pq, Occur.MUST);
|
||||
q.add(new TermQuery(new Term("field", "c")), Occur.FILTER);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q.build(), ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(q.build()), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertTrue(scorer instanceof ConjunctionScorer);
|
||||
assertNotNull(scorer.twoPhaseIterator());
|
||||
|
@ -646,7 +646,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(pq, Occur.SHOULD);
|
||||
q.add(new TermQuery(new Term("field", "c")), Occur.SHOULD);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q.build(), ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(q.build()), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||
assertTrue(scorer instanceof DisjunctionScorer);
|
||||
assertNotNull(scorer.twoPhaseIterator());
|
||||
|
@ -677,7 +677,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(pq, Occur.SHOULD);
|
||||
q.add(new TermQuery(new Term("field", "d")), Occur.SHOULD);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q.build(), ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(q.build()), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertTrue(scorer instanceof ExactPhraseScorer);
|
||||
assertNotNull(scorer.twoPhaseIterator());
|
||||
|
@ -706,7 +706,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(pq, Occur.SHOULD);
|
||||
q.add(new TermQuery(new Term("field", "c")), Occur.MUST_NOT);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q.build(), ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(q.build()), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||
assertTrue(scorer instanceof ReqExclScorer);
|
||||
assertNotNull(scorer.twoPhaseIterator());
|
||||
|
@ -735,7 +735,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(pq, Occur.MUST);
|
||||
q.add(new TermQuery(new Term("field", "c")), Occur.SHOULD);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q.build(), ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(q.build()), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||
assertTrue(scorer instanceof ReqOptSumScorer);
|
||||
assertNotNull(scorer.twoPhaseIterator());
|
||||
|
@ -768,11 +768,11 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
BooleanQuery bq = bqBuilder.build();
|
||||
|
||||
Set<Term> scoringTerms = new HashSet<>();
|
||||
searcher.createNormalizedWeight(bq, ScoreMode.COMPLETE).extractTerms(scoringTerms);
|
||||
searcher.createWeight(searcher.rewrite(bq), ScoreMode.COMPLETE, 1).extractTerms(scoringTerms);
|
||||
assertEquals(new HashSet<>(Arrays.asList(a, b)), scoringTerms);
|
||||
|
||||
Set<Term> matchingTerms = new HashSet<>();
|
||||
searcher.createNormalizedWeight(bq, ScoreMode.COMPLETE_NO_SCORES).extractTerms(matchingTerms);
|
||||
searcher.createWeight(searcher.rewrite(bq), ScoreMode.COMPLETE_NO_SCORES, 1).extractTerms(matchingTerms);
|
||||
assertEquals(new HashSet<>(Arrays.asList(a, b, c)), matchingTerms);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -196,7 +196,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
|||
bq1.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
|
||||
bq1.add(new PhraseQuery(F2, "search", "engine"), Occur.SHOULD);
|
||||
|
||||
Weight w1 = scorerSearcher.createNormalizedWeight(bq1.build(), ScoreMode.COMPLETE);
|
||||
Weight w1 = scorerSearcher.createWeight(scorerSearcher.rewrite(bq1.build()), ScoreMode.COMPLETE, 1);
|
||||
Scorer s1 = w1.scorer(reader.leaves().get(0));
|
||||
assertEquals(0, s1.iterator().nextDoc());
|
||||
assertEquals(2, s1.getChildren().size());
|
||||
|
@ -205,7 +205,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
|||
bq2.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
|
||||
bq2.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
|
||||
|
||||
Weight w2 = scorerSearcher.createNormalizedWeight(bq2.build(), ScoreMode.COMPLETE);
|
||||
Weight w2 = scorerSearcher.createWeight(scorerSearcher.rewrite(bq2.build()), ScoreMode.COMPLETE, 1);
|
||||
Scorer s2 = w2.scorer(reader.leaves().get(0));
|
||||
assertEquals(0, s2.iterator().nextDoc());
|
||||
assertEquals(1, s2.getChildren().size());
|
||||
|
@ -218,7 +218,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
|||
bq.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
|
||||
bq.setMinimumNumberShouldMatch(2);
|
||||
|
||||
Weight w = scorerSearcher.createNormalizedWeight(bq.build(), ScoreMode.COMPLETE);
|
||||
Weight w = scorerSearcher.createWeight(scorerSearcher.rewrite(bq.build()), ScoreMode.COMPLETE, 1);
|
||||
Scorer s = w.scorer(reader.leaves().get(0));
|
||||
assertEquals(0, s.iterator().nextDoc());
|
||||
assertEquals(2, s.getChildren().size());
|
||||
|
|
|
@ -93,7 +93,7 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
|||
BooleanQuery.Builder query2 = new BooleanQuery.Builder();
|
||||
query2.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
||||
query2.add(new TermQuery(new Term("field", "b")), Occur.SHOULD);
|
||||
final Weight weight = searcher.createNormalizedWeight(query2.build(), ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(query2.build()), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||
assertEquals(0, scorer.iterator().nextDoc());
|
||||
assertTrue(scorer.getClass().getName(), scorer instanceof FilterScorer);
|
||||
|
|
|
@ -172,7 +172,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
.build();
|
||||
|
||||
// no scores -> term scorer
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
BulkScorer scorer = ((BooleanWeight) weight).booleanScorer(ctx);
|
||||
assertTrue(scorer instanceof DefaultBulkScorer); // term scorer
|
||||
|
||||
|
@ -181,7 +181,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD) // existing term
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) // missing term
|
||||
.build();
|
||||
weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
scorer = ((BooleanWeight) weight).booleanScorer(ctx);
|
||||
assertTrue(scorer instanceof DefaultBulkScorer); // term scorer
|
||||
|
||||
|
@ -210,7 +210,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
|
||||
.build();
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
BulkScorer scorer = ((BooleanWeight) weight).booleanScorer(ctx);
|
||||
assertTrue(scorer instanceof ReqExclBulkScorer);
|
||||
|
||||
|
@ -219,7 +219,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
.add(new MatchAllDocsQuery(), Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
|
||||
.build();
|
||||
weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
scorer = ((BooleanWeight) weight).booleanScorer(ctx);
|
||||
assertTrue(scorer instanceof ReqExclBulkScorer);
|
||||
|
||||
|
@ -227,7 +227,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
.add(new TermQuery(new Term("foo", "baz")), Occur.MUST)
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
|
||||
.build();
|
||||
weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
scorer = ((BooleanWeight) weight).booleanScorer(ctx);
|
||||
assertTrue(scorer instanceof ReqExclBulkScorer);
|
||||
|
||||
|
@ -235,7 +235,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
.add(new TermQuery(new Term("foo", "baz")), Occur.FILTER)
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
|
||||
.build();
|
||||
weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
scorer = ((BooleanWeight) weight).booleanScorer(ctx);
|
||||
assertTrue(scorer instanceof ReqExclBulkScorer);
|
||||
|
||||
|
|
|
@ -201,9 +201,9 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
|
||||
PhraseQuery pq = new PhraseQuery("field", "a", "b");
|
||||
|
||||
ConstantScoreQuery q = new ConstantScoreQuery(pq);
|
||||
Query q = searcher.rewrite(new ConstantScoreQuery(pq));
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(q, ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertNotNull(scorer.twoPhaseIterator());
|
||||
|
||||
|
@ -215,14 +215,14 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
public void testExtractTerms() throws Exception {
|
||||
final IndexSearcher searcher = newSearcher(new MultiReader());
|
||||
final TermQuery termQuery = new TermQuery(new Term("foo", "bar"));
|
||||
final ConstantScoreQuery csq = new ConstantScoreQuery(termQuery);
|
||||
final Query csq = searcher.rewrite(new ConstantScoreQuery(termQuery));
|
||||
|
||||
final Set<Term> scoringTerms = new HashSet<>();
|
||||
searcher.createNormalizedWeight(csq, ScoreMode.COMPLETE).extractTerms(scoringTerms);
|
||||
searcher.createWeight(csq, ScoreMode.COMPLETE, 1).extractTerms(scoringTerms);
|
||||
assertEquals(Collections.emptySet(), scoringTerms);
|
||||
|
||||
final Set<Term> matchingTerms = new HashSet<>();
|
||||
searcher.createNormalizedWeight(csq, ScoreMode.COMPLETE_NO_SCORES).extractTerms(matchingTerms);
|
||||
searcher.createWeight(csq, ScoreMode.COMPLETE_NO_SCORES, 1).extractTerms(matchingTerms);
|
||||
assertEquals(Collections.singleton(new Term("foo", "bar")), matchingTerms);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -175,7 +175,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
|||
|
||||
QueryUtils.check(random(), dq, s);
|
||||
assertTrue(s.getTopReaderContext() instanceof LeafReaderContext);
|
||||
final Weight dw = s.createNormalizedWeight(dq, ScoreMode.COMPLETE);
|
||||
final Weight dw = s.createWeight(s.rewrite(dq), ScoreMode.COMPLETE, 1);
|
||||
LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
|
||||
final Scorer ds = dw.scorer(context);
|
||||
final boolean skipOk = ds.iterator().advance(3) != DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
@ -191,7 +191,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
|||
|
||||
assertTrue(s.getTopReaderContext() instanceof LeafReaderContext);
|
||||
QueryUtils.check(random(), dq, s);
|
||||
final Weight dw = s.createNormalizedWeight(dq, ScoreMode.COMPLETE);
|
||||
final Weight dw = s.createWeight(s.rewrite(dq), ScoreMode.COMPLETE, 1);
|
||||
LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
|
||||
final Scorer ds = dw.scorer(context);
|
||||
assertTrue("firsttime skipTo found no match",
|
||||
|
|
|
@ -230,7 +230,7 @@ public class TestDocValuesQueries extends LuceneTestCase {
|
|||
SortedNumericDocValuesField.newSlowRangeQuery("foo", 2, 4),
|
||||
SortedDocValuesField.newSlowRangeQuery("foo", new BytesRef("abc"), new BytesRef("bcd"), random().nextBoolean(), random().nextBoolean()),
|
||||
SortedSetDocValuesField.newSlowRangeQuery("foo", new BytesRef("abc"), new BytesRef("bcd"), random().nextBoolean(), random().nextBoolean()))) {
|
||||
Weight w = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
Weight w = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
assertNull(w.scorer(searcher.getIndexReader().leaves().get(0)));
|
||||
}
|
||||
reader.close();
|
||||
|
|
|
@ -67,7 +67,7 @@ public class TestIndexOrDocValuesQuery extends LuceneTestCase {
|
|||
.add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 2), NumericDocValuesField.newSlowRangeQuery("f2", 2L, 2L)), Occur.MUST)
|
||||
.build();
|
||||
|
||||
final Weight w1 = searcher.createNormalizedWeight(q1, ScoreMode.COMPLETE);
|
||||
final Weight w1 = searcher.createWeight(searcher.rewrite(q1), ScoreMode.COMPLETE, 1);
|
||||
final Scorer s1 = w1.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertNotNull(s1.twoPhaseIterator()); // means we use doc values
|
||||
|
||||
|
@ -77,7 +77,7 @@ public class TestIndexOrDocValuesQuery extends LuceneTestCase {
|
|||
.add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 42), NumericDocValuesField.newSlowRangeQuery("f2", 42L, 42L)), Occur.MUST)
|
||||
.build();
|
||||
|
||||
final Weight w2 = searcher.createNormalizedWeight(q2, ScoreMode.COMPLETE);
|
||||
final Weight w2 = searcher.createWeight(searcher.rewrite(q2), ScoreMode.COMPLETE, 1);
|
||||
final Scorer s2 = w2.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertNull(s2.twoPhaseIterator()); // means we use points
|
||||
|
||||
|
|
|
@ -1141,7 +1141,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
|||
LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE, context -> true);
|
||||
|
||||
// test that the bulk scorer is propagated when a scorer should not be cached
|
||||
Weight weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), ScoreMode.COMPLETE_NO_SCORES);
|
||||
Weight weight = searcher.createWeight(new MatchAllDocsQuery(), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
weight = new WeightWrapper(weight, scorerCalled, bulkScorerCalled);
|
||||
weight = cache.doCache(weight, NEVER_CACHE);
|
||||
weight.bulkScorer(leaf);
|
||||
|
@ -1151,7 +1151,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
|||
|
||||
// test that the doc id set is computed using the bulk scorer
|
||||
bulkScorerCalled.set(false);
|
||||
weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), ScoreMode.COMPLETE_NO_SCORES);
|
||||
weight = searcher.createWeight(new MatchAllDocsQuery(), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
weight = new WeightWrapper(weight, scorerCalled, bulkScorerCalled);
|
||||
weight = cache.doCache(weight, QueryCachingPolicy.ALWAYS_CACHE);
|
||||
weight.scorer(leaf);
|
||||
|
@ -1424,7 +1424,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
|||
|
||||
AtomicBoolean scorerCreated = new AtomicBoolean(false);
|
||||
Query query = new DummyQuery2(scorerCreated);
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
ScorerSupplier supplier = weight.scorerSupplier(searcher.getIndexReader().leaves().get(0));
|
||||
assertFalse(scorerCreated.get());
|
||||
supplier.get(random().nextLong() & 0x7FFFFFFFFFFFFFFFL);
|
||||
|
|
|
@ -118,7 +118,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase {
|
|||
}
|
||||
bq.setMinimumNumberShouldMatch(minShouldMatch);
|
||||
|
||||
BooleanWeight weight = (BooleanWeight) searcher.createNormalizedWeight(bq.build(), ScoreMode.COMPLETE);
|
||||
BooleanWeight weight = (BooleanWeight) searcher.createWeight(searcher.rewrite(bq.build()), ScoreMode.COMPLETE, 1);
|
||||
|
||||
switch (mode) {
|
||||
case DOC_VALUES:
|
||||
|
|
|
@ -1903,7 +1903,7 @@ public class TestPointQueries extends LuceneTestCase {
|
|||
upperBound[i] = value[i] + random().nextInt(1);
|
||||
}
|
||||
Query query = IntPoint.newRangeQuery("point", lowerBound, upperBound);
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertEquals(DocIdSetIterator.all(1).getClass(), scorer.iterator().getClass());
|
||||
|
||||
|
@ -1914,7 +1914,7 @@ public class TestPointQueries extends LuceneTestCase {
|
|||
reader = w.getReader();
|
||||
searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCache(null);
|
||||
weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertFalse(DocIdSetIterator.all(1).getClass().equals(scorer.iterator().getClass()));
|
||||
|
||||
|
|
|
@ -62,7 +62,7 @@ public class TestReqOptSumScorer extends LuceneTestCase {
|
|||
.add(new ConstantScoreQuery(new TermQuery(new Term("f", "foo"))), Occur.MUST)
|
||||
.add(new ConstantScoreQuery(new TermQuery(new Term("f", "bar"))), Occur.SHOULD)
|
||||
.build();
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.TOP_SCORES);
|
||||
Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1);
|
||||
LeafReaderContext context = searcher.getIndexReader().leaves().get(0);
|
||||
|
||||
Scorer scorer = weight.scorer(context);
|
||||
|
|
|
@ -73,10 +73,10 @@ public class TestTermQuery extends LuceneTestCase {
|
|||
IndexSearcher noSeekSearcher = new IndexSearcher(noSeekReader);
|
||||
Query query = new TermQuery(new Term("foo", "bar"));
|
||||
AssertionError e = expectThrows(AssertionError.class,
|
||||
() -> noSeekSearcher.createNormalizedWeight(query, ScoreMode.COMPLETE));
|
||||
() -> noSeekSearcher.createWeight(noSeekSearcher.rewrite(query), ScoreMode.COMPLETE, 1));
|
||||
assertEquals("no seek", e.getMessage());
|
||||
|
||||
noSeekSearcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES); // no exception
|
||||
noSeekSearcher.createWeight(noSeekSearcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1); // no exception
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
// use a collector rather than searcher.count() which would just read the
|
||||
// doc freq instead of creating a scorer
|
||||
|
|
|
@ -82,7 +82,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
Term allTerm = new Term(FIELD, "all");
|
||||
TermQuery termQuery = new TermQuery(allTerm);
|
||||
|
||||
Weight weight = indexSearcher.createNormalizedWeight(termQuery, ScoreMode.COMPLETE);
|
||||
Weight weight = indexSearcher.createWeight(termQuery, ScoreMode.COMPLETE, 1);
|
||||
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
||||
LeafReaderContext context = (LeafReaderContext)indexSearcher.getTopReaderContext();
|
||||
BulkScorer ts = weight.bulkScorer(context);
|
||||
|
@ -133,7 +133,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
Term allTerm = new Term(FIELD, "all");
|
||||
TermQuery termQuery = new TermQuery(allTerm);
|
||||
|
||||
Weight weight = indexSearcher.createNormalizedWeight(termQuery, ScoreMode.COMPLETE);
|
||||
Weight weight = indexSearcher.createWeight(termQuery, ScoreMode.COMPLETE, 1);
|
||||
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
||||
LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext();
|
||||
Scorer ts = weight.scorer(context);
|
||||
|
@ -150,7 +150,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
Term allTerm = new Term(FIELD, "all");
|
||||
TermQuery termQuery = new TermQuery(allTerm);
|
||||
|
||||
Weight weight = indexSearcher.createNormalizedWeight(termQuery, ScoreMode.COMPLETE);
|
||||
Weight weight = indexSearcher.createWeight(termQuery, ScoreMode.COMPLETE, 1);
|
||||
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
||||
LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext();
|
||||
Scorer ts = weight.scorer(context);
|
||||
|
@ -199,12 +199,12 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
// We don't use newSearcher because it sometimes runs checkIndex which loads norms
|
||||
IndexSearcher indexSearcher = new IndexSearcher(forbiddenNorms);
|
||||
|
||||
Weight weight = indexSearcher.createNormalizedWeight(termQuery, ScoreMode.COMPLETE);
|
||||
Weight weight = indexSearcher.createWeight(termQuery, ScoreMode.COMPLETE, 1);
|
||||
expectThrows(AssertionError.class, () -> {
|
||||
weight.scorer(forbiddenNorms.getContext()).iterator().nextDoc();
|
||||
});
|
||||
|
||||
Weight weight2 = indexSearcher.createNormalizedWeight(termQuery, ScoreMode.COMPLETE_NO_SCORES);
|
||||
Weight weight2 = indexSearcher.createWeight(termQuery, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
// should not fail this time since norms are not necessary
|
||||
weight2.scorer(forbiddenNorms.getContext()).iterator().nextDoc();
|
||||
}
|
||||
|
|
|
@ -316,7 +316,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
|
|||
}
|
||||
|
||||
// ... then all shards:
|
||||
final Weight w = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
final Weight w = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
|
||||
final TopDocs[] shardHits;
|
||||
if (sort == null) {
|
||||
|
|
|
@ -92,7 +92,7 @@ public class TestWANDScorer extends LuceneTestCase {
|
|||
.build();
|
||||
|
||||
Scorer scorer = searcher
|
||||
.createNormalizedWeight(query, ScoreMode.TOP_SCORES)
|
||||
.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1)
|
||||
.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
|
||||
assertEquals(0, scorer.iterator().nextDoc());
|
||||
|
@ -113,7 +113,7 @@ public class TestWANDScorer extends LuceneTestCase {
|
|||
assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
|
||||
|
||||
scorer = searcher
|
||||
.createNormalizedWeight(query, ScoreMode.TOP_SCORES)
|
||||
.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1)
|
||||
.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
scorer.setMinCompetitiveScore(4);
|
||||
|
||||
|
@ -126,7 +126,7 @@ public class TestWANDScorer extends LuceneTestCase {
|
|||
assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
|
||||
|
||||
scorer = searcher
|
||||
.createNormalizedWeight(query, ScoreMode.TOP_SCORES)
|
||||
.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1)
|
||||
.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
|
||||
assertEquals(0, scorer.iterator().nextDoc());
|
||||
|
@ -147,7 +147,7 @@ public class TestWANDScorer extends LuceneTestCase {
|
|||
.build();
|
||||
|
||||
scorer = searcher
|
||||
.createNormalizedWeight(query, ScoreMode.TOP_SCORES)
|
||||
.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1)
|
||||
.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
|
||||
assertEquals(3, scorer.iterator().nextDoc());
|
||||
|
@ -159,7 +159,7 @@ public class TestWANDScorer extends LuceneTestCase {
|
|||
assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
|
||||
|
||||
scorer = searcher
|
||||
.createNormalizedWeight(query, ScoreMode.TOP_SCORES)
|
||||
.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1)
|
||||
.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
|
||||
scorer.setMinCompetitiveScore(2);
|
||||
|
@ -177,7 +177,7 @@ public class TestWANDScorer extends LuceneTestCase {
|
|||
.build();
|
||||
|
||||
scorer = searcher
|
||||
.createNormalizedWeight(query, ScoreMode.TOP_SCORES)
|
||||
.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1)
|
||||
.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
|
||||
assertEquals(0, scorer.iterator().nextDoc());
|
||||
|
@ -192,7 +192,7 @@ public class TestWANDScorer extends LuceneTestCase {
|
|||
assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
|
||||
|
||||
scorer = searcher
|
||||
.createNormalizedWeight(query, ScoreMode.TOP_SCORES)
|
||||
.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1)
|
||||
.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
|
||||
scorer.setMinCompetitiveScore(3);
|
||||
|
|
|
@ -194,7 +194,7 @@ public class TestNearSpansOrdered extends LuceneTestCase {
|
|||
*/
|
||||
public void testSpanNearScorerSkipTo1() throws Exception {
|
||||
SpanNearQuery q = makeQuery();
|
||||
Weight w = searcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Weight w = searcher.createWeight(searcher.rewrite(q), ScoreMode.COMPLETE, 1);
|
||||
IndexReaderContext topReaderContext = searcher.getTopReaderContext();
|
||||
LeafReaderContext leave = topReaderContext.leaves().get(0);
|
||||
Scorer s = w.scorer(leave);
|
||||
|
|
|
@ -84,7 +84,7 @@ class DrillSidewaysQuery extends Query {
|
|||
final Weight baseWeight = baseQuery.createWeight(searcher, scoreMode, boost);
|
||||
final Weight[] drillDowns = new Weight[drillDownQueries.length];
|
||||
for(int dim=0;dim<drillDownQueries.length;dim++) {
|
||||
drillDowns[dim] = searcher.createNormalizedWeight(drillDownQueries[dim], ScoreMode.COMPLETE_NO_SCORES);
|
||||
drillDowns[dim] = searcher.createWeight(searcher.rewrite(drillDownQueries[dim]), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
}
|
||||
|
||||
return new Weight(DrillSidewaysQuery.this) {
|
||||
|
|
|
@ -102,7 +102,7 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
|
|||
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
|
||||
final IndexSearcher searcher = new IndexSearcher(topLevelContext);
|
||||
searcher.setQueryCache(null);
|
||||
final Weight fastMatchWeight = searcher.createNormalizedWeight(fastMatchQuery, ScoreMode.COMPLETE_NO_SCORES);
|
||||
final Weight fastMatchWeight = searcher.createWeight(searcher.rewrite(fastMatchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
Scorer s = fastMatchWeight.scorer(hits.context);
|
||||
if (s == null) {
|
||||
continue;
|
||||
|
|
|
@ -85,7 +85,7 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
|
|||
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
|
||||
final IndexSearcher searcher = new IndexSearcher(topLevelContext);
|
||||
searcher.setQueryCache(null);
|
||||
final Weight fastMatchWeight = searcher.createNormalizedWeight(fastMatchQuery, ScoreMode.COMPLETE_NO_SCORES);
|
||||
final Weight fastMatchWeight = searcher.createWeight(searcher.rewrite(fastMatchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
Scorer s = fastMatchWeight.scorer(hits.context);
|
||||
if (s == null) {
|
||||
continue;
|
||||
|
|
|
@ -172,7 +172,8 @@ public class GroupingSearch {
|
|||
|
||||
protected TopGroups<?> groupByDocBlock(IndexSearcher searcher, Query query, int groupOffset, int groupLimit) throws IOException {
|
||||
int topN = groupOffset + groupLimit;
|
||||
final Weight groupEndDocs = searcher.createNormalizedWeight(this.groupEndDocs, ScoreMode.COMPLETE_NO_SCORES);
|
||||
final Query endDocsQuery = searcher.rewrite(this.groupEndDocs);
|
||||
final Weight groupEndDocs = searcher.createWeight(endDocsQuery, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
BlockGroupingCollector c = new BlockGroupingCollector(groupSort, topN, includeScores, groupEndDocs);
|
||||
searcher.search(query, c);
|
||||
int topNInsideGroup = groupDocsOffset + groupDocsLimit;
|
||||
|
|
|
@ -1042,7 +1042,8 @@ public class TestGrouping extends LuceneTestCase {
|
|||
}
|
||||
|
||||
final boolean needsScores = getScores || getMaxScores || docSort == null;
|
||||
final BlockGroupingCollector c3 = new BlockGroupingCollector(groupSort, groupOffset+topNGroups, needsScores, sBlocks.createNormalizedWeight(lastDocInBlock, ScoreMode.COMPLETE_NO_SCORES));
|
||||
final BlockGroupingCollector c3 = new BlockGroupingCollector(groupSort, groupOffset+topNGroups, needsScores,
|
||||
sBlocks.createWeight(sBlocks.rewrite(lastDocInBlock), ScoreMode.COMPLETE_NO_SCORES, 1));
|
||||
final AllGroupsCollector<BytesRef> allGroupsCollector2;
|
||||
final Collector c4;
|
||||
if (doAllGroups) {
|
||||
|
@ -1163,7 +1164,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
System.out.println("TEST: " + subSearchers.length + " shards: " + Arrays.toString(subSearchers) + " canUseIDV=" + canUseIDV);
|
||||
}
|
||||
// Run 1st pass collector to get top groups per shard
|
||||
final Weight w = topSearcher.createNormalizedWeight(query, getScores || getMaxScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES);
|
||||
final Weight w = topSearcher.createWeight(topSearcher.rewrite(query), getScores || getMaxScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
final List<Collection<SearchGroup<BytesRef>>> shardGroups = new ArrayList<>();
|
||||
List<FirstPassGroupingCollector<?>> firstPassGroupingCollectors = new ArrayList<>();
|
||||
FirstPassGroupingCollector<?> firstPassCollector = null;
|
||||
|
|
|
@ -129,7 +129,7 @@ public final class QueryTermExtractor
|
|||
else {
|
||||
HashSet<Term> nonWeightedTerms = new HashSet<>();
|
||||
try {
|
||||
EMPTY_INDEXSEARCHER.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES).extractTerms(nonWeightedTerms);
|
||||
EMPTY_INDEXSEARCHER.createWeight(EMPTY_INDEXSEARCHER.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1).extractTerms(nonWeightedTerms);
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException("Should not happen on an empty index", bogus);
|
||||
}
|
||||
|
|
|
@ -306,7 +306,7 @@ public class WeightedSpanTermExtractor {
|
|||
q = spanQuery;
|
||||
}
|
||||
LeafReaderContext context = getLeafContext();
|
||||
SpanWeight w = (SpanWeight) searcher.createNormalizedWeight(q, ScoreMode.COMPLETE_NO_SCORES);
|
||||
SpanWeight w = (SpanWeight) searcher.createWeight(searcher.rewrite(q), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
Bits acceptDocs = context.reader().getLiveDocs();
|
||||
final Spans spans = w.getSpans(context, SpanWeight.Postings.POSITIONS);
|
||||
if (spans == null) {
|
||||
|
@ -360,7 +360,7 @@ public class WeightedSpanTermExtractor {
|
|||
protected void extractWeightedTerms(Map<String,WeightedSpanTerm> terms, Query query, float boost) throws IOException {
|
||||
Set<Term> nonWeightedTerms = new HashSet<>();
|
||||
final IndexSearcher searcher = new IndexSearcher(getLeafContext());
|
||||
searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES).extractTerms(nonWeightedTerms);
|
||||
searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1).extractTerms(nonWeightedTerms);
|
||||
|
||||
for (final Term queryTerm : nonWeightedTerms) {
|
||||
|
||||
|
|
|
@ -227,7 +227,7 @@ public class PhraseHelper {
|
|||
}
|
||||
};
|
||||
for (Query query : spanQueries) {
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
Scorer scorer = weight.scorer(leafReader.getContext());
|
||||
if (scorer == null) {
|
||||
continue;
|
||||
|
|
|
@ -144,7 +144,7 @@ public class UnifiedHighlighter {
|
|||
*/
|
||||
protected static Set<Term> extractTerms(Query query) throws IOException {
|
||||
Set<Term> queryTerms = new HashSet<>();
|
||||
EMPTY_INDEXSEARCHER.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES).extractTerms(queryTerms);
|
||||
EMPTY_INDEXSEARCHER.createWeight(EMPTY_INDEXSEARCHER.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1).extractTerms(queryTerms);
|
||||
return queryTerms;
|
||||
}
|
||||
|
||||
|
|
|
@ -70,7 +70,8 @@ public class QueryBitSetProducer implements BitSetProducer {
|
|||
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
|
||||
final IndexSearcher searcher = new IndexSearcher(topLevelContext);
|
||||
searcher.setQueryCache(null);
|
||||
final Weight weight = searcher.createNormalizedWeight(query, org.apache.lucene.search.ScoreMode.COMPLETE_NO_SCORES);
|
||||
final Query rewritten = searcher.rewrite(query);
|
||||
final Weight weight = searcher.createWeight(rewritten, org.apache.lucene.search.ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
final Scorer s = weight.scorer(context);
|
||||
|
||||
if (s == null) {
|
||||
|
|
|
@ -1113,7 +1113,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
CheckJoinIndex.check(s.getIndexReader(), parentFilter);
|
||||
|
||||
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
||||
Weight weight = s.createNormalizedWeight(q, org.apache.lucene.search.ScoreMode.COMPLETE);
|
||||
Weight weight = s.createWeight(s.rewrite(q), org.apache.lucene.search.ScoreMode.COMPLETE, 1);
|
||||
Scorer sc = weight.scorer(s.getIndexReader().leaves().get(0));
|
||||
assertEquals(1, sc.iterator().advance(1));
|
||||
r.close();
|
||||
|
@ -1147,7 +1147,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
CheckJoinIndex.check(s.getIndexReader(), parentFilter);
|
||||
|
||||
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
||||
Weight weight = s.createNormalizedWeight(q, org.apache.lucene.search.ScoreMode.COMPLETE);
|
||||
Weight weight = s.createWeight(s.rewrite(q), org.apache.lucene.search.ScoreMode.COMPLETE, 1);
|
||||
Scorer sc = weight.scorer(s.getIndexReader().leaves().get(0));
|
||||
assertEquals(2, sc.iterator().advance(0));
|
||||
r.close();
|
||||
|
@ -1199,7 +1199,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
CheckJoinIndex.check(r, parentsFilter);
|
||||
ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
|
||||
|
||||
Weight weight = searcher.createNormalizedWeight(childJoinQuery, RandomPicks.randomFrom(random(), org.apache.lucene.search.ScoreMode.values()));
|
||||
Weight weight = searcher.createWeight(searcher.rewrite(childJoinQuery), RandomPicks.randomFrom(random(), org.apache.lucene.search.ScoreMode.values()), 1);
|
||||
Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertNull(scorer);
|
||||
|
||||
|
@ -1207,7 +1207,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
childQuery = new TermQuery(new Term("bogus", "bogus"));
|
||||
childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
|
||||
|
||||
weight = searcher.createNormalizedWeight(childJoinQuery, RandomPicks.randomFrom(random(), org.apache.lucene.search.ScoreMode.values()));
|
||||
weight = searcher.createWeight(searcher.rewrite(childJoinQuery), RandomPicks.randomFrom(random(), org.apache.lucene.search.ScoreMode.values()), 1);
|
||||
scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertNull(scorer);
|
||||
|
||||
|
@ -1401,7 +1401,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
|
||||
ToChildBlockJoinQuery parentJoinQuery = new ToChildBlockJoinQuery(parentQuery, parentFilter);
|
||||
|
||||
Weight weight = s.createNormalizedWeight(parentJoinQuery, RandomPicks.randomFrom(random(), org.apache.lucene.search.ScoreMode.values()));
|
||||
Weight weight = s.createWeight(s.rewrite(parentJoinQuery), RandomPicks.randomFrom(random(), org.apache.lucene.search.ScoreMode.values()), 1);
|
||||
Scorer advancingScorer = weight.scorer(s.getIndexReader().leaves().get(0));
|
||||
Scorer nextDocScorer = weight.scorer(s.getIndexReader().leaves().get(0));
|
||||
|
||||
|
|
|
@ -103,7 +103,7 @@ public class TestBlockJoinValidation extends LuceneTestCase {
|
|||
ToChildBlockJoinQuery blockJoinQuery = new ToChildBlockJoinQuery(parentQuery, parentsFilter);
|
||||
|
||||
final LeafReaderContext context = indexSearcher.getIndexReader().leaves().get(0);
|
||||
Weight weight = indexSearcher.createNormalizedWeight(blockJoinQuery, org.apache.lucene.search.ScoreMode.COMPLETE);
|
||||
Weight weight = indexSearcher.createWeight(indexSearcher.rewrite(blockJoinQuery), org.apache.lucene.search.ScoreMode.COMPLETE, 1);
|
||||
Scorer scorer = weight.scorer(context);
|
||||
final Bits parentDocs = parentsFilter.getBitSet(context);
|
||||
|
||||
|
|
|
@ -106,7 +106,8 @@ public class PKIndexSplitter {
|
|||
try {
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCache(null);
|
||||
final Weight preserveWeight = searcher.createNormalizedWeight(preserveFilter, ScoreMode.COMPLETE_NO_SCORES);
|
||||
preserveFilter = searcher.rewrite(preserveFilter);
|
||||
final Weight preserveWeight = searcher.createWeight(preserveFilter, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
final List<LeafReaderContext> leaves = reader.leaves();
|
||||
final CodecReader[] subReaders = new CodecReader[leaves.size()];
|
||||
int i = 0;
|
||||
|
|
|
@ -72,7 +72,8 @@ public class QueryValueSource extends ValueSource {
|
|||
|
||||
@Override
|
||||
public void createWeight(Map context, IndexSearcher searcher) throws IOException {
|
||||
Weight w = searcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Query rewritten = searcher.rewrite(q);
|
||||
Weight w = searcher.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
context.put(this, w);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -159,14 +159,14 @@ public class TestIndexReaderFunctions extends LuceneTestCase {
|
|||
|
||||
void assertCacheable(DoubleValuesSource vs, boolean expected) throws Exception {
|
||||
Query q = new FunctionScoreQuery(new MatchAllDocsQuery(), vs);
|
||||
Weight w = searcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Weight w = searcher.createWeight(q, ScoreMode.COMPLETE, 1);
|
||||
LeafReaderContext ctx = reader.leaves().get(0);
|
||||
assertEquals(expected, w.isCacheable(ctx));
|
||||
}
|
||||
|
||||
void assertCacheable(LongValuesSource vs, boolean expected) throws Exception {
|
||||
Query q = new FunctionScoreQuery(new MatchAllDocsQuery(), vs.toDoubleValuesSource());
|
||||
Weight w = searcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Weight w = searcher.createWeight(q, ScoreMode.COMPLETE, 1);
|
||||
LeafReaderContext ctx = reader.leaves().get(0);
|
||||
assertEquals(expected, w.isCacheable(ctx));
|
||||
}
|
||||
|
|
|
@ -172,7 +172,8 @@ public class PayloadSpanUtil {
|
|||
final IndexSearcher searcher = new IndexSearcher(context);
|
||||
searcher.setQueryCache(null);
|
||||
|
||||
SpanWeight w = (SpanWeight) searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
query = (SpanQuery) searcher.rewrite(query);
|
||||
SpanWeight w = (SpanWeight) searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
|
||||
PayloadSpanCollector collector = new PayloadSpanCollector();
|
||||
for (LeafReaderContext leafReaderContext : context.leaves()) {
|
||||
|
|
|
@ -16,22 +16,21 @@
|
|||
*/
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Locale;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.Random;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Utility class for asserting expected hits in tests.
|
||||
*/
|
||||
|
@ -541,8 +540,9 @@ public class CheckHits {
|
|||
}
|
||||
|
||||
private static void doCheckMaxScores(Random random, Query query, IndexSearcher searcher) throws IOException {
|
||||
Weight w1 = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
Weight w2 = searcher.createNormalizedWeight(query, ScoreMode.TOP_SCORES);
|
||||
query = searcher.rewrite(query);
|
||||
Weight w1 = searcher.createWeight(query, ScoreMode.COMPLETE, 1);
|
||||
Weight w2 = searcher.createWeight(query, ScoreMode.TOP_SCORES, 1);
|
||||
|
||||
// Check boundaries and max scores when iterating all matches
|
||||
for (LeafReaderContext ctx : searcher.getIndexReader().leaves()) {
|
||||
|
|
|
@ -309,7 +309,8 @@ public class QueryUtils {
|
|||
lastDoc[0] = doc;
|
||||
try {
|
||||
if (scorer == null) {
|
||||
Weight w = s.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Query rewritten = s.rewrite(q);
|
||||
Weight w = s.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
LeafReaderContext context = readerContextArray.get(leafPtr);
|
||||
scorer = w.scorer(context);
|
||||
iterator = scorer.iterator();
|
||||
|
@ -374,7 +375,8 @@ public class QueryUtils {
|
|||
final LeafReader previousReader = lastReader[0];
|
||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
|
||||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Query rewritten = indexSearcher.rewrite(q);
|
||||
Weight w = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
LeafReaderContext ctx = (LeafReaderContext)indexSearcher.getTopReaderContext();
|
||||
Scorer scorer = w.scorer(ctx);
|
||||
if (scorer != null) {
|
||||
|
@ -404,7 +406,8 @@ public class QueryUtils {
|
|||
final LeafReader previousReader = lastReader[0];
|
||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
|
||||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Query rewritten = indexSearcher.rewrite(q);
|
||||
Weight w = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
LeafReaderContext ctx = previousReader.getContext();
|
||||
Scorer scorer = w.scorer(ctx);
|
||||
if (scorer != null) {
|
||||
|
@ -430,6 +433,7 @@ public class QueryUtils {
|
|||
final int lastDoc[] = {-1};
|
||||
final LeafReader lastReader[] = {null};
|
||||
final List<LeafReaderContext> context = s.getTopReaderContext().leaves();
|
||||
Query rewritten = s.rewrite(q);
|
||||
s.search(q,new SimpleCollector() {
|
||||
private Scorer scorer;
|
||||
private int leafPtr;
|
||||
|
@ -443,7 +447,7 @@ public class QueryUtils {
|
|||
try {
|
||||
long startMS = System.currentTimeMillis();
|
||||
for (int i=lastDoc[0]+1; i<=doc; i++) {
|
||||
Weight w = s.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Weight w = s.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
Scorer scorer = w.scorer(context.get(leafPtr));
|
||||
Assert.assertTrue("query collected "+doc+" but advance("+i+") says no more docs!",scorer.iterator().advance(i) != DocIdSetIterator.NO_MORE_DOCS);
|
||||
Assert.assertEquals("query collected "+doc+" but advance("+i+") got to "+scorer.docID(),doc,scorer.docID());
|
||||
|
@ -476,7 +480,7 @@ public class QueryUtils {
|
|||
final LeafReader previousReader = lastReader[0];
|
||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
|
||||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Weight w = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext());
|
||||
if (scorer != null) {
|
||||
DocIdSetIterator iterator = scorer.iterator();
|
||||
|
@ -504,7 +508,7 @@ public class QueryUtils {
|
|||
final LeafReader previousReader = lastReader[0];
|
||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
|
||||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
|
||||
Weight w = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE, 1);
|
||||
Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext());
|
||||
if (scorer != null) {
|
||||
DocIdSetIterator iterator = scorer.iterator();
|
||||
|
@ -523,7 +527,8 @@ public class QueryUtils {
|
|||
|
||||
/** Check that the scorer and bulk scorer advance consistently. */
|
||||
public static void checkBulkScorerSkipTo(Random r, Query query, IndexSearcher searcher) throws IOException {
|
||||
Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
query = searcher.rewrite(query);
|
||||
Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1);
|
||||
for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
|
||||
final Scorer scorer = weight.scorer(context);
|
||||
final BulkScorer bulkScorer = weight.bulkScorer(context);
|
||||
|
|
|
@ -230,7 +230,8 @@ public abstract class ShardSearchingTestBase extends LuceneTestCase {
|
|||
@Override
|
||||
public Query rewrite(Query original) throws IOException {
|
||||
final IndexSearcher localSearcher = new IndexSearcher(getIndexReader());
|
||||
final Weight weight = localSearcher.createNormalizedWeight(original, ScoreMode.COMPLETE);
|
||||
original = localSearcher.rewrite(original);
|
||||
final Weight weight = localSearcher.createWeight(original, ScoreMode.COMPLETE, 1);
|
||||
final Set<Term> terms = new HashSet<>();
|
||||
weight.extractTerms(terms);
|
||||
|
||||
|
|
|
@ -115,7 +115,7 @@ public class LTRRescorer extends Rescorer {
|
|||
final ScoreDoc[] reranked = new ScoreDoc[topN];
|
||||
final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||
final LTRScoringQuery.ModelWeight modelWeight = (LTRScoringQuery.ModelWeight) searcher
|
||||
.createNormalizedWeight(scoringQuery, ScoreMode.COMPLETE);
|
||||
.createWeight(searcher.rewrite(scoringQuery), ScoreMode.COMPLETE, 1);
|
||||
|
||||
scoreFeatures(searcher, firstPassTopDocs,topN, modelWeight, hits, leaves, reranked);
|
||||
// Must sort all documents that we reranked, and then select the top
|
||||
|
@ -219,8 +219,8 @@ public class LTRRescorer extends Rescorer {
|
|||
final int n = ReaderUtil.subIndex(docID, leafContexts);
|
||||
final LeafReaderContext context = leafContexts.get(n);
|
||||
final int deBasedDoc = docID - context.docBase;
|
||||
final Weight modelWeight = searcher.createNormalizedWeight(scoringQuery,
|
||||
ScoreMode.COMPLETE);
|
||||
final Weight modelWeight = searcher.createWeight(searcher.rewrite(scoringQuery),
|
||||
ScoreMode.COMPLETE, 1);
|
||||
return modelWeight.explain(context, deBasedDoc);
|
||||
}
|
||||
|
||||
|
|
|
@ -70,7 +70,7 @@ public class OriginalScoreFeature extends Feature {
|
|||
public OriginalScoreWeight(IndexSearcher searcher,
|
||||
SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi) throws IOException {
|
||||
super(OriginalScoreFeature.this, searcher, request, originalQuery, efi);
|
||||
w = searcher.createNormalizedWeight(originalQuery, ScoreMode.COMPLETE);
|
||||
w = searcher.createWeight(searcher.rewrite(originalQuery), ScoreMode.COMPLETE, 1);
|
||||
};
|
||||
|
||||
|
||||
|
|
|
@ -179,7 +179,7 @@ public class SolrFeature extends Feature {
|
|||
// leaving nothing for the phrase query to parse.
|
||||
if (query != null) {
|
||||
queryAndFilters.add(query);
|
||||
solrQueryWeight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE);
|
||||
solrQueryWeight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
|
||||
} else {
|
||||
solrQueryWeight = null;
|
||||
}
|
||||
|
|
|
@ -99,7 +99,7 @@ public class TestLTRScoringQuery extends LuceneTestCase {
|
|||
final LeafReaderContext context = leafContexts.get(n);
|
||||
final int deBasedDoc = hits.scoreDocs[0].doc - context.docBase;
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(model, ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(model), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(context);
|
||||
|
||||
// rerank using the field final-score
|
||||
|
|
|
@ -80,7 +80,7 @@ public class TestSelectiveWeightCreation extends TestRerankBase {
|
|||
final LeafReaderContext context = leafContexts.get(n);
|
||||
final int deBasedDoc = hits.scoreDocs[0].doc - context.docBase;
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(model, ScoreMode.COMPLETE);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(model), ScoreMode.COMPLETE, 1);
|
||||
final Scorer scorer = weight.scorer(context);
|
||||
|
||||
// rerank using the field final-score
|
||||
|
|
|
@ -66,7 +66,8 @@ public class QueryWrapperFilter extends Filter {
|
|||
public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
// get a private context that is used to rewrite, createWeight and score eventually
|
||||
final LeafReaderContext privateContext = context.reader().getContext();
|
||||
final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES);
|
||||
final IndexSearcher searcher = new IndexSearcher(privateContext);
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||
|
||||
DocIdSet set = new DocIdSet() {
|
||||
@Override
|
||||
|
|
|
@ -1061,7 +1061,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
|
|||
List<Weight> weights = new ArrayList<>(notCached.size());
|
||||
for (Query q : notCached) {
|
||||
Query qq = QueryUtils.makeQueryable(q);
|
||||
weights.add(createNormalizedWeight(qq, ScoreMode.COMPLETE));
|
||||
weights.add(createWeight(rewrite(qq), ScoreMode.COMPLETE, 1));
|
||||
}
|
||||
pf.filter = new FilterImpl(answer, weights);
|
||||
pf.hasDeletedDocs = (answer == null); // if all clauses were uncached, the resulting filter may match deleted docs
|
||||
|
|
|
@ -16,6 +16,16 @@
|
|||
*/
|
||||
package org.apache.solr.search.stats;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
|
@ -39,16 +49,6 @@ import org.apache.solr.search.SolrIndexSearcher;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* This class implements exact caching of statistics. It requires an additional
|
||||
* round-trip to parse query at shard servers, and return term statistics for
|
||||
|
@ -157,7 +157,7 @@ public class ExactStatsCache extends StatsCache {
|
|||
Query q = rb.getQuery();
|
||||
try {
|
||||
HashSet<Term> terms = new HashSet<>();
|
||||
searcher.createNormalizedWeight(q, ScoreMode.COMPLETE).extractTerms(terms);
|
||||
searcher.createWeight(searcher.rewrite(q), ScoreMode.COMPLETE, 1).extractTerms(terms);
|
||||
IndexReaderContext context = searcher.getTopReaderContext();
|
||||
HashMap<String,TermStats> statsMap = new HashMap<>();
|
||||
HashMap<String,CollectionStats> colMap = new HashMap<>();
|
||||
|
|
|
@ -228,7 +228,7 @@ public class TestQueryWrapperFilter extends LuceneTestCase {
|
|||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCache(null); // to still have approximations
|
||||
final Query query = new QueryWrapperFilter(new RandomApproximationQuery(new TermQuery(new Term("foo", "bar")), random()));
|
||||
final Weight weight = searcher.createNormalizedWeight(query, RandomPicks.randomFrom(random(), ScoreMode.values()));
|
||||
final Weight weight = searcher.createWeight(searcher.rewrite(query), RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||
assertNotNull(scorer.twoPhaseIterator());
|
||||
reader.close();
|
||||
|
|
Loading…
Reference in New Issue