[TEST] Added unit tests for diversified sampler aggregator.
This commit is contained in:
parent
9d4aff524c
commit
b01070a390
|
@ -16,8 +16,7 @@
|
|||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket;
|
||||
package org.elasticsearch.search.aggregations.bucket.sampler;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
|
@ -34,11 +33,11 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.common.util.ObjectArray;
|
||||
import org.elasticsearch.search.aggregations.BucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -48,13 +47,11 @@ import java.util.List;
|
|||
* {@link BestDocsDeferringCollector#createTopDocsCollector(int)} is designed to
|
||||
* be overridden and allows subclasses to choose a custom collector
|
||||
* implementation for determining the top N matches.
|
||||
*
|
||||
*/
|
||||
|
||||
public class BestDocsDeferringCollector extends DeferringBucketCollector implements Releasable {
|
||||
final List<PerSegmentCollects> entries = new ArrayList<>();
|
||||
BucketCollector deferred;
|
||||
ObjectArray<PerParentBucketSamples> perBucketSamples;
|
||||
private final List<PerSegmentCollects> entries = new ArrayList<>();
|
||||
private BucketCollector deferred;
|
||||
private ObjectArray<PerParentBucketSamples> perBucketSamples;
|
||||
private int shardSize;
|
||||
private PerSegmentCollects perSegCollector;
|
||||
private final BigArrays bigArrays;
|
||||
|
@ -65,14 +62,12 @@ public class BestDocsDeferringCollector extends DeferringBucketCollector impleme
|
|||
* @param shardSize
|
||||
* The number of top-scoring docs to collect for each bucket
|
||||
*/
|
||||
public BestDocsDeferringCollector(int shardSize, BigArrays bigArrays) {
|
||||
BestDocsDeferringCollector(int shardSize, BigArrays bigArrays) {
|
||||
this.shardSize = shardSize;
|
||||
this.bigArrays = bigArrays;
|
||||
perBucketSamples = bigArrays.newObjectArray(1);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
|
@ -126,7 +121,6 @@ public class BestDocsDeferringCollector extends DeferringBucketCollector impleme
|
|||
}
|
||||
|
||||
private void runDeferredAggs() throws IOException {
|
||||
|
||||
List<ScoreDoc> allDocs = new ArrayList<>(shardSize);
|
||||
for (int i = 0; i < perBucketSamples.size(); i++) {
|
||||
PerParentBucketSamples perBucketSample = perBucketSamples.get(i);
|
||||
|
@ -138,15 +132,12 @@ public class BestDocsDeferringCollector extends DeferringBucketCollector impleme
|
|||
|
||||
// Sort the top matches by docID for the benefit of deferred collector
|
||||
ScoreDoc[] docsArr = allDocs.toArray(new ScoreDoc[allDocs.size()]);
|
||||
Arrays.sort(docsArr, new Comparator<ScoreDoc>() {
|
||||
@Override
|
||||
public int compare(ScoreDoc o1, ScoreDoc o2) {
|
||||
if(o1.doc == o2.doc){
|
||||
return o1.shardIndex - o2.shardIndex;
|
||||
}
|
||||
return o1.doc - o2.doc;
|
||||
}
|
||||
});
|
||||
Arrays.sort(docsArr, (o1, o2) -> {
|
||||
if(o1.doc == o2.doc){
|
||||
return o1.shardIndex - o2.shardIndex;
|
||||
}
|
||||
return o1.doc - o2.doc;
|
||||
});
|
||||
try {
|
||||
for (PerSegmentCollects perSegDocs : entries) {
|
||||
perSegDocs.replayRelatedMatches(docsArr);
|
||||
|
@ -295,7 +286,6 @@ public class BestDocsDeferringCollector extends DeferringBucketCollector impleme
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public int getDocCount(long parentBucket) {
|
||||
PerParentBucketSamples sampler = perBucketSamples.get((int) parentBucket);
|
||||
if (sampler == null) {
|
|
@ -43,7 +43,7 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<
|
|||
private final int maxDocsPerValue;
|
||||
private final String executionHint;
|
||||
|
||||
public DiversifiedAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, int shardSize, int maxDocsPerValue,
|
||||
DiversifiedAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, int shardSize, int maxDocsPerValue,
|
||||
String executionHint, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
|
||||
Map<String, Object> metaData) throws IOException {
|
||||
super(name, config, context, parent, subFactoriesBuilder, metaData);
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.elasticsearch.ElasticsearchException;
|
|||
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
|
@ -48,7 +47,7 @@ public class DiversifiedBytesHashSamplerAggregator extends SamplerAggregator {
|
|||
private ValuesSource valuesSource;
|
||||
private int maxDocsPerValue;
|
||||
|
||||
public DiversifiedBytesHashSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
DiversifiedBytesHashSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
|
||||
ValuesSource valuesSource,
|
||||
int maxDocsPerValue) throws IOException {
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.elasticsearch.common.util.BytesRefHash;
|
|||
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
|
@ -47,7 +46,7 @@ public class DiversifiedMapSamplerAggregator extends SamplerAggregator {
|
|||
private int maxDocsPerValue;
|
||||
private BytesRefHash bucketOrds;
|
||||
|
||||
public DiversifiedMapSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
DiversifiedMapSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
|
||||
ValuesSource valuesSource, int maxDocsPerValue) throws IOException {
|
||||
super(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.lucene.search.TopDocsCollector;
|
|||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
|
@ -43,7 +42,7 @@ public class DiversifiedNumericSamplerAggregator extends SamplerAggregator {
|
|||
private ValuesSource.Numeric valuesSource;
|
||||
private int maxDocsPerValue;
|
||||
|
||||
public DiversifiedNumericSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
DiversifiedNumericSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
|
||||
ValuesSource.Numeric valuesSource, int maxDocsPerValue) throws IOException {
|
||||
super(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.lucene.search.DiversifiedTopDocsCollector.ScoreDocKey;
|
|||
import org.apache.lucene.search.TopDocsCollector;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
|
@ -44,7 +43,7 @@ public class DiversifiedOrdinalsSamplerAggregator extends SamplerAggregator {
|
|||
private ValuesSource.Bytes.WithOrdinals.FieldData valuesSource;
|
||||
private int maxDocsPerValue;
|
||||
|
||||
public DiversifiedOrdinalsSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
DiversifiedOrdinalsSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
|
||||
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
|
||||
ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, int maxDocsPerValue) throws IOException {
|
||||
super(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
|
@ -53,7 +52,6 @@ public class SamplerAggregator extends SingleBucketAggregator {
|
|||
public static final ParseField MAX_DOCS_PER_VALUE_FIELD = new ParseField("max_docs_per_value");
|
||||
public static final ParseField EXECUTION_HINT_FIELD = new ParseField("execution_hint");
|
||||
|
||||
|
||||
public enum ExecutionMode {
|
||||
|
||||
MAP(new ParseField("map")) {
|
||||
|
@ -141,7 +139,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
|
|||
protected final int shardSize;
|
||||
protected BestDocsDeferringCollector bdd;
|
||||
|
||||
public SamplerAggregator(String name, int shardSize, AggregatorFactories factories, SearchContext context,
|
||||
SamplerAggregator(String name, int shardSize, AggregatorFactories factories, SearchContext context,
|
||||
Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
super(name, factories, context, parent, pipelineAggregators, metaData);
|
||||
this.shardSize = shardSize;
|
||||
|
@ -156,10 +154,8 @@ public class SamplerAggregator extends SingleBucketAggregator {
|
|||
public DeferringBucketCollector getDeferringCollector() {
|
||||
bdd = new BestDocsDeferringCollector(shardSize, context.bigArrays());
|
||||
return bdd;
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected boolean shouldDefer(Aggregator aggregator) {
|
||||
return true;
|
||||
|
@ -193,4 +189,3 @@ public class SamplerAggregator extends SingleBucketAggregator {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -33,7 +33,7 @@ public class SamplerAggregatorFactory extends AggregatorFactory<SamplerAggregato
|
|||
|
||||
private final int shardSize;
|
||||
|
||||
public SamplerAggregatorFactory(String name, int shardSize, SearchContext context, AggregatorFactory<?> parent,
|
||||
SamplerAggregatorFactory(String name, int shardSize, SearchContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
|
||||
super(name, context, parent, subFactories, metaData);
|
||||
this.shardSize = shardSize;
|
||||
|
|
|
@ -31,7 +31,7 @@ import java.util.Map;
|
|||
public class UnmappedSampler extends InternalSampler {
|
||||
public static final String NAME = "unmapped_sampler";
|
||||
|
||||
public UnmappedSampler(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
|
||||
UnmappedSampler(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
|
||||
super(name, 0, InternalAggregations.EMPTY, pipelineAggregators, metaData);
|
||||
}
|
||||
|
||||
|
|
|
@ -51,7 +51,6 @@ import org.elasticsearch.search.internal.ContextIndexSearcher;
|
|||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.lookup.SearchLookup;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.IndexSettingsModule;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -75,15 +74,31 @@ public abstract class AggregatorTestCase extends ESTestCase {
|
|||
protected <A extends Aggregator, B extends AggregationBuilder> A createAggregator(B aggregationBuilder,
|
||||
IndexSearcher indexSearcher,
|
||||
MappedFieldType... fieldTypes) throws IOException {
|
||||
IndexSettings indexSettings = new IndexSettings(
|
||||
IndexMetaData.builder("_index").settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
|
||||
.numberOfShards(1)
|
||||
.numberOfReplicas(0)
|
||||
.creationDate(System.currentTimeMillis())
|
||||
.build(),
|
||||
Settings.EMPTY
|
||||
);
|
||||
IndexSettings indexSettings = createIndexSettings();
|
||||
SearchContext searchContext = createSearchContext(indexSearcher, indexSettings);
|
||||
CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService();
|
||||
when(searchContext.bigArrays()).thenReturn(new MockBigArrays(Settings.EMPTY, circuitBreakerService));
|
||||
// TODO: now just needed for top_hits, this will need to be revised for other agg unit tests:
|
||||
MapperService mapperService = mapperServiceMock();
|
||||
when(mapperService.hasNested()).thenReturn(false);
|
||||
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||
IndexFieldDataService ifds = new IndexFieldDataService(indexSettings,
|
||||
new IndicesFieldDataCache(Settings.EMPTY, new IndexFieldDataCache.Listener() {
|
||||
}), circuitBreakerService, mapperService);
|
||||
when(searchContext.fieldData()).thenReturn(ifds);
|
||||
|
||||
SearchLookup searchLookup = new SearchLookup(mapperService, ifds, new String[]{"type"});
|
||||
when(searchContext.lookup()).thenReturn(searchLookup);
|
||||
|
||||
QueryShardContext queryShardContext = queryShardContextMock(fieldTypes, indexSettings, circuitBreakerService);
|
||||
when(searchContext.getQueryShardContext()).thenReturn(queryShardContext);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
A aggregator = (A) aggregationBuilder.build(searchContext, null).create(null, true);
|
||||
return aggregator;
|
||||
}
|
||||
|
||||
protected SearchContext createSearchContext(IndexSearcher indexSearcher, IndexSettings indexSettings) {
|
||||
Engine.Searcher searcher = new Engine.Searcher("aggregator_test", indexSearcher);
|
||||
QueryCache queryCache = new DisabledQueryCache(indexSettings);
|
||||
QueryCachingPolicy queryCachingPolicy = new QueryCachingPolicy() {
|
||||
|
@ -99,38 +114,29 @@ public abstract class AggregatorTestCase extends ESTestCase {
|
|||
};
|
||||
ContextIndexSearcher contextIndexSearcher = new ContextIndexSearcher(searcher, queryCache, queryCachingPolicy);
|
||||
|
||||
CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService();
|
||||
SearchContext searchContext = mock(SearchContext.class);
|
||||
when(searchContext.numberOfShards()).thenReturn(1);
|
||||
when(searchContext.searcher()).thenReturn(contextIndexSearcher);
|
||||
when(searchContext.bigArrays()).thenReturn(new MockBigArrays(Settings.EMPTY, circuitBreakerService));
|
||||
when(searchContext.fetchPhase())
|
||||
.thenReturn(new FetchPhase(Arrays.asList(new FetchSourceSubPhase(), new DocValueFieldsFetchSubPhase())));
|
||||
.thenReturn(new FetchPhase(Arrays.asList(new FetchSourceSubPhase(), new DocValueFieldsFetchSubPhase())));
|
||||
doAnswer(invocation -> {
|
||||
/* Store the releasables so we can release them at the end of the test case. This is important because aggregations don't
|
||||
* close their sub-aggregations. This is fairly similar to what the production code does. */
|
||||
releasables.add((Releasable) invocation.getArguments()[0]);
|
||||
return null;
|
||||
}).when(searchContext).addReleasable(anyObject(), anyObject());
|
||||
return searchContext;
|
||||
}
|
||||
|
||||
// TODO: now just needed for top_hits, this will need to be revised for other agg unit tests:
|
||||
MapperService mapperService = mapperServiceMock();
|
||||
when(mapperService.hasNested()).thenReturn(false);
|
||||
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||
IndexFieldDataService ifds = new IndexFieldDataService(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY),
|
||||
new IndicesFieldDataCache(Settings.EMPTY, new IndexFieldDataCache.Listener() {
|
||||
}), circuitBreakerService, mapperService);
|
||||
when(searchContext.fieldData()).thenReturn(ifds);
|
||||
|
||||
SearchLookup searchLookup = new SearchLookup(mapperService, ifds, new String[]{"type"});
|
||||
when(searchContext.lookup()).thenReturn(searchLookup);
|
||||
|
||||
QueryShardContext queryShardContext = queryShardContextMock(fieldTypes, indexSettings, circuitBreakerService);
|
||||
when(searchContext.getQueryShardContext()).thenReturn(queryShardContext);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
A aggregator = (A) aggregationBuilder.build(searchContext, null).create(null, true);
|
||||
return aggregator;
|
||||
protected IndexSettings createIndexSettings() {
|
||||
return new IndexSettings(
|
||||
IndexMetaData.builder("_index").settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
|
||||
.numberOfShards(1)
|
||||
.numberOfReplicas(0)
|
||||
.creationDate(System.currentTimeMillis())
|
||||
.build(),
|
||||
Settings.EMPTY
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorTestCase;
|
||||
import org.elasticsearch.search.aggregations.BucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
public class BestBucketsDeferringCollectorTests extends AggregatorTestCase {
|
||||
|
||||
public void testReplay() throws Exception {
|
||||
Directory directory = newDirectory();
|
||||
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
|
||||
int numDocs = randomInt(128);
|
||||
int maxNumValues = randomInt(16);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document document = new Document();
|
||||
document.add(new StringField("field", String.valueOf(randomInt(maxNumValues)), Field.Store.NO));
|
||||
indexWriter.addDocument(document);
|
||||
}
|
||||
|
||||
indexWriter.close();
|
||||
IndexReader indexReader = DirectoryReader.open(directory);
|
||||
IndexSearcher indexSearcher = new IndexSearcher(indexReader);
|
||||
|
||||
TermQuery termQuery = new TermQuery(new Term("field", String.valueOf(randomInt(maxNumValues))));
|
||||
TopDocs topDocs = indexSearcher.search(termQuery, numDocs);
|
||||
|
||||
SearchContext searchContext = createSearchContext(indexSearcher, createIndexSettings());
|
||||
BestBucketsDeferringCollector collector = new BestBucketsDeferringCollector(searchContext);
|
||||
Set<Integer> deferredCollectedDocIds = new HashSet<>();
|
||||
collector.setDeferredCollector(Collections.singleton(bla(deferredCollectedDocIds)));
|
||||
collector.preCollection();
|
||||
indexSearcher.search(termQuery, collector);
|
||||
collector.postCollection();
|
||||
collector.replay(0);
|
||||
|
||||
assertEquals(topDocs.scoreDocs.length, deferredCollectedDocIds.size());
|
||||
for (ScoreDoc scoreDoc : topDocs.scoreDocs) {
|
||||
assertTrue("expected docid [" + scoreDoc.doc + "] is missing", deferredCollectedDocIds.contains(scoreDoc.doc));
|
||||
}
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
private BucketCollector bla(Set<Integer> docIds) {
|
||||
return new BucketCollector() {
|
||||
@Override
|
||||
public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException {
|
||||
return new LeafBucketCollector() {
|
||||
@Override
|
||||
public void collect(int doc, long bucket) throws IOException {
|
||||
docIds.add(ctx.docBase + doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preCollection() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postCollection() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,113 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.search.aggregations.bucket.sampler;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.MockBigArrays;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
import org.elasticsearch.search.aggregations.AggregatorTestCase;
|
||||
import org.elasticsearch.search.aggregations.BucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
public class BestDocsDeferringCollectorTests extends AggregatorTestCase {
|
||||
|
||||
public void testReplay() throws Exception {
|
||||
Directory directory = newDirectory();
|
||||
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
|
||||
int numDocs = randomIntBetween(1, 128);
|
||||
int maxNumValues = randomInt(16);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document document = new Document();
|
||||
document.add(new StringField("field", String.valueOf(randomInt(maxNumValues)), Field.Store.NO));
|
||||
indexWriter.addDocument(document);
|
||||
}
|
||||
|
||||
indexWriter.close();
|
||||
IndexReader indexReader = DirectoryReader.open(directory);
|
||||
IndexSearcher indexSearcher = new IndexSearcher(indexReader);
|
||||
|
||||
TermQuery termQuery = new TermQuery(new Term("field", String.valueOf(randomInt(maxNumValues))));
|
||||
TopDocs topDocs = indexSearcher.search(termQuery, numDocs);
|
||||
|
||||
BestDocsDeferringCollector collector =
|
||||
new BestDocsDeferringCollector(numDocs, new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService()));
|
||||
Set<Integer> deferredCollectedDocIds = new HashSet<>();
|
||||
collector.setDeferredCollector(Collections.singleton(testCollector(deferredCollectedDocIds)));
|
||||
collector.preCollection();
|
||||
indexSearcher.search(termQuery, collector);
|
||||
collector.postCollection();
|
||||
collector.replay(0);
|
||||
|
||||
assertEquals(topDocs.scoreDocs.length, deferredCollectedDocIds.size());
|
||||
for (ScoreDoc scoreDoc : topDocs.scoreDocs) {
|
||||
assertTrue("expected docid [" + scoreDoc.doc + "] is missing", deferredCollectedDocIds.contains(scoreDoc.doc));
|
||||
}
|
||||
collector.close();
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
private BucketCollector testCollector(Set<Integer> docIds) {
|
||||
return new BucketCollector() {
|
||||
@Override
|
||||
public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException {
|
||||
return new LeafBucketCollector() {
|
||||
@Override
|
||||
public void collect(int doc, long bucket) throws IOException {
|
||||
docIds.add(ctx.docBase + doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preCollection() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postCollection() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -17,14 +17,13 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket;
|
||||
package org.elasticsearch.search.aggregations.bucket.sampler;
|
||||
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
|
||||
import org.elasticsearch.search.aggregations.bucket.sampler.DiversifiedAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.ExecutionMode;
|
||||
|
||||
public class DiversifiedSamplerTests extends BaseAggregationTestCase<DiversifiedAggregationBuilder> {
|
||||
public class DiversifiedAggregationBuilderTests extends BaseAggregationTestCase<DiversifiedAggregationBuilder> {
|
||||
|
||||
@Override
|
||||
protected final DiversifiedAggregationBuilder createTestAggregatorBuilder() {
|
|
@ -0,0 +1,165 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.search.aggregations.bucket.sampler;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoubleDocValuesField;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.common.lucene.search.function.FieldValueFactorFunction;
|
||||
import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
|
||||
import org.elasticsearch.index.fielddata.plain.SortedNumericDVIndexFieldData;
|
||||
import org.elasticsearch.index.mapper.KeywordFieldMapper;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.NumberFieldMapper;
|
||||
import org.elasticsearch.search.aggregations.AggregatorTestCase;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
public class DiversifiedSamplerTests extends AggregatorTestCase {
|
||||
|
||||
public void testDiversifiedSampler() throws Exception {
|
||||
String data[] = {
|
||||
// "id,cat,name,price,inStock,author_t,series_t,sequence_i,genre_s,genre_id",
|
||||
"0553573403,book,A Game of Thrones,7.99,true,George R.R. Martin,A Song of Ice and Fire,1,fantasy,0",
|
||||
"0553579908,book,A Clash of Kings,7.99,true,George R.R. Martin,A Song of Ice and Fire,2,fantasy,0",
|
||||
"055357342X,book,A Storm of Swords,7.99,true,George R.R. Martin,A Song of Ice and Fire,3,fantasy,0",
|
||||
"0553293354,book,Foundation,17.99,true,Isaac Asimov,Foundation Novels,1,scifi,1",
|
||||
"0812521390,book,The Black Company,6.99,false,Glen Cook,The Chronicles of The Black Company,1,fantasy,0",
|
||||
"0812550706,book,Ender's Game,6.99,true,Orson Scott Card,Ender,1,scifi,1",
|
||||
"0441385532,book,Jhereg,7.95,false,Steven Brust,Vlad Taltos,1,fantasy,0",
|
||||
"0380014300,book,Nine Princes In Amber,6.99,true,Roger Zelazny,the Chronicles of Amber,1,fantasy,0",
|
||||
"0805080481,book,The Book of Three,5.99,true,Lloyd Alexander,The Chronicles of Prydain,1,fantasy,0",
|
||||
"080508049X,book,The Black Cauldron,5.99,true,Lloyd Alexander,The Chronicles of Prydain,2,fantasy,0"
|
||||
};
|
||||
|
||||
Directory directory = newDirectory();
|
||||
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
|
||||
for (String entry : data) {
|
||||
String[] parts = entry.split(",");
|
||||
Document document = new Document();
|
||||
document.add(new SortedDocValuesField("id", new BytesRef(parts[0])));
|
||||
document.add(new StringField("cat", parts[1], Field.Store.NO));
|
||||
document.add(new TextField("name", parts[2], Field.Store.NO));
|
||||
document.add(new DoubleDocValuesField("price", Double.valueOf(parts[3])));
|
||||
document.add(new StringField("inStock", parts[4], Field.Store.NO));
|
||||
document.add(new StringField("author", parts[5], Field.Store.NO));
|
||||
document.add(new StringField("series", parts[6], Field.Store.NO));
|
||||
document.add(new StringField("sequence", parts[7], Field.Store.NO));
|
||||
document.add(new SortedDocValuesField("genre", new BytesRef(parts[8])));
|
||||
document.add(new NumericDocValuesField("genre_id", Long.valueOf(parts[9])));
|
||||
indexWriter.addDocument(document);
|
||||
}
|
||||
|
||||
indexWriter.close();
|
||||
IndexReader indexReader = DirectoryReader.open(directory);
|
||||
IndexSearcher indexSearcher = new IndexSearcher(indexReader);
|
||||
|
||||
MappedFieldType genreFieldType = new KeywordFieldMapper.KeywordFieldType();
|
||||
genreFieldType.setName("genre");
|
||||
genreFieldType.setHasDocValues(true);
|
||||
Consumer<InternalSampler> verify = result -> {
|
||||
Terms terms = result.getAggregations().get("terms");
|
||||
assertEquals(2, terms.getBuckets().size());
|
||||
assertEquals("0805080481", terms.getBuckets().get(0).getKeyAsString());
|
||||
assertEquals("0812550706", terms.getBuckets().get(1).getKeyAsString());
|
||||
};
|
||||
testCase(indexSearcher, genreFieldType, "map", verify);
|
||||
testCase(indexSearcher, genreFieldType, "global_ordinals", verify);
|
||||
testCase(indexSearcher, genreFieldType, "bytes_hash", verify);
|
||||
|
||||
genreFieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
|
||||
genreFieldType.setName("genre_id");
|
||||
testCase(indexSearcher, genreFieldType, null, verify);
|
||||
|
||||
// wrong field:
|
||||
genreFieldType = new KeywordFieldMapper.KeywordFieldType();
|
||||
genreFieldType.setName("wrong_field");
|
||||
genreFieldType.setHasDocValues(true);
|
||||
testCase(indexSearcher, genreFieldType, null, result -> {
|
||||
Terms terms = result.getAggregations().get("terms");
|
||||
assertEquals(1, terms.getBuckets().size());
|
||||
assertEquals("0805080481", terms.getBuckets().get(0).getKeyAsString());
|
||||
});
|
||||
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
private void testCase(IndexSearcher indexSearcher, MappedFieldType genreFieldType, String executionHint,
|
||||
Consumer<InternalSampler> verify) throws IOException {
|
||||
MappedFieldType idFieldType = new KeywordFieldMapper.KeywordFieldType();
|
||||
idFieldType.setName("id");
|
||||
idFieldType.setHasDocValues(true);
|
||||
|
||||
SortedNumericDVIndexFieldData fieldData = new SortedNumericDVIndexFieldData(new Index("index", "index"), "price",
|
||||
IndexNumericFieldData.NumericType.DOUBLE);
|
||||
FunctionScoreQuery query = new FunctionScoreQuery(new MatchAllDocsQuery(),
|
||||
new FieldValueFactorFunction("price", 1, FieldValueFactorFunction.Modifier.RECIPROCAL, null, fieldData));
|
||||
|
||||
DiversifiedAggregationBuilder builder = new DiversifiedAggregationBuilder("_name")
|
||||
.field(genreFieldType.name())
|
||||
.executionHint(executionHint)
|
||||
.subAggregation(new TermsAggregationBuilder("terms", null).field("id"));
|
||||
|
||||
InternalSampler result = search(indexSearcher, query, builder, genreFieldType, idFieldType);
|
||||
verify.accept(result);
|
||||
}
|
||||
|
||||
public void testDiversifiedSampler_noDocs() throws Exception {
|
||||
Directory directory = newDirectory();
|
||||
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
|
||||
indexWriter.close();
|
||||
IndexReader indexReader = DirectoryReader.open(directory);
|
||||
IndexSearcher indexSearcher = new IndexSearcher(indexReader);
|
||||
|
||||
MappedFieldType idFieldType = new KeywordFieldMapper.KeywordFieldType();
|
||||
idFieldType.setName("id");
|
||||
idFieldType.setHasDocValues(true);
|
||||
|
||||
MappedFieldType genreFieldType = new KeywordFieldMapper.KeywordFieldType();
|
||||
genreFieldType.setName("genre");
|
||||
genreFieldType.setHasDocValues(true);
|
||||
|
||||
DiversifiedAggregationBuilder builder = new DiversifiedAggregationBuilder("_name")
|
||||
.field(genreFieldType.name())
|
||||
.subAggregation(new TermsAggregationBuilder("terms", null).field("id"));
|
||||
|
||||
InternalSampler result = search(indexSearcher, new MatchAllDocsQuery(), builder, genreFieldType, idFieldType);
|
||||
Terms terms = result.getAggregations().get("terms");
|
||||
assertEquals(0, terms.getBuckets().size());
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue