mirror of https://github.com/apache/lucene.git
LUCENE-8483: Scorer cannot have a null Weight
This commit is contained in:
parent
0dc66c236d
commit
66c671ea80
|
@ -76,6 +76,8 @@ API Changes
|
|||
* LUCENE-8475: Deprecated constants have been removed from RamUsageEstimator.
|
||||
(Dimitrios Athanasiou)
|
||||
|
||||
* LUCENE-8483: Scorers may no longer take null as a Weight (Alan Woodward)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-8333: Switch MoreLikeThis.setMaxDocFreqPct to use maxDoc instead of
|
||||
|
|
|
@ -113,3 +113,8 @@ Scorer has a number of methods that should never be called from Collectors, for
|
|||
those that advance the underlying iterators. To hide these, LeafCollector.setScorer()
|
||||
now takes a Scorable, an abstract class that Scorers can extend, with methods
|
||||
docId() and score() (LUCENE-6228)
|
||||
|
||||
## Scorers must have non-null Weights ##
|
||||
|
||||
If a custom Scorer implementation does not have an associated Weight, it can probably
|
||||
be replaced with a Scorable instead.
|
||||
|
|
|
@ -20,9 +20,7 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedNumericSelector;
|
||||
|
@ -445,30 +443,5 @@ final class Sorter {
|
|||
public String toString() {
|
||||
return getID();
|
||||
}
|
||||
|
||||
static final Scorer FAKESCORER = new Scorer(null) {
|
||||
|
||||
float score;
|
||||
int doc = -1;
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
|
|
@ -30,13 +30,13 @@ import org.apache.lucene.search.BooleanClause.Occur;
|
|||
|
||||
final class Boolean2ScorerSupplier extends ScorerSupplier {
|
||||
|
||||
private final BooleanWeight weight;
|
||||
private final Weight weight;
|
||||
private final Map<BooleanClause.Occur, Collection<ScorerSupplier>> subs;
|
||||
private final ScoreMode scoreMode;
|
||||
private final int minShouldMatch;
|
||||
private long cost = -1;
|
||||
|
||||
Boolean2ScorerSupplier(BooleanWeight weight,
|
||||
Boolean2ScorerSupplier(Weight weight,
|
||||
Map<Occur, Collection<ScorerSupplier>> subs,
|
||||
ScoreMode scoreMode, int minShouldMatch) {
|
||||
if (minShouldMatch < 0) {
|
||||
|
|
|
@ -119,7 +119,7 @@ final class BooleanScorer extends BulkScorer {
|
|||
final BulkScorerAndDoc[] leads;
|
||||
final HeadPriorityQueue head;
|
||||
final TailPriorityQueue tail;
|
||||
final FakeScorer fakeScorer = new FakeScorer();
|
||||
final ScoreAndDoc scoreAndDoc = new ScoreAndDoc();
|
||||
final int minShouldMatch;
|
||||
final long cost;
|
||||
|
||||
|
@ -178,12 +178,12 @@ final class BooleanScorer extends BulkScorer {
|
|||
}
|
||||
|
||||
private void scoreDocument(LeafCollector collector, int base, int i) throws IOException {
|
||||
final FakeScorer fakeScorer = this.fakeScorer;
|
||||
final ScoreAndDoc scoreAndDoc = this.scoreAndDoc;
|
||||
final Bucket bucket = buckets[i];
|
||||
if (bucket.freq >= minShouldMatch) {
|
||||
fakeScorer.score = (float) bucket.score;
|
||||
scoreAndDoc.score = (float) bucket.score;
|
||||
final int doc = base | i;
|
||||
fakeScorer.doc = doc;
|
||||
scoreAndDoc.doc = doc;
|
||||
collector.collect(doc);
|
||||
}
|
||||
bucket.freq = 0;
|
||||
|
@ -276,7 +276,7 @@ final class BooleanScorer extends BulkScorer {
|
|||
bulkScorer.score(collector, acceptDocs, windowMin, end);
|
||||
|
||||
// reset the scorer that should be used for the general case
|
||||
collector.setScorer(fakeScorer);
|
||||
collector.setScorer(scoreAndDoc);
|
||||
}
|
||||
|
||||
private BulkScorerAndDoc scoreWindow(BulkScorerAndDoc top, LeafCollector collector,
|
||||
|
@ -307,8 +307,8 @@ final class BooleanScorer extends BulkScorer {
|
|||
|
||||
@Override
|
||||
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
|
||||
fakeScorer.doc = -1;
|
||||
collector.setScorer(fakeScorer);
|
||||
scoreAndDoc.doc = -1;
|
||||
collector.setScorer(scoreAndDoc);
|
||||
|
||||
BulkScorerAndDoc top = advance(min);
|
||||
while (top.next < max) {
|
||||
|
|
|
@ -160,7 +160,7 @@ final class BooleanWeight extends Weight {
|
|||
@Override
|
||||
public int score(final LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
|
||||
final LeafCollector noScoreCollector = new LeafCollector() {
|
||||
FakeScorer fake = new FakeScorer();
|
||||
ScoreAndDoc fake = new ScoreAndDoc();
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorable scorer) throws IOException {
|
||||
|
|
|
@ -47,7 +47,7 @@ public abstract class CachingCollector extends FilterCollector {
|
|||
|
||||
private static final int INITIAL_ARRAY_SIZE = 128;
|
||||
|
||||
private static final class CachedScorer extends Scorer {
|
||||
private static final class CachedScorable extends Scorable {
|
||||
|
||||
// NOTE: these members are package-private b/c that way accessing them from
|
||||
// the outer class does not incur access check by the JVM. The same
|
||||
|
@ -56,21 +56,9 @@ public abstract class CachingCollector extends FilterCollector {
|
|||
int doc;
|
||||
float score;
|
||||
|
||||
private CachedScorer() { super(null); }
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final float score() { return score; }
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
|
@ -188,7 +176,7 @@ public abstract class CachingCollector extends FilterCollector {
|
|||
final int[] docs = this.docs.get(i);
|
||||
final float[] scores = this.scores.get(i);
|
||||
assert docs.length == scores.length;
|
||||
final CachedScorer scorer = new CachedScorer();
|
||||
final CachedScorable scorer = new CachedScorable();
|
||||
collector.setScorer(scorer);
|
||||
for (int j = 0; j < docs.length; ++j) {
|
||||
scorer.doc = docs[j];
|
||||
|
|
|
@ -1,62 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
/** Used by {@link BulkScorer}s that need to pass a {@link
|
||||
* Scorer} to {@link LeafCollector#setScorer}. */
|
||||
final class FakeScorer extends Scorer {
|
||||
float score;
|
||||
int doc = -1;
|
||||
|
||||
public FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight getWeight() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorable> getChildren() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -53,7 +53,7 @@ public final class MatchAllDocsQuery extends Query {
|
|||
@Override
|
||||
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
|
||||
max = Math.min(max, maxDoc);
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
scorer.score = score;
|
||||
collector.setScorer(scorer);
|
||||
for (int doc = min; doc < max; ++doc) {
|
||||
|
|
|
@ -14,39 +14,22 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search.join;
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
|
||||
class FakeScorer extends Scorer {
|
||||
|
||||
/** Used by {@link BulkScorer}s that need to pass a {@link
|
||||
* Scorable} to {@link LeafCollector#setScorer}. */
|
||||
final class ScoreAndDoc extends Scorable {
|
||||
float score;
|
||||
int doc = -1;
|
||||
|
||||
FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
public float score() {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
}
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Expert: Common scoring functionality for different types of queries.
|
||||
|
@ -38,8 +39,8 @@ import java.io.IOException;
|
|||
* with these scores.
|
||||
*/
|
||||
public abstract class Scorer extends Scorable {
|
||||
/** the Scorer's parent Weight. in some cases this may be null */
|
||||
// TODO can we clean this up?
|
||||
|
||||
/** the Scorer's parent Weight */
|
||||
protected final Weight weight;
|
||||
|
||||
/**
|
||||
|
@ -47,7 +48,7 @@ public abstract class Scorer extends Scorable {
|
|||
* @param weight The scorers <code>Weight</code>.
|
||||
*/
|
||||
protected Scorer(Weight weight) {
|
||||
this.weight = weight;
|
||||
this.weight = Objects.requireNonNull(weight);
|
||||
}
|
||||
|
||||
/** returns parent Weight
|
||||
|
|
|
@ -59,7 +59,7 @@ public class SortRescorer extends Rescorer {
|
|||
int docBase = 0;
|
||||
|
||||
LeafCollector leafCollector = null;
|
||||
FakeScorer fakeScorer = new FakeScorer();
|
||||
ScoreAndDoc scoreAndDoc = new ScoreAndDoc();
|
||||
|
||||
while (hitUpto < hits.length) {
|
||||
ScoreDoc hit = hits[hitUpto];
|
||||
|
@ -74,14 +74,14 @@ public class SortRescorer extends Rescorer {
|
|||
if (readerContext != null) {
|
||||
// We advanced to another segment:
|
||||
leafCollector = collector.getLeafCollector(readerContext);
|
||||
leafCollector.setScorer(fakeScorer);
|
||||
leafCollector.setScorer(scoreAndDoc);
|
||||
docBase = readerContext.docBase;
|
||||
}
|
||||
|
||||
fakeScorer.score = hit.score;
|
||||
fakeScorer.doc = docID - docBase;
|
||||
scoreAndDoc.score = hit.score;
|
||||
scoreAndDoc.doc = docID - docBase;
|
||||
|
||||
leafCollector.collect(fakeScorer.doc);
|
||||
leafCollector.collect(scoreAndDoc.doc);
|
||||
|
||||
hitUpto++;
|
||||
}
|
||||
|
|
|
@ -70,7 +70,7 @@ public class MultiCollectorTest extends LuceneTestCase {
|
|||
final LeafCollector ac = c.getLeafCollector(null);
|
||||
ac.collect(1);
|
||||
c.getLeafCollector(null);
|
||||
c.getLeafCollector(null).setScorer(new FakeScorer());
|
||||
c.getLeafCollector(null).setScorer(new ScoreAndDoc());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -92,7 +92,7 @@ public class MultiCollectorTest extends LuceneTestCase {
|
|||
LeafCollector ac = c.getLeafCollector(null);
|
||||
ac.collect(1);
|
||||
ac = c.getLeafCollector(null);
|
||||
ac.setScorer(new FakeScorer());
|
||||
ac.setScorer(new ScoreAndDoc());
|
||||
|
||||
for (DummyCollector dc : dcs) {
|
||||
assertTrue(dc.collectCalled);
|
||||
|
@ -142,23 +142,23 @@ public class MultiCollectorTest extends LuceneTestCase {
|
|||
final LeafReaderContext ctx = reader.leaves().get(0);
|
||||
|
||||
expectThrows(AssertionError.class, () -> {
|
||||
collector(ScoreMode.COMPLETE_NO_SCORES, ScoreCachingWrappingScorer.class).getLeafCollector(ctx).setScorer(new FakeScorer());
|
||||
collector(ScoreMode.COMPLETE_NO_SCORES, ScoreCachingWrappingScorer.class).getLeafCollector(ctx).setScorer(new ScoreAndDoc());
|
||||
});
|
||||
|
||||
// no collector needs scores => no caching
|
||||
Collector c1 = collector(ScoreMode.COMPLETE_NO_SCORES, FakeScorer.class);
|
||||
Collector c2 = collector(ScoreMode.COMPLETE_NO_SCORES, FakeScorer.class);
|
||||
MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new FakeScorer());
|
||||
Collector c1 = collector(ScoreMode.COMPLETE_NO_SCORES, ScoreAndDoc.class);
|
||||
Collector c2 = collector(ScoreMode.COMPLETE_NO_SCORES, ScoreAndDoc.class);
|
||||
MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new ScoreAndDoc());
|
||||
|
||||
// only one collector needs scores => no caching
|
||||
c1 = collector(ScoreMode.COMPLETE, FakeScorer.class);
|
||||
c2 = collector(ScoreMode.COMPLETE_NO_SCORES, FakeScorer.class);
|
||||
MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new FakeScorer());
|
||||
c1 = collector(ScoreMode.COMPLETE, ScoreAndDoc.class);
|
||||
c2 = collector(ScoreMode.COMPLETE_NO_SCORES, ScoreAndDoc.class);
|
||||
MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new ScoreAndDoc());
|
||||
|
||||
// several collectors need scores => caching
|
||||
c1 = collector(ScoreMode.COMPLETE, ScoreCachingWrappingScorer.class);
|
||||
c2 = collector(ScoreMode.COMPLETE, ScoreCachingWrappingScorer.class);
|
||||
MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new FakeScorer());
|
||||
MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new ScoreAndDoc());
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
|
|
@ -22,7 +22,10 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.EnumMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
@ -31,12 +34,39 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
|||
|
||||
public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
||||
|
||||
private static class FakeWeight extends Weight {
|
||||
|
||||
FakeWeight() {
|
||||
super(new MatchNoDocsQuery());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable(LeafReaderContext ctx) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private static class FakeScorer extends Scorer {
|
||||
|
||||
private final DocIdSetIterator it;
|
||||
|
||||
FakeScorer(long cost) {
|
||||
super(null);
|
||||
super(new FakeWeight());
|
||||
this.it = DocIdSetIterator.all(Math.toIntExact(cost));
|
||||
}
|
||||
|
||||
|
@ -124,17 +154,17 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
}
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
|
||||
ScorerSupplier s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
|
||||
ScorerSupplier s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
|
||||
assertEquals(42, s.cost());
|
||||
assertEquals(42, s.get(random().nextInt(100)).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
|
||||
s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
|
||||
s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
|
||||
assertEquals(42 + 12, s.cost());
|
||||
assertEquals(42 + 12, s.get(random().nextInt(100)).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
|
||||
s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
|
||||
s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
|
||||
assertEquals(42 + 12 + 20, s.cost());
|
||||
assertEquals(42 + 12 + 20, s.get(random().nextInt(100)).iterator().cost());
|
||||
}
|
||||
|
@ -147,26 +177,26 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
|
||||
ScorerSupplier s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
|
||||
ScorerSupplier s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
|
||||
assertEquals(42 + 12, s.cost());
|
||||
assertEquals(42 + 12, s.get(random().nextInt(100)).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
|
||||
s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
|
||||
s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
|
||||
assertEquals(42 + 12 + 20, s.cost());
|
||||
assertEquals(42 + 12 + 20, s.get(random().nextInt(100)).iterator().cost());
|
||||
s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
|
||||
s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
|
||||
assertEquals(12 + 20, s.cost());
|
||||
assertEquals(12 + 20, s.get(random().nextInt(100)).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30));
|
||||
s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
|
||||
s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
|
||||
assertEquals(42 + 12 + 20 + 30, s.cost());
|
||||
assertEquals(42 + 12 + 20 + 30, s.get(random().nextInt(100)).iterator().cost());
|
||||
s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
|
||||
s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
|
||||
assertEquals(12 + 20 + 30, s.cost());
|
||||
assertEquals(12 + 20 + 30, s.get(random().nextInt(100)).iterator().cost());
|
||||
s = new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3);
|
||||
s = new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3);
|
||||
assertEquals(12 + 20, s.cost());
|
||||
assertEquals(12 + 20, s.get(random().nextInt(100)).iterator().cost());
|
||||
}
|
||||
|
@ -201,7 +231,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
continue;
|
||||
}
|
||||
int minShouldMatch = numShoulds == 0 ? 0 : TestUtil.nextInt(random(), 0, numShoulds - 1);
|
||||
Boolean2ScorerSupplier supplier = new Boolean2ScorerSupplier(null,
|
||||
Boolean2ScorerSupplier supplier = new Boolean2ScorerSupplier(new FakeWeight(),
|
||||
subs, scoreMode, minShouldMatch);
|
||||
long cost1 = supplier.cost();
|
||||
long cost2 = supplier.get(Long.MAX_VALUE).iterator().cost();
|
||||
|
@ -226,7 +256,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
// If the clauses are less costly than the lead cost, the min cost is the new lead cost
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42, 12));
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12, 12));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(Long.MAX_VALUE); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(Long.MAX_VALUE); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
|
@ -236,7 +266,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
// If the lead cost is less that the clauses' cost, then we don't modify it
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42, 7));
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12, 7));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(7); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(7); // triggers assertions as a side-effect
|
||||
}
|
||||
|
||||
public void testDisjunctionLeadCost() throws IOException {
|
||||
|
@ -246,12 +276,12 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
}
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 54));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 54));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(100); // triggers assertions as a side-effect
|
||||
|
||||
subs.get(Occur.SHOULD).clear();
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 20));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 20));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(20); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(20); // triggers assertions as a side-effect
|
||||
}
|
||||
|
||||
public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException {
|
||||
|
@ -265,7 +295,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(50, 42));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 42));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 42));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2).get(100); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
|
@ -276,7 +306,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 20));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 20));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 20));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2).get(20); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2).get(20); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
|
@ -287,7 +317,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 62));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 62));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, 62));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2).get(100); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
|
@ -298,7 +328,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 32));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 32));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, 32));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3).get(100); // triggers assertions as a side-effect
|
||||
}
|
||||
|
||||
public void testProhibitedLeadCost() throws IOException {
|
||||
|
@ -310,19 +340,19 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
// The MUST_NOT clause is called with the same lead cost as the MUST clause
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
|
||||
subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(30, 42));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(100); // triggers assertions as a side-effect
|
||||
|
||||
subs.get(Occur.MUST).clear();
|
||||
subs.get(Occur.MUST_NOT).clear();
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
|
||||
subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(80, 42));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(100); // triggers assertions as a side-effect
|
||||
|
||||
subs.get(Occur.MUST).clear();
|
||||
subs.get(Occur.MUST_NOT).clear();
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 20));
|
||||
subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(30, 20));
|
||||
new Boolean2ScorerSupplier(null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(20); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0).get(20); // triggers assertions as a side-effect
|
||||
}
|
||||
|
||||
public void testMixedLeadCost() throws IOException {
|
||||
|
@ -334,19 +364,19 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
|||
// The SHOULD clause is always called with the same lead cost as the MUST clause
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 42));
|
||||
new Boolean2ScorerSupplier(null, subs, ScoreMode.COMPLETE, 0).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0).get(100); // triggers assertions as a side-effect
|
||||
|
||||
subs.get(Occur.MUST).clear();
|
||||
subs.get(Occur.SHOULD).clear();
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(80, 42));
|
||||
new Boolean2ScorerSupplier(null, subs, ScoreMode.COMPLETE, 0).get(100); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0).get(100); // triggers assertions as a side-effect
|
||||
|
||||
subs.get(Occur.MUST).clear();
|
||||
subs.get(Occur.SHOULD).clear();
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 20));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(80, 20));
|
||||
new Boolean2ScorerSupplier(null, subs, ScoreMode.COMPLETE, 0).get(20); // triggers assertions as a side-effect
|
||||
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0).get(20); // triggers assertions as a side-effect
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -221,7 +221,7 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
|
||||
private static BulkScorer scorer(int... matches) {
|
||||
return new BulkScorer() {
|
||||
final FakeScorer scorer = new FakeScorer();
|
||||
final ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
int i = 0;
|
||||
@Override
|
||||
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
|
||||
|
|
|
@ -105,7 +105,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
@Override
|
||||
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
|
||||
assert min == 0;
|
||||
collector.setScorer(new FakeScorer());
|
||||
collector.setScorer(new ScoreAndDoc());
|
||||
collector.collect(0);
|
||||
return DocIdSetIterator.NO_MORE_DOCS;
|
||||
}
|
||||
|
|
|
@ -25,25 +25,14 @@ public class TestCachingCollector extends LuceneTestCase {
|
|||
|
||||
private static final double ONE_BYTE = 1.0 / (1024 * 1024); // 1 byte out of MB
|
||||
|
||||
private static class MockScorer extends Scorer {
|
||||
|
||||
private MockScorer() {
|
||||
super((Weight) null);
|
||||
}
|
||||
private static class MockScorable extends Scorable {
|
||||
|
||||
@Override
|
||||
public float score() throws IOException { return 0; }
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException { return 0; }
|
||||
public float score() { return 0; }
|
||||
|
||||
@Override
|
||||
public int docID() { return 0; }
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
private static class NoOpCollector extends SimpleCollector {
|
||||
|
@ -62,7 +51,7 @@ public class TestCachingCollector extends LuceneTestCase {
|
|||
for (boolean cacheScores : new boolean[] { false, true }) {
|
||||
CachingCollector cc = CachingCollector.create(new NoOpCollector(), cacheScores, 1.0);
|
||||
LeafCollector acc = cc.getLeafCollector(null);
|
||||
acc.setScorer(new MockScorer());
|
||||
acc.setScorer(new MockScorable());
|
||||
|
||||
// collect 1000 docs
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
|
@ -90,7 +79,7 @@ public class TestCachingCollector extends LuceneTestCase {
|
|||
public void testIllegalStateOnReplay() throws Exception {
|
||||
CachingCollector cc = CachingCollector.create(new NoOpCollector(), true, 50 * ONE_BYTE);
|
||||
LeafCollector acc = cc.getLeafCollector(null);
|
||||
acc.setScorer(new MockScorer());
|
||||
acc.setScorer(new MockScorable());
|
||||
|
||||
// collect 130 docs, this should be enough for triggering cache abort.
|
||||
for (int i = 0; i < 130; i++) {
|
||||
|
@ -115,7 +104,7 @@ public class TestCachingCollector extends LuceneTestCase {
|
|||
CachingCollector cc = CachingCollector.create(new NoOpCollector(),
|
||||
cacheScores, bytesPerDoc * ONE_BYTE * numDocs);
|
||||
LeafCollector acc = cc.getLeafCollector(null);
|
||||
acc.setScorer(new MockScorer());
|
||||
acc.setScorer(new MockScorable());
|
||||
for (int i = 0; i < numDocs; i++) acc.collect(i);
|
||||
assertTrue(cc.isCached());
|
||||
|
||||
|
@ -130,7 +119,7 @@ public class TestCachingCollector extends LuceneTestCase {
|
|||
// create w/ null wrapped collector, and test that the methods work
|
||||
CachingCollector cc = CachingCollector.create(cacheScores, 50 * ONE_BYTE);
|
||||
LeafCollector acc = cc.getLeafCollector(null);
|
||||
acc.setScorer(new MockScorer());
|
||||
acc.setScorer(new MockScorable());
|
||||
acc.collect(0);
|
||||
|
||||
assertTrue(cc.isCached());
|
||||
|
|
|
@ -22,7 +22,10 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.BitDocIdSet;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -82,6 +85,33 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
return scorer(TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator), twoPhaseIterator);
|
||||
}
|
||||
|
||||
private static class FakeWeight extends Weight {
|
||||
|
||||
protected FakeWeight() {
|
||||
super(new MatchNoDocsQuery());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable(LeafReaderContext ctx) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a {@link Scorer} that wraps the given {@link DocIdSetIterator}. It
|
||||
* also accepts a {@link TwoPhaseIterator} view, which is exposed in
|
||||
|
@ -91,7 +121,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
* advantage of the {@link TwoPhaseIterator} view.
|
||||
*/
|
||||
private static Scorer scorer(DocIdSetIterator it, TwoPhaseIterator twoPhaseIterator) {
|
||||
return new Scorer(null) {
|
||||
return new Scorer(new FakeWeight()) {
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
|
@ -204,12 +234,12 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
case 0:
|
||||
// simple iterator
|
||||
sets[i] = set;
|
||||
iterators[i] = new ConstantScoreScorer(null, 0f, anonymizeIterator(new BitDocIdSet(set).iterator()));
|
||||
iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, anonymizeIterator(new BitDocIdSet(set).iterator()));
|
||||
break;
|
||||
case 1:
|
||||
// bitSet iterator
|
||||
sets[i] = set;
|
||||
iterators[i] = new ConstantScoreScorer(null, 0f, new BitDocIdSet(set).iterator());
|
||||
iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator());
|
||||
break;
|
||||
default:
|
||||
// scorer with approximation
|
||||
|
@ -240,7 +270,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
if (random().nextBoolean()) {
|
||||
// simple iterator
|
||||
sets[i] = set;
|
||||
iterators[i] = new ConstantScoreScorer(null, 0f, new BitDocIdSet(set).iterator());
|
||||
iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator());
|
||||
} else {
|
||||
// scorer with approximation
|
||||
final FixedBitSet confirmed = clearRandomBits(set);
|
||||
|
@ -276,12 +306,12 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
case 0:
|
||||
// simple iterator
|
||||
sets[i] = set;
|
||||
newIterator = new ConstantScoreScorer(null, 0f, anonymizeIterator(new BitDocIdSet(set).iterator()));
|
||||
newIterator = new ConstantScoreScorer(new FakeWeight(), 0f, anonymizeIterator(new BitDocIdSet(set).iterator()));
|
||||
break;
|
||||
case 1:
|
||||
// bitSet iterator
|
||||
sets[i] = set;
|
||||
newIterator = new ConstantScoreScorer(null, 0f, new BitDocIdSet(set).iterator());
|
||||
newIterator = new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator());
|
||||
break;
|
||||
default:
|
||||
// scorer with approximation
|
||||
|
@ -322,7 +352,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
if (random().nextBoolean()) {
|
||||
// simple iterator
|
||||
sets[i] = set;
|
||||
scorers.add(new ConstantScoreScorer(null, 0f, new BitDocIdSet(set).iterator()));
|
||||
scorers.add(new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator()));
|
||||
} else {
|
||||
// scorer with approximation
|
||||
final FixedBitSet confirmed = clearRandomBits(set);
|
||||
|
@ -340,9 +370,9 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
List<Scorer> subIterators = scorers.subList(subSeqStart, subSeqEnd);
|
||||
Scorer subConjunction;
|
||||
if (wrapWithScorer) {
|
||||
subConjunction = new ConjunctionScorer(null, subIterators, Collections.emptyList());
|
||||
subConjunction = new ConjunctionScorer(new FakeWeight(), subIterators, Collections.emptyList());
|
||||
} else {
|
||||
subConjunction = new ConstantScoreScorer(null, 0f, ConjunctionDISI.intersectScorers(subIterators));
|
||||
subConjunction = new ConstantScoreScorer(new FakeWeight(), 0f, ConjunctionDISI.intersectScorers(subIterators));
|
||||
}
|
||||
scorers.set(subSeqStart, subConjunction);
|
||||
int toRemove = subSeqEnd - subSeqStart - 1;
|
||||
|
@ -352,7 +382,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
|
|||
}
|
||||
if (scorers.size() == 1) {
|
||||
// ConjunctionDISI needs two iterators
|
||||
scorers.add(new ConstantScoreScorer(null, 0f, DocIdSetIterator.all(maxDoc)));
|
||||
scorers.add(new ConstantScoreScorer(new FakeWeight(), 0f, DocIdSetIterator.all(maxDoc)));
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -54,7 +54,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
QueryUtils.checkUnequal(q1, new TermQuery(new Term("a", "b")));
|
||||
}
|
||||
|
||||
private void checkHits(IndexSearcher searcher, Query q, final float expectedScore, final Class<? extends Scorer> innerScorerClass) throws IOException {
|
||||
private void checkHits(IndexSearcher searcher, Query q, final float expectedScore, final Class<? extends Scorable> innerScorerClass) throws IOException {
|
||||
final int[] count = new int[1];
|
||||
searcher.search(q, new SimpleCollector() {
|
||||
private Scorable scorer;
|
||||
|
@ -131,7 +131,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
checkHits(searcher, csq2, csq2.getBoost(), TermScorer.class);
|
||||
|
||||
// for the combined BQ, the scorer should always be BooleanScorer's BucketScorer, because our scorer supports out-of order collection!
|
||||
final Class<FakeScorer> bucketScorerClass = FakeScorer.class;
|
||||
final Class<ScoreAndDoc> bucketScorerClass = ScoreAndDoc.class;
|
||||
checkHits(searcher, csqbq, csqbq.getBoost(), bucketScorerClass);
|
||||
} finally {
|
||||
IOUtils.close(reader, directory);
|
||||
|
|
|
@ -22,7 +22,10 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
|
@ -30,13 +33,40 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
|||
|
||||
public class TestMaxScoreSumPropagator extends LuceneTestCase {
|
||||
|
||||
private static class FakeWeight extends Weight {
|
||||
|
||||
FakeWeight() {
|
||||
super(new MatchNoDocsQuery());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable(LeafReaderContext ctx) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private static class FakeScorer extends Scorer {
|
||||
|
||||
final float maxScore;
|
||||
float minCompetitiveScore;
|
||||
|
||||
FakeScorer(float maxScore) {
|
||||
super(null);
|
||||
FakeScorer(float maxScore) throws IOException {
|
||||
super(new FakeWeight());
|
||||
this.maxScore = maxScore;
|
||||
}
|
||||
|
||||
|
|
|
@ -134,7 +134,7 @@ public class TestMultiCollector extends LuceneTestCase {
|
|||
collector1 = new TerminateAfterCollector(collector1, 1);
|
||||
collector2 = new TerminateAfterCollector(collector2, 2);
|
||||
|
||||
Scorer scorer = new FakeScorer();
|
||||
Scorable scorer = new ScoreAndDoc();
|
||||
|
||||
List<Collector> collectors = Arrays.asList(collector1, collector2);
|
||||
Collections.shuffle(collectors, random());
|
||||
|
@ -172,27 +172,17 @@ public class TestMultiCollector extends LuceneTestCase {
|
|||
IndexReader reader = DirectoryReader.open(w);
|
||||
w.close();
|
||||
|
||||
Scorer scorer = new Scorer(null) {
|
||||
Scorable scorer = new Scorable() {
|
||||
@Override
|
||||
public int docID() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
public float score() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMinCompetitiveScore(float minScore) {
|
||||
throw new AssertionError();
|
||||
|
|
|
@ -429,7 +429,7 @@ public class TestQueryRescorer extends LuceneTestCase {
|
|||
@Override
|
||||
public Scorer scorer(final LeafReaderContext context) throws IOException {
|
||||
|
||||
return new Scorer(null) {
|
||||
return new Scorer(this) {
|
||||
int docID = -1;
|
||||
|
||||
@Override
|
||||
|
|
|
@ -192,15 +192,11 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private static class FakeScorer extends Scorer {
|
||||
private static class ScoreAndDoc extends Scorable {
|
||||
int doc = -1;
|
||||
float score;
|
||||
Float minCompetitiveScore = null;
|
||||
|
||||
FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMinCompetitiveScore(float minCompetitiveScore) {
|
||||
this.minCompetitiveScore = minCompetitiveScore;
|
||||
|
@ -215,16 +211,6 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
public float score() throws IOException {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
public void testSetMinCompetitiveScore() throws Exception {
|
||||
|
@ -240,7 +226,7 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
w.close();
|
||||
|
||||
TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
|
||||
LeafCollector leafCollector = collector.getLeafCollector(reader.leaves().get(0));
|
||||
leafCollector.setScorer(scorer);
|
||||
|
@ -269,7 +255,7 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f);
|
||||
|
||||
// Make sure the min score is set on scorers on new segments
|
||||
scorer = new FakeScorer();
|
||||
scorer = new ScoreAndDoc();
|
||||
leafCollector = collector.getLeafCollector(reader.leaves().get(1));
|
||||
leafCollector.setScorer(scorer);
|
||||
assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f);
|
||||
|
@ -302,7 +288,7 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
|
||||
for (int totalHitsThreshold = 1; totalHitsThreshold < 20; ++ totalHitsThreshold) {
|
||||
TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, totalHitsThreshold);
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
|
||||
LeafCollector leafCollector = collector.getLeafCollector(reader.leaves().get(0));
|
||||
leafCollector.setScorer(scorer);
|
||||
|
|
|
@ -149,7 +149,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
for (int totalHitsThreshold = 1; totalHitsThreshold < 20; ++ totalHitsThreshold) {
|
||||
for (FieldDoc after : new FieldDoc[] { null, new FieldDoc(4, Float.NaN, new Object[] { 2L })}) {
|
||||
TopFieldCollector collector = TopFieldCollector.create(sort, 2, after, totalHitsThreshold);
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
|
||||
LeafCollector leafCollector1 = collector.getLeafCollector(reader.leaves().get(0));
|
||||
leafCollector1.setScorer(scorer);
|
||||
|
|
|
@ -1,53 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.expressions;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
|
||||
class FakeScorer extends Scorer {
|
||||
|
||||
float score;
|
||||
int doc = -1;
|
||||
|
||||
FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
}
|
|
@ -26,9 +26,9 @@ import org.apache.lucene.search.BulkScorer;
|
|||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.Scorable;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.TwoPhaseIterator;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
||||
|
@ -82,7 +82,7 @@ class DrillSidewaysScorer extends BulkScorer {
|
|||
// System.out.println("\nscore: reader=" + context.reader());
|
||||
//}
|
||||
//System.out.println("score r=" + context.reader());
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
collector.setScorer(scorer);
|
||||
if (drillDownCollector != null) {
|
||||
drillDownLeafCollector = drillDownCollector.getLeafCollector(context);
|
||||
|
@ -580,41 +580,23 @@ class DrillSidewaysScorer extends BulkScorer {
|
|||
sidewaysCollector.collect(collectDocID);
|
||||
}
|
||||
|
||||
private final class FakeScorer extends Scorer {
|
||||
|
||||
public FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
private final class ScoreAndDoc extends Scorable {
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return collectDocID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
return collectScore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorable> getChildren() {
|
||||
return Collections.singletonList(new ChildScorable(baseScorer, "MUST"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight getWeight() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
static class DocsAndCost {
|
||||
|
|
|
@ -286,7 +286,7 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
}
|
||||
int totalGroupedHitCount = 0;
|
||||
|
||||
final FakeScorer fakeScorer = new FakeScorer();
|
||||
final ScoreAndDoc fakeScorer = new ScoreAndDoc();
|
||||
|
||||
float maxScore = Float.MIN_VALUE;
|
||||
|
||||
|
@ -494,4 +494,21 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
public ScoreMode scoreMode() {
|
||||
return needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
|
||||
}
|
||||
|
||||
private static class ScoreAndDoc extends Scorable {
|
||||
|
||||
float score;
|
||||
int doc = -1;
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
return score;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,52 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search.grouping;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
|
||||
class FakeScorer extends Scorer {
|
||||
|
||||
float score;
|
||||
int doc = -1;
|
||||
|
||||
FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
}
|
|
@ -34,7 +34,7 @@ import org.apache.lucene.search.Weight;
|
|||
* range. The score is the float value. This can be a slow query if run by itself since it must visit all docs;
|
||||
* ideally it's combined with other queries.
|
||||
* It's mostly a wrapper around
|
||||
* {@link FunctionValues#getRangeScorer(LeafReaderContext, String, String, boolean, boolean)}.
|
||||
* {@link FunctionValues#getRangeScorer(Weight, LeafReaderContext, String, String, boolean, boolean)}.
|
||||
*
|
||||
* A similar class is {@code org.apache.lucene.search.DocValuesRangeQuery} in the sandbox module. That one is
|
||||
* constant scoring.
|
||||
|
@ -152,7 +152,7 @@ public class FunctionRangeQuery extends Query {
|
|||
public ValueSourceScorer scorer(LeafReaderContext context) throws IOException {
|
||||
FunctionValues functionValues = valueSource.getValues(vsContext, context);
|
||||
// getRangeScorer takes String args and parses them. Weird.
|
||||
return functionValues.getRangeScorer(context, lowerVal, upperVal, includeLower, includeUpper);
|
||||
return functionValues.getRangeScorer(this, context, lowerVal, upperVal, includeLower, includeUpper);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueFloat;
|
||||
|
@ -144,8 +145,8 @@ public abstract class FunctionValues {
|
|||
* Yields a {@link Scorer} that matches all documents,
|
||||
* and that which produces scores equal to {@link #floatVal(int)}.
|
||||
*/
|
||||
public ValueSourceScorer getScorer(LeafReaderContext readerContext) {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
public ValueSourceScorer getScorer(Weight weight, LeafReaderContext readerContext) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) {
|
||||
return true;
|
||||
|
@ -161,7 +162,7 @@ public abstract class FunctionValues {
|
|||
// because it needs different behavior depending on the type of fields. There is also
|
||||
// a setup cost - parsing and normalizing params, and doing a binary search on the StringIndex.
|
||||
// TODO: change "reader" to LeafReaderContext
|
||||
public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) throws IOException {
|
||||
public ValueSourceScorer getRangeScorer(Weight weight, LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) throws IOException {
|
||||
float lower;
|
||||
float upper;
|
||||
|
||||
|
@ -180,7 +181,7 @@ public abstract class FunctionValues {
|
|||
final float u = upper;
|
||||
|
||||
if (includeLower && includeUpper) {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
@ -190,7 +191,7 @@ public abstract class FunctionValues {
|
|||
};
|
||||
}
|
||||
else if (includeLower && !includeUpper) {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
@ -200,7 +201,7 @@ public abstract class FunctionValues {
|
|||
};
|
||||
}
|
||||
else if (!includeLower && includeUpper) {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
@ -210,7 +211,7 @@ public abstract class FunctionValues {
|
|||
};
|
||||
}
|
||||
else {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.Map;
|
|||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.DoubleValues;
|
||||
import org.apache.lucene.search.DoubleValuesSource;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
|
@ -32,6 +31,7 @@ import org.apache.lucene.search.FieldComparatorSource;
|
|||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.LongValues;
|
||||
import org.apache.lucene.search.LongValuesSource;
|
||||
import org.apache.lucene.search.Scorable;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.SimpleFieldComparator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
@ -87,34 +87,20 @@ public abstract class ValueSource {
|
|||
return context;
|
||||
}
|
||||
|
||||
private static class FakeScorer extends Scorer {
|
||||
private static class ScoreAndDoc extends Scorable {
|
||||
|
||||
int current = -1;
|
||||
float score = 0;
|
||||
|
||||
FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return current;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
public float score() {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -135,7 +121,7 @@ public abstract class ValueSource {
|
|||
@Override
|
||||
public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
Map context = new IdentityHashMap<>();
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
context.put("scorer", scorer);
|
||||
final FunctionValues fv = in.getValues(context, ctx);
|
||||
return new LongValues() {
|
||||
|
@ -211,7 +197,7 @@ public abstract class ValueSource {
|
|||
@Override
|
||||
public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
Map context = new HashMap<>();
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
context.put("scorer", scorer);
|
||||
context.put("searcher", searcher);
|
||||
FunctionValues fv = in.getValues(context, ctx);
|
||||
|
@ -248,7 +234,7 @@ public abstract class ValueSource {
|
|||
@Override
|
||||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) throws IOException {
|
||||
Map context = new HashMap<>();
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
scorer.score = scoreExplanation.getValue().floatValue();
|
||||
context.put("scorer", scorer);
|
||||
context.put("searcher", searcher);
|
||||
|
|
|
@ -22,13 +22,13 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.TwoPhaseIterator;
|
||||
import org.apache.lucene.search.Weight;
|
||||
|
||||
/**
|
||||
* {@link Scorer} which returns the result of {@link FunctionValues#floatVal(int)} as
|
||||
* the score for a document, and which filters out documents that don't match {@link #matches(int)}.
|
||||
* This Scorer has a {@link TwoPhaseIterator}. This is similar to {@link FunctionQuery},
|
||||
* but this one has no {@link org.apache.lucene.search.Weight} normalization factors/multipliers
|
||||
* and that one doesn't filter either.
|
||||
* with an added filter.
|
||||
* <p>
|
||||
* Note: If the scores are needed, then the underlying value will probably be
|
||||
* fetched/computed twice -- once to filter and next to return the score. If that's non-trivial then
|
||||
|
@ -43,8 +43,8 @@ public abstract class ValueSourceScorer extends Scorer {
|
|||
private final TwoPhaseIterator twoPhaseIterator;
|
||||
private final DocIdSetIterator disi;
|
||||
|
||||
protected ValueSourceScorer(LeafReaderContext readerContext, FunctionValues values) {
|
||||
super(null);//no weight
|
||||
protected ValueSourceScorer(Weight weight, LeafReaderContext readerContext, FunctionValues values) {
|
||||
super(weight);
|
||||
this.values = values;
|
||||
final DocIdSetIterator approximation = DocIdSetIterator.all(readerContext.reader().maxDoc()); // no approximation!
|
||||
this.twoPhaseIterator = new TwoPhaseIterator(approximation) {
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.index.SortedDocValues;
|
|||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.ValueSourceScorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
|
@ -115,7 +116,7 @@ public abstract class DocTermsIndexDocValues extends FunctionValues {
|
|||
public abstract Object objectVal(int doc) throws IOException; // force subclasses to override
|
||||
|
||||
@Override
|
||||
public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) throws IOException {
|
||||
public ValueSourceScorer getRangeScorer(Weight weight, LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) throws IOException {
|
||||
// TODO: are lowerVal and upperVal in indexed form or not?
|
||||
lowerVal = lowerVal == null ? null : toTerm(lowerVal);
|
||||
upperVal = upperVal == null ? null : toTerm(upperVal);
|
||||
|
@ -143,7 +144,7 @@ public abstract class DocTermsIndexDocValues extends FunctionValues {
|
|||
final int ll = lower;
|
||||
final int uu = upper;
|
||||
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
final SortedDocValues values = readerContext.reader().getSortedDocValues(field);
|
||||
private int lastDocID;
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.ValueSourceScorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueDouble;
|
||||
|
||||
|
@ -85,7 +86,7 @@ public abstract class DoubleDocValues extends FunctionValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
public ValueSourceScorer getRangeScorer(Weight weight, LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
double lower,upper;
|
||||
|
||||
if (lowerVal==null) {
|
||||
|
@ -105,7 +106,7 @@ public abstract class DoubleDocValues extends FunctionValues {
|
|||
|
||||
|
||||
if (includeLower && includeUpper) {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
@ -115,7 +116,7 @@ public abstract class DoubleDocValues extends FunctionValues {
|
|||
};
|
||||
}
|
||||
else if (includeLower && !includeUpper) {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
@ -125,7 +126,7 @@ public abstract class DoubleDocValues extends FunctionValues {
|
|||
};
|
||||
}
|
||||
else if (!includeLower && includeUpper) {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
@ -135,7 +136,7 @@ public abstract class DoubleDocValues extends FunctionValues {
|
|||
};
|
||||
}
|
||||
else {
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.ValueSourceScorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueInt;
|
||||
|
||||
|
@ -80,7 +81,7 @@ public abstract class IntDocValues extends FunctionValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
public ValueSourceScorer getRangeScorer(Weight weight, LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
int lower,upper;
|
||||
|
||||
// instead of using separate comparison functions, adjust the endpoints.
|
||||
|
@ -102,7 +103,7 @@ public abstract class IntDocValues extends FunctionValues {
|
|||
final int ll = lower;
|
||||
final int uu = upper;
|
||||
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.ValueSourceScorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueLong;
|
||||
|
||||
|
@ -89,7 +90,7 @@ public abstract class LongDocValues extends FunctionValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
public ValueSourceScorer getRangeScorer(Weight weight, LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
long lower,upper;
|
||||
|
||||
// instead of using separate comparison functions, adjust the endpoints.
|
||||
|
@ -111,7 +112,7 @@ public abstract class LongDocValues extends FunctionValues {
|
|||
final long ll = lower;
|
||||
final long uu = upper;
|
||||
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.NumericDocValues;
|
|||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSourceScorer;
|
||||
import org.apache.lucene.queries.function.docvalues.IntDocValues;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueInt;
|
||||
|
||||
|
@ -135,7 +136,7 @@ public class EnumFieldSource extends FieldCacheSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
public ValueSourceScorer getRangeScorer(Weight weight, LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
Integer lower = stringValueToIntValue(lowerVal);
|
||||
Integer upper = stringValueToIntValue(upperVal);
|
||||
|
||||
|
@ -156,7 +157,7 @@ public class EnumFieldSource extends FieldCacheSource {
|
|||
final int ll = lower;
|
||||
final int uu = upper;
|
||||
|
||||
return new ValueSourceScorer(readerContext, this) {
|
||||
return new ValueSourceScorer(weight, readerContext, this) {
|
||||
@Override
|
||||
public boolean matches(int doc) throws IOException {
|
||||
if (!exists(doc)) return false;
|
||||
|
|
|
@ -35,15 +35,13 @@ import org.apache.lucene.index.IndexReaderContext;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.MatchNoDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Scorable;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.search.grouping.GroupDocs;
|
||||
import org.apache.lucene.search.grouping.SearchGroup;
|
||||
import org.apache.lucene.search.grouping.TopGroups;
|
||||
|
@ -469,7 +467,7 @@ public class QueryComponent extends SearchComponent
|
|||
}
|
||||
|
||||
doc -= currentLeaf.docBase; // adjust for what segment this is in
|
||||
leafComparator.setScorer(new FakeScorer(doc, score));
|
||||
leafComparator.setScorer(new ScoreAndDoc(doc, score));
|
||||
leafComparator.copy(0, doc);
|
||||
Object val = comparator.value(0);
|
||||
if (null != ft) val = ft.marshalSortValue(val);
|
||||
|
@ -1461,12 +1459,11 @@ public class QueryComponent extends SearchComponent
|
|||
*
|
||||
* TODO: when SOLR-5595 is fixed, this wont be needed, as we dont need to recompute sort values here from the comparator
|
||||
*/
|
||||
protected static class FakeScorer extends Scorer {
|
||||
protected static class ScoreAndDoc extends Scorable {
|
||||
final int docid;
|
||||
final float score;
|
||||
|
||||
FakeScorer(int docid, float score) {
|
||||
super(null);
|
||||
ScoreAndDoc(int docid, float score) {
|
||||
this.docid = docid;
|
||||
this.score = score;
|
||||
}
|
||||
|
@ -1480,25 +1477,5 @@ public class QueryComponent extends SearchComponent
|
|||
public float score() throws IOException {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight getWeight() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorable> getChildren() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,7 +56,6 @@ import org.apache.lucene.search.LeafFieldComparator;
|
|||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorable;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
|
@ -434,36 +433,18 @@ public class CollapsingQParserPlugin extends QParserPlugin {
|
|||
}
|
||||
|
||||
|
||||
private static class DummyScorer extends Scorer {
|
||||
private static class ScoreAndDoc extends Scorable {
|
||||
|
||||
public float score;
|
||||
public int docId;
|
||||
|
||||
public DummyScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
public float score() {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return Float.POSITIVE_INFINITY;
|
||||
}
|
||||
|
||||
public int freq() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
public int docID() {
|
||||
return docId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -647,7 +628,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
|
|||
|
||||
int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
|
||||
leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
|
||||
DummyScorer dummy = new DummyScorer();
|
||||
ScoreAndDoc dummy = new ScoreAndDoc();
|
||||
leafDelegate.setScorer(dummy);
|
||||
DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
|
||||
int docId = -1;
|
||||
|
@ -850,7 +831,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
|
|||
collapseValues = DocValues.getNumeric(contexts[currentContext].reader(), this.field);
|
||||
int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
|
||||
leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
|
||||
DummyScorer dummy = new DummyScorer();
|
||||
ScoreAndDoc dummy = new ScoreAndDoc();
|
||||
leafDelegate.setScorer(dummy);
|
||||
DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
|
||||
int globalDoc = -1;
|
||||
|
@ -1022,7 +1003,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
|
|||
|
||||
int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
|
||||
leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
|
||||
DummyScorer dummy = new DummyScorer();
|
||||
ScoreAndDoc dummy = new ScoreAndDoc();
|
||||
leafDelegate.setScorer(dummy);
|
||||
DocIdSetIterator it = new BitSetIterator(collapseStrategy.getCollapsedSet(), 0); // cost is not useful here
|
||||
int globalDoc = -1;
|
||||
|
@ -1181,7 +1162,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
|
|||
this.collapseValues = DocValues.getNumeric(contexts[currentContext].reader(), this.collapseField);
|
||||
int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
|
||||
leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
|
||||
DummyScorer dummy = new DummyScorer();
|
||||
ScoreAndDoc dummy = new ScoreAndDoc();
|
||||
leafDelegate.setScorer(dummy);
|
||||
DocIdSetIterator it = new BitSetIterator(collapseStrategy.getCollapsedSet(), 0); // cost is not useful here
|
||||
int globalDoc = -1;
|
||||
|
|
|
@ -89,7 +89,7 @@ public abstract class Filter extends Query {
|
|||
//
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
|
||||
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) {
|
||||
return new Weight(this) {
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,10 +24,13 @@ import org.apache.lucene.queries.function.FunctionValues;
|
|||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.ValueSourceScorer;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.solr.search.function.ValueSourceRangeFilter;
|
||||
|
||||
// This class works as either a normal constant score query, or as a PostFilter using a collector
|
||||
public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFilter {
|
||||
|
||||
final ValueSourceRangeFilter rangeFilt;
|
||||
|
||||
public FunctionRangeQuery(ValueSourceRangeFilter filter) {
|
||||
|
@ -39,16 +42,19 @@ public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFi
|
|||
@Override
|
||||
public DelegatingCollector getFilterCollector(IndexSearcher searcher) {
|
||||
Map fcontext = ValueSource.newContext(searcher);
|
||||
return new FunctionRangeCollector(fcontext);
|
||||
Weight weight = rangeFilt.createWeight(searcher, ScoreMode.COMPLETE, 1);
|
||||
return new FunctionRangeCollector(fcontext, weight);
|
||||
}
|
||||
|
||||
class FunctionRangeCollector extends DelegatingCollector {
|
||||
final Map fcontext;
|
||||
final Weight weight;
|
||||
ValueSourceScorer scorer;
|
||||
int maxdoc;
|
||||
|
||||
public FunctionRangeCollector(Map fcontext) {
|
||||
public FunctionRangeCollector(Map fcontext, Weight weight) {
|
||||
this.fcontext = fcontext;
|
||||
this.weight = weight;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -64,7 +70,7 @@ public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFi
|
|||
super.doSetNextReader(context);
|
||||
maxdoc = context.reader().maxDoc();
|
||||
FunctionValues dv = rangeFilt.getValueSource().getValues(fcontext, context);
|
||||
scorer = dv.getRangeScorer(context, rangeFilt.getLowerVal(), rangeFilt.getUpperVal(), rangeFilt.isIncludeLower(), rangeFilt.isIncludeUpper());
|
||||
scorer = dv.getRangeScorer(weight, context, rangeFilt.getLowerVal(), rangeFilt.getUpperVal(), rangeFilt.isIncludeLower(), rangeFilt.isIncludeUpper());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,9 @@ import org.apache.lucene.queries.function.ValueSource;
|
|||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.solr.search.BitsFilteredDocIdSet;
|
||||
import org.apache.solr.search.SolrFilter;
|
||||
|
@ -77,10 +79,13 @@ public class ValueSourceRangeFilter extends SolrFilter {
|
|||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(final Map context, final LeafReaderContext readerContext, Bits acceptDocs) throws IOException {
|
||||
return BitsFilteredDocIdSet.wrap(new DocIdSet() {
|
||||
// NB the IndexSearcher parameter here can be null because Filter Weights don't
|
||||
// actually use it.
|
||||
Weight weight = createWeight(null, ScoreMode.COMPLETE, 1);
|
||||
return BitsFilteredDocIdSet.wrap(new DocIdSet() {
|
||||
@Override
|
||||
public DocIdSetIterator iterator() throws IOException {
|
||||
Scorer scorer = valueSource.getValues(context, readerContext).getRangeScorer(readerContext, lowerVal, upperVal, includeLower, includeUpper);
|
||||
Scorer scorer = valueSource.getValues(context, readerContext).getRangeScorer(weight, readerContext, lowerVal, upperVal, includeLower, includeUpper);
|
||||
return scorer == null ? null : scorer.iterator();
|
||||
}
|
||||
@Override
|
||||
|
|
|
@ -20,7 +20,6 @@ import java.io.IOException;
|
|||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
|
@ -32,7 +31,6 @@ import org.apache.lucene.index.IndexReaderContext;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
|
@ -41,7 +39,6 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.Scorable;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
|
@ -424,7 +421,7 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
}
|
||||
|
||||
doc -= currentLeaf.docBase; // adjust for what segment this is in
|
||||
leafComparator.setScorer(new FakeScorer(doc, score));
|
||||
leafComparator.setScorer(new ScoreAndDoc(doc, score));
|
||||
leafComparator.copy(0, doc);
|
||||
Object val = comparator.value(0);
|
||||
if (null != ft) val = ft.marshalSortValue(val);
|
||||
|
@ -438,13 +435,12 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
}
|
||||
}
|
||||
|
||||
private static class FakeScorer extends Scorer {
|
||||
private static class ScoreAndDoc extends Scorable {
|
||||
|
||||
final int docid;
|
||||
final float score;
|
||||
|
||||
FakeScorer(int docid, float score) {
|
||||
super(null);
|
||||
ScoreAndDoc(int docid, float score) {
|
||||
this.docid = docid;
|
||||
this.score = score;
|
||||
}
|
||||
|
@ -455,29 +451,9 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
public float score() {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getMaxScore(int upTo) throws IOException {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight getWeight() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorable> getChildren() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
public void merge(ResponseBuilder rb, ShardRequest sreq) {
|
||||
|
|
Loading…
Reference in New Issue