Upgrade to lucene-5.1.0-snapshot-1652032.
This new Lucene snapshot does not have out-of-order scoring anymore. Close #9318
This commit is contained in:
parent
95bc7df289
commit
1fc24a8837
6
pom.xml
6
pom.xml
|
@ -31,8 +31,8 @@
|
|||
</parent>
|
||||
|
||||
<properties>
|
||||
<lucene.version>5.0.0</lucene.version>
|
||||
<lucene.maven.version>5.0.0-snapshot-1650327</lucene.maven.version>
|
||||
<lucene.version>5.1.0</lucene.version>
|
||||
<lucene.maven.version>5.1.0-snapshot-1652032</lucene.maven.version>
|
||||
<tests.jvms>auto</tests.jvms>
|
||||
<tests.shuffle>true</tests.shuffle>
|
||||
<tests.output>onerror</tests.output>
|
||||
|
@ -54,7 +54,7 @@
|
|||
</repository>
|
||||
<repository>
|
||||
<id>Lucene snapshots</id>
|
||||
<url>https://download.elasticsearch.org/lucenesnapshots/1650327</url>
|
||||
<url>https://download.elasticsearch.org/lucenesnapshots/1652032</url>
|
||||
</repository>
|
||||
</repositories>
|
||||
|
||||
|
|
|
@ -223,7 +223,7 @@ public class Version {
|
|||
public static final int V_1_5_0_ID = 1050099;
|
||||
public static final Version V_1_5_0 = new Version(V_1_5_0_ID, true, org.apache.lucene.util.Version.LUCENE_4_10_3);
|
||||
public static final int V_2_0_0_ID = 2000099;
|
||||
public static final Version V_2_0_0 = new Version(V_2_0_0_ID, true, org.apache.lucene.util.Version.LUCENE_5_0_0);
|
||||
public static final Version V_2_0_0 = new Version(V_2_0_0_ID, true, org.apache.lucene.util.Version.LUCENE_5_1_0);
|
||||
|
||||
public static final Version CURRENT = V_2_0_0;
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.common.lucene;
|
|||
|
||||
import org.apache.lucene.analysis.core.KeywordAnalyzer;
|
||||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
|
@ -537,11 +536,6 @@ public class Lucene {
|
|||
public void doSetNextReader(LeafReaderContext atomicReaderContext) throws IOException {
|
||||
leafCollector = delegate.getLeafCollector(atomicReaderContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return leafCollector.acceptsDocsOutOfOrder();
|
||||
}
|
||||
}
|
||||
|
||||
private Lucene() {
|
||||
|
|
|
@ -60,9 +60,4 @@ public class MinimumScoreCollector extends SimpleCollector {
|
|||
public void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
leafCollector = collector.getLeafCollector(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return leafCollector.acceptsDocsOutOfOrder();
|
||||
}
|
||||
}
|
|
@ -71,19 +71,6 @@ public class MultiCollector extends SimpleCollector implements XCollector {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
if (!leafCollector.acceptsDocsOutOfOrder()) {
|
||||
return false;
|
||||
}
|
||||
for (LeafCollector leafCollector : leafCollectors) {
|
||||
if (!leafCollector.acceptsDocsOutOfOrder()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postCollection() throws IOException {
|
||||
if (collector instanceof XCollector) {
|
||||
|
|
|
@ -77,10 +77,6 @@ public class FilteredCollector implements XCollector {
|
|||
public void collect(int doc) throws IOException {
|
||||
// no-op
|
||||
}
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -98,12 +94,6 @@ public class FilteredCollector implements XCollector {
|
|||
in.collect(doc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
// we only support iterating in order because the iterator can only advance
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -43,9 +43,4 @@ public class NoopCollector extends SimpleCollector {
|
|||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -96,9 +96,4 @@ final class QueriesLoaderCollector extends SimpleCollector {
|
|||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -131,11 +131,6 @@ public class IncludeNestedDocsQuery extends Query {
|
|||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
return null; //Query is used internally and not by users, so explain can be empty
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean scoresDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
static class IncludeNestedDocsScorer extends Scorer {
|
||||
|
|
|
@ -120,12 +120,6 @@ abstract class QueryCollector extends SimpleCollector {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
static Match match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
|
||||
return new Match(logger, context, highlightPhase, isNestedDoc);
|
||||
}
|
||||
|
@ -231,7 +225,7 @@ abstract class QueryCollector extends SimpleCollector {
|
|||
MatchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
|
||||
super(logger, context, isNestedDoc);
|
||||
// TODO: Use TopFieldCollector.create(...) for ascending and descending scoring?
|
||||
topDocsCollector = TopScoreDocCollector.create(context.size(), false);
|
||||
topDocsCollector = TopScoreDocCollector.create(context.size());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.SimpleCollector;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.lucene.search.XCollector;
|
||||
|
@ -179,11 +178,6 @@ public class AggregationPhase implements SearchPhase {
|
|||
aggregationContext.setNextReader(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return !aggregationContext.scoreDocsInOrder();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postCollection() throws IOException {
|
||||
for (Aggregator collector : collectors) {
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedLongValues;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -105,11 +104,6 @@ public class RecordingPerReaderBucketCollector extends RecordingBucketCollector
|
|||
}
|
||||
}
|
||||
|
||||
public RecordingPerReaderBucketCollector(AggregationContext context) {
|
||||
// Call this method to achieve better compression in the recorded arrays of matches
|
||||
context.ensureScoreDocsInOrder();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNextReader(LeafReaderContext reader) {
|
||||
if(recordingComplete){
|
||||
|
|
|
@ -51,7 +51,7 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa
|
|||
|
||||
public DeferringBucketCollector(BucketCollector deferred, AggregationContext context) {
|
||||
this.deferred = deferred;
|
||||
this.recording = new RecordingPerReaderBucketCollector(context);
|
||||
this.recording = new RecordingPerReaderBucketCollector();
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
|
|
|
@ -55,8 +55,6 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
|
|||
super(name, factories, aggregationContext, parentAggregator, metaData);
|
||||
this.parentAggregator = parentAggregator;
|
||||
childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy);
|
||||
// The childDocs need to be consumed in docId order, this ensures that:
|
||||
aggregationContext.ensureScoreDocsInOrder();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -57,7 +57,6 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
|
|||
parentFilter = SearchContext.current().bitsetFilterCache().getBitDocIdSetFilter(objectMapper.nestedTypeFilter());
|
||||
}
|
||||
bucketOrdToLastCollectedParentDoc = new LongIntOpenHashMap(32);
|
||||
aggregationContext.ensureScoreDocsInOrder();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -117,7 +117,7 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
|
|||
if (collectors == null) {
|
||||
Sort sort = subSearchContext.sort();
|
||||
int topN = subSearchContext.from() + subSearchContext.size();
|
||||
TopDocsCollector<?> topLevelCollector = sort != null ? TopFieldCollector.create(sort, topN, true, subSearchContext.trackScores(), subSearchContext.trackScores(), false) : TopScoreDocCollector.create(topN, false);
|
||||
TopDocsCollector<?> topLevelCollector = sort != null ? TopFieldCollector.create(sort, topN, true, subSearchContext.trackScores(), subSearchContext.trackScores()) : TopScoreDocCollector.create(topN);
|
||||
collectors = new TopDocsAndLeafCollector(topLevelCollector);
|
||||
collectors.leafCollector = collectors.topLevelCollector.getLeafCollector(currentContext);
|
||||
collectors.leafCollector.setScorer(currentScorer);
|
||||
|
|
|
@ -58,7 +58,6 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
|
|||
|
||||
private LeafReaderContext reader;
|
||||
private Scorer scorer;
|
||||
private boolean scoreDocsInOrder = false;
|
||||
|
||||
public AggregationContext(SearchContext searchContext) {
|
||||
this.searchContext = searchContext;
|
||||
|
@ -104,14 +103,6 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
|
|||
}
|
||||
}
|
||||
|
||||
public boolean scoreDocsInOrder() {
|
||||
return scoreDocsInOrder;
|
||||
}
|
||||
|
||||
public void ensureScoreDocsInOrder() {
|
||||
this.scoreDocsInOrder = true;
|
||||
}
|
||||
|
||||
/** Get a value source given its configuration and the depth of the aggregator in the aggregation tree. */
|
||||
public <VS extends ValuesSource> VS valuesSource(ValuesSourceConfig<VS> config, int depth) throws IOException {
|
||||
assert config.valid() : "value source config is invalid - must have either a field context or a script or marked as unmapped";
|
||||
|
|
|
@ -114,12 +114,12 @@ public final class InnerHitsContext {
|
|||
int topN = from() + size();
|
||||
if (sort() != null) {
|
||||
try {
|
||||
topDocsCollector = TopFieldCollector.create(sort(), topN, true, trackScores(), trackScores(), true);
|
||||
topDocsCollector = TopFieldCollector.create(sort(), topN, true, trackScores(), trackScores());
|
||||
} catch (IOException e) {
|
||||
throw ExceptionsHelper.convertToElastic(e);
|
||||
}
|
||||
} else {
|
||||
topDocsCollector = TopScoreDocCollector.create(topN, true);
|
||||
topDocsCollector = TopScoreDocCollector.create(topN);
|
||||
}
|
||||
|
||||
Filter rawParentFilter;
|
||||
|
@ -249,12 +249,12 @@ public final class InnerHitsContext {
|
|||
int topN = from() + size();
|
||||
if (sort() != null) {
|
||||
try {
|
||||
topDocsCollector = TopFieldCollector.create(sort(), topN, true, trackScores(), trackScores(), false);
|
||||
topDocsCollector = TopFieldCollector.create(sort(), topN, true, trackScores(), trackScores());
|
||||
} catch (IOException e) {
|
||||
throw ExceptionsHelper.convertToElastic(e);
|
||||
}
|
||||
} else {
|
||||
topDocsCollector = TopScoreDocCollector.create(topN, false);
|
||||
topDocsCollector = TopScoreDocCollector.create(topN);
|
||||
}
|
||||
|
||||
String field;
|
||||
|
|
|
@ -132,11 +132,6 @@ public class ScanContext {
|
|||
this.readerState = new ReaderState();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return false;
|
||||
}
|
||||
|
||||
public static final RuntimeException StopCollectingException = new StopCollectingException();
|
||||
|
||||
static class StopCollectingException extends RuntimeException {
|
||||
|
|
|
@ -22,6 +22,7 @@ import com.carrotsearch.hppc.FloatArrayList;
|
|||
import com.carrotsearch.hppc.IntOpenHashSet;
|
||||
import com.carrotsearch.hppc.ObjectObjectOpenHashMap;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoubleField;
|
||||
|
@ -216,17 +217,15 @@ public class ChildrenQueryTests extends AbstractChildTests {
|
|||
Query query = parseQuery(queryBuilder);
|
||||
BitSetCollector collector = new BitSetCollector(indexReader.maxDoc());
|
||||
int numHits = 1 + random().nextInt(25);
|
||||
TopScoreDocCollector actualTopDocsCollector = TopScoreDocCollector.create(numHits, false);
|
||||
TopScoreDocCollector actualTopDocsCollector = TopScoreDocCollector.create(numHits);
|
||||
searcher.search(query, MultiCollector.wrap(collector, actualTopDocsCollector));
|
||||
FixedBitSet actualResult = collector.getResult();
|
||||
|
||||
FixedBitSet expectedResult = new FixedBitSet(indexReader.maxDoc());
|
||||
MockScorer mockScorer = new MockScorer(scoreType);
|
||||
TopScoreDocCollector expectedTopDocsCollector = TopScoreDocCollector.create(numHits, false);
|
||||
TopScoreDocCollector expectedTopDocsCollector = TopScoreDocCollector.create(numHits);
|
||||
if (childValueToParentIds.containsKey(childValue)) {
|
||||
LeafReader slowLeafReader = SlowCompositeReaderWrapper.wrap(indexReader);
|
||||
final LeafCollector leafCollector = expectedTopDocsCollector.getLeafCollector(slowLeafReader.getContext());
|
||||
leafCollector.setScorer(mockScorer);
|
||||
final FloatArrayList[] scores = new FloatArrayList[slowLeafReader.maxDoc()];
|
||||
Terms terms = slowLeafReader.terms(UidFieldMapper.NAME);
|
||||
if (terms != null) {
|
||||
NavigableMap<String, FloatArrayList> parentIdToChildScores = childValueToParentIds.lget();
|
||||
|
@ -239,14 +238,20 @@ public class ChildrenQueryTests extends AbstractChildTests {
|
|||
if (seekStatus == TermsEnum.SeekStatus.FOUND) {
|
||||
docsEnum = termsEnum.docs(slowLeafReader.getLiveDocs(), docsEnum, DocsEnum.FLAG_NONE);
|
||||
expectedResult.set(docsEnum.nextDoc());
|
||||
mockScorer.scores = entry.getValue();
|
||||
leafCollector.collect(docsEnum.docID());
|
||||
scores[docsEnum.docID()] = new FloatArrayList(entry.getValue());
|
||||
} else if (seekStatus == TermsEnum.SeekStatus.END) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
MockScorer mockScorer = new MockScorer(scoreType);
|
||||
final LeafCollector leafCollector = expectedTopDocsCollector.getLeafCollector(slowLeafReader.getContext());
|
||||
leafCollector.setScorer(mockScorer);
|
||||
for (int doc = expectedResult.nextSetBit(0); doc < slowLeafReader.maxDoc(); doc = doc + 1 >= expectedResult.length() ? DocIdSetIterator.NO_MORE_DOCS : expectedResult.nextSetBit(doc + 1)) {
|
||||
mockScorer.scores = scores[doc];
|
||||
leafCollector.collect(doc);
|
||||
}
|
||||
}
|
||||
|
||||
assertBitSet(actualResult, expectedResult, searcher);
|
||||
|
|
|
@ -197,18 +197,15 @@ public class ParentQueryTests extends AbstractChildTests {
|
|||
|
||||
BitSetCollector collector = new BitSetCollector(indexReader.maxDoc());
|
||||
int numHits = 1 + random().nextInt(25);
|
||||
TopScoreDocCollector actualTopDocsCollector = TopScoreDocCollector.create(numHits, false);
|
||||
TopScoreDocCollector actualTopDocsCollector = TopScoreDocCollector.create(numHits);
|
||||
searcher.search(query, MultiCollector.wrap(collector, actualTopDocsCollector));
|
||||
FixedBitSet actualResult = collector.getResult();
|
||||
|
||||
FixedBitSet expectedResult = new FixedBitSet(indexReader.maxDoc());
|
||||
MockScorer mockScorer = new MockScorer(ScoreType.MAX); // just save one score per parent...
|
||||
mockScorer.scores = new FloatArrayList();
|
||||
TopScoreDocCollector expectedTopDocsCollector = TopScoreDocCollector.create(numHits, false);
|
||||
TopScoreDocCollector expectedTopDocsCollector = TopScoreDocCollector.create(numHits);
|
||||
if (parentValueToChildIds.containsKey(parentValue)) {
|
||||
LeafReader slowLeafReader = SlowCompositeReaderWrapper.wrap(indexReader);
|
||||
final LeafCollector leafCollector = expectedTopDocsCollector.getLeafCollector(slowLeafReader.getContext());
|
||||
leafCollector.setScorer(mockScorer);
|
||||
final FloatArrayList[] scores = new FloatArrayList[slowLeafReader.maxDoc()];
|
||||
Terms terms = slowLeafReader.terms(UidFieldMapper.NAME);
|
||||
if (terms != null) {
|
||||
NavigableMap<String, Float> childIdsAndScore = parentValueToChildIds.lget();
|
||||
|
@ -219,14 +216,25 @@ public class ParentQueryTests extends AbstractChildTests {
|
|||
if (seekStatus == TermsEnum.SeekStatus.FOUND) {
|
||||
docsEnum = termsEnum.docs(slowLeafReader.getLiveDocs(), docsEnum, DocsEnum.FLAG_NONE);
|
||||
expectedResult.set(docsEnum.nextDoc());
|
||||
mockScorer.scores.add(entry.getValue());
|
||||
leafCollector.collect(docsEnum.docID());
|
||||
mockScorer.scores.clear();
|
||||
FloatArrayList s = scores[docsEnum.docID()];
|
||||
if (s == null) {
|
||||
scores[docsEnum.docID()] = s = new FloatArrayList(2);
|
||||
}
|
||||
s.add(entry.getValue());
|
||||
} else if (seekStatus == TermsEnum.SeekStatus.END) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
MockScorer mockScorer = new MockScorer(ScoreType.MAX);
|
||||
mockScorer.scores = new FloatArrayList();
|
||||
final LeafCollector leafCollector = expectedTopDocsCollector.getLeafCollector(slowLeafReader.getContext());
|
||||
leafCollector.setScorer(mockScorer);
|
||||
for (int doc = expectedResult.nextSetBit(0); doc < slowLeafReader.maxDoc(); doc = doc + 1 >= expectedResult.length() ? DocIdSetIterator.NO_MORE_DOCS : expectedResult.nextSetBit(doc + 1)) {
|
||||
mockScorer.scores.clear();
|
||||
mockScorer.scores.addAll(scores[doc]);
|
||||
leafCollector.collect(doc);
|
||||
}
|
||||
}
|
||||
|
||||
assertBitSet(actualResult, expectedResult, searcher);
|
||||
|
|
Loading…
Reference in New Issue