Aggs: Refactor aggregations to use lucene5-style collectors.

Aggregators now return a new collector instance per segment, like Lucene 5 does
with its oal.search.Collector API. This is important for us because things like
knowing whether the field is single or multi-valued is only known at a segment
level.

In order to do that I had to change aggregators to notify their sub aggregators
of new incoming segments (pretty much in the spirit of #6477) while everything
used to be centralized in the AggregationContext class. While this might slow
down a bit deeply nested aggregation trees, this also makes the children
aggregation and the `breadth_first` collection mode much better options since
they can now only replay what they need while they used to have to replay the
whole aggregation tree.

I also took advantage of this big refactoring to remove some abstractions that
were not really required like ValuesSource.MetaData or BucketAnalysisCollector.
I also splitted Aggregator into Aggregator and AggregatorBase in order to
separate the Aggregator API from implementation helpers.

Close #9544
This commit is contained in:
Adrien Grand 2015-01-27 14:22:01 +01:00
parent d1deb6bd1e
commit de41981373
78 changed files with 2277 additions and 2684 deletions

View File

@ -23,9 +23,13 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.SloppyMath; import org.apache.lucene.util.SloppyMath;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.common.unit.DistanceUnit; import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.*; import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.GeoPointValues;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.SortingNumericDoubleValues;
import java.util.List;
import java.util.Locale; import java.util.Locale;
/** /**
@ -398,8 +402,7 @@ public enum GeoDistance {
@Override @Override
public void setDocument(int doc) { public void setDocument(int doc) {
geoPointValues.setDocument(doc); geoPointValues.setDocument(doc);
count = geoPointValues.count() * distances.length; resize(geoPointValues.count() * distances.length);
grow();
int valueCounter = 0; int valueCounter = 0;
for (FixedSourceDistance distance : distances) { for (FixedSourceDistance distance : distances) {
for (int i = 0; i < geoPointValues.count(); ++i) { for (int i = 0; i < geoPointValues.count(); ++i) {

View File

@ -24,8 +24,31 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.*; import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.search.*; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.ComplexExplanation;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TimeLimitingCollector;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopFieldDocs;
import org.apache.lucene.search.TotalHitCountCollector;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexInput;
@ -34,6 +57,7 @@ import org.apache.lucene.util.Counter;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -579,4 +603,37 @@ public class Lucene {
return defaultValue; return defaultValue;
} }
} }
/**
* Return a Scorer that throws an ElasticsearchIllegalStateException
* on all operations with the given message.
*/
public static Scorer illegalScorer(final String message) {
return new Scorer(null) {
@Override
public float score() throws IOException {
throw new ElasticsearchIllegalStateException(message);
}
@Override
public int freq() throws IOException {
throw new ElasticsearchIllegalStateException(message);
}
@Override
public int advance(int arg0) throws IOException {
throw new ElasticsearchIllegalStateException(message);
}
@Override
public long cost() {
throw new ElasticsearchIllegalStateException(message);
}
@Override
public int docID() {
throw new ElasticsearchIllegalStateException(message);
}
@Override
public int nextDoc() throws IOException {
throw new ElasticsearchIllegalStateException(message);
}
};
}
} }

View File

@ -1,29 +0,0 @@
/*
* 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.common.lucene;
import org.apache.lucene.index.IndexReaderContext;
/**
*
*/
public interface TopReaderContextAware {
public void setNextReader(IndexReaderContext reader);
}

View File

@ -52,7 +52,7 @@ public final class LongHash extends AbstractHash {
} }
/** /**
* Get the id associated with <code>key</code> * Get the id associated with <code>key</code> or -1 if the key is not contained in the hash.
*/ */
public long find(long key) { public long find(long key) {
final long slot = slot(hash(key), mask); final long slot = slot(hash(key), mask);

View File

@ -28,7 +28,7 @@ import org.apache.lucene.util.Sorter;
*/ */
public abstract class SortingNumericDoubleValues extends SortedNumericDoubleValues { public abstract class SortingNumericDoubleValues extends SortedNumericDoubleValues {
protected int count; private int count;
protected double[] values; protected double[] values;
private final Sorter sorter; private final Sorter sorter;
@ -51,9 +51,11 @@ public abstract class SortingNumericDoubleValues extends SortedNumericDoubleValu
} }
/** /**
* Make sure the {@link #values} array can store at least {@link #count} entries. * Set the {@link #count()} and ensure that the {@link #values} array can
* store at least that many entries.
*/ */
protected final void grow() { protected final void resize(int newSize) {
count = newSize;
values = ArrayUtil.grow(values, count); values = ArrayUtil.grow(values, count);
} }
@ -65,10 +67,12 @@ public abstract class SortingNumericDoubleValues extends SortedNumericDoubleValu
sorter.sort(0, count); sorter.sort(0, count);
} }
@Override
public final int count() { public final int count() {
return count; return count;
} }
@Override
public final double valueAt(int index) { public final double valueAt(int index) {
return values[index]; return values[index];
} }

View File

@ -772,11 +772,7 @@ public class PercolatorService extends AbstractComponent {
percolatorTypeFilter = context.indexService().cache().filter().cache(percolatorTypeFilter, null, context.indexService().queryParserService().autoFilterCachePolicy()); percolatorTypeFilter = context.indexService().cache().filter().cache(percolatorTypeFilter, null, context.indexService().queryParserService().autoFilterCachePolicy());
FilteredQuery query = new FilteredQuery(context.percolateQuery(), percolatorTypeFilter); FilteredQuery query = new FilteredQuery(context.percolateQuery(), percolatorTypeFilter);
percolatorSearcher.searcher().search(query, percolateCollector); percolatorSearcher.searcher().search(query, percolateCollector);
for (Collector queryCollector : percolateCollector.aggregatorCollector) { percolateCollector.aggregatorCollector.postCollection();
if (queryCollector instanceof XCollector) {
((XCollector) queryCollector).postCollection();
}
}
if (context.aggregations() != null) { if (context.aggregations() != null) {
aggregationPhase.execute(context); aggregationPhase.execute(context);
} }

View File

@ -19,20 +19,27 @@
package org.elasticsearch.percolator; package org.elasticsearch.percolator;
import com.carrotsearch.hppc.FloatArrayList; import com.carrotsearch.hppc.FloatArrayList;
import com.google.common.collect.ImmutableList;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.*; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.search.XCollector;
import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.internal.IdFieldMapper; import org.elasticsearch.index.mapper.internal.IdFieldMapper;
import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.search.nested.NonNestedDocsFilter; import org.elasticsearch.index.search.nested.NonNestedDocsFilter;
import org.elasticsearch.search.aggregations.AggregationPhase;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.BucketCollector;
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.highlight.HighlightField; import org.elasticsearch.search.highlight.HighlightField;
@ -59,9 +66,8 @@ abstract class QueryCollector extends SimpleCollector {
SortedBinaryDocValues values; SortedBinaryDocValues values;
final List<Collector> aggregatorCollector; final XCollector aggregatorCollector;
LeafCollector aggregatorLeafCollector;
List<LeafCollector> aggregatorLeafCollectors;
QueryCollector(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException { QueryCollector(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
this.logger = logger; this.logger = logger;
@ -71,63 +77,43 @@ abstract class QueryCollector extends SimpleCollector {
this.idFieldData = context.fieldData().getForField(idMapper); this.idFieldData = context.fieldData().getForField(idMapper);
this.isNestedDoc = isNestedDoc; this.isNestedDoc = isNestedDoc;
ImmutableList.Builder<Collector> aggCollectorBuilder = ImmutableList.builder(); List<Aggregator> aggregatorCollectors = new ArrayList<>();
if (context.aggregations() != null) { if (context.aggregations() != null) {
AggregationContext aggregationContext = new AggregationContext(context); AggregationContext aggregationContext = new AggregationContext(context);
context.aggregations().aggregationContext(aggregationContext); context.aggregations().aggregationContext(aggregationContext);
List<Aggregator> aggregatorCollectors = new ArrayList<>();
Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext); Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext);
for (int i = 0; i < aggregators.length; i++) { for (int i = 0; i < aggregators.length; i++) {
if (!(aggregators[i] instanceof GlobalAggregator)) { if (!(aggregators[i] instanceof GlobalAggregator)) {
Aggregator aggregator = aggregators[i]; Aggregator aggregator = aggregators[i];
if (aggregator.shouldCollect()) { aggregatorCollectors.add(aggregator);
aggregatorCollectors.add(aggregator);
}
} }
} }
context.aggregations().aggregators(aggregators); context.aggregations().aggregators(aggregators);
if (!aggregatorCollectors.isEmpty()) {
aggCollectorBuilder.add(new AggregationPhase.AggregationsCollector(aggregatorCollectors, aggregationContext));
}
aggregationContext.setNextReader(context.searcher().getIndexReader().getContext());
} }
aggregatorCollector = aggCollectorBuilder.build(); aggregatorCollector = BucketCollector.wrap(aggregatorCollectors);
aggregatorLeafCollectors = new ArrayList<>(aggregatorCollector.size());
} }
public void postMatch(int doc) throws IOException { public void postMatch(int doc) throws IOException {
for (LeafCollector collector : aggregatorLeafCollectors) { aggregatorLeafCollector.collect(doc);
collector.collect(doc);
}
} }
@Override @Override
public void setScorer(Scorer scorer) throws IOException { public void setScorer(Scorer scorer) throws IOException {
for (LeafCollector collector : aggregatorLeafCollectors) { aggregatorLeafCollector.setScorer(scorer);
collector.setScorer(scorer);
}
} }
@Override @Override
public boolean needsScores() { public boolean needsScores() {
for (Collector collector : aggregatorCollector) { return aggregatorCollector.needsScores();
if (collector.needsScores()) {
return true;
}
}
return false;
} }
@Override @Override
public void doSetNextReader(LeafReaderContext context) throws IOException { public void doSetNextReader(LeafReaderContext context) throws IOException {
// we use the UID because id might not be indexed // we use the UID because id might not be indexed
values = idFieldData.load(context).getBytesValues(); values = idFieldData.load(context).getBytesValues();
aggregatorLeafCollectors.clear(); aggregatorLeafCollector = aggregatorCollector.getLeafCollector(context);
for (Collector collector : aggregatorCollector) {
aggregatorLeafCollectors.add(collector.getLeafCollector(context));
}
} }
static Match match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException { static Match match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {

View File

@ -20,13 +20,10 @@ package org.elasticsearch.search.aggregations;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilteredQuery; import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.Query; 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.ElasticsearchException;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.Queries;
@ -40,7 +37,6 @@ import org.elasticsearch.search.query.QueryPhaseExecutionException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -85,18 +81,14 @@ public class AggregationPhase implements SearchPhase {
throw new AggregationInitializationException("Could not initialize aggregators", e); throw new AggregationInitializationException("Could not initialize aggregators", e);
} }
for (int i = 0; i < aggregators.length; i++) { for (int i = 0; i < aggregators.length; i++) {
if (!(aggregators[i] instanceof GlobalAggregator)) { if (aggregators[i] instanceof GlobalAggregator == false) {
Aggregator aggregator = aggregators[i]; collectors.add(aggregators[i]);
if (aggregator.shouldCollect()) {
collectors.add(aggregator);
}
} }
} }
context.aggregations().aggregators(aggregators); context.aggregations().aggregators(aggregators);
if (!collectors.isEmpty()) { if (!collectors.isEmpty()) {
context.searcher().queryCollectors().put(AggregationPhase.class, new AggregationsCollector(collectors, aggregationContext)); context.searcher().queryCollectors().put(AggregationPhase.class, (BucketCollector.wrap(collectors)));
} }
aggregationContext.setNextReader(context.searcher().getIndexReader().getContext());
} }
} }
@ -122,7 +114,7 @@ public class AggregationPhase implements SearchPhase {
// optimize the global collector based execution // optimize the global collector based execution
if (!globals.isEmpty()) { if (!globals.isEmpty()) {
AggregationsCollector collector = new AggregationsCollector(globals, context.aggregations().aggregationContext()); XCollector collector = BucketCollector.wrap(globals);
Query query = new ConstantScoreQuery(Queries.MATCH_ALL_FILTER); Query query = new ConstantScoreQuery(Queries.MATCH_ALL_FILTER);
Filter searchFilter = context.searchFilter(context.types()); Filter searchFilter = context.searchFilter(context.types());
if (searchFilter != null) { if (searchFilter != null) {
@ -151,44 +143,4 @@ public class AggregationPhase implements SearchPhase {
context.searcher().queryCollectors().remove(AggregationPhase.class); context.searcher().queryCollectors().remove(AggregationPhase.class);
} }
public static class AggregationsCollector extends SimpleCollector implements XCollector {
private final AggregationContext aggregationContext;
private final Aggregator[] collectors;
public AggregationsCollector(Collection<Aggregator> collectors, AggregationContext aggregationContext) {
this.collectors = collectors.toArray(new Aggregator[collectors.size()]);
this.aggregationContext = aggregationContext;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
aggregationContext.setScorer(scorer);
}
@Override
public boolean needsScores() {
return aggregationContext.needsScores();
}
@Override
public void collect(int doc) throws IOException {
for (Aggregator collector : collectors) {
collector.collect(doc, 0);
}
}
@Override
public void doSetNextReader(LeafReaderContext context) throws IOException {
aggregationContext.setNextReader(context);
}
@Override
public void postCollection() throws IOException {
for (Aggregator collector : collectors) {
collector.postCollection();
}
}
}
} }

View File

@ -16,355 +16,27 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.search.aggregations; package org.elasticsearch.search.aggregations;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Scorer;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import org.elasticsearch.search.query.QueryPhaseExecutionException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* An Aggregator.
*/
// IMPORTANT: DO NOT add methods to this class unless strictly required.
// On the other hand, if you can remove methods from it, you are highly welcome!
public abstract class Aggregator extends BucketCollector implements Releasable { public abstract class Aggregator extends BucketCollector implements Releasable {
/**
* Returns whether one of the parents is a {@link BucketsAggregator}.
*/
public static boolean descendsFromBucketAggregator(Aggregator parent) {
while (parent != null) {
if (parent instanceof BucketsAggregator) {
return true;
}
parent = parent.parent;
}
return false;
}
private static final Predicate<Aggregator> COLLECTABLE_AGGREGATOR = new Predicate<Aggregator>() {
@Override
public boolean apply(Aggregator aggregator) {
return aggregator.shouldCollect();
}
};
public static final ParseField COLLECT_MODE = new ParseField("collect_mode");
public enum SubAggCollectionMode {
/**
* Creates buckets and delegates to child aggregators in a single pass over
* the matching documents
*/
DEPTH_FIRST(new ParseField("depth_first")),
/**
* Creates buckets for all matching docs and then prunes to top-scoring buckets
* before a second pass over the data when child aggregators are called
* but only for docs from the top-scoring buckets
*/
BREADTH_FIRST(new ParseField("breadth_first"));
private final ParseField parseField;
SubAggCollectionMode(ParseField parseField) {
this.parseField = parseField;
}
public ParseField parseField() {
return parseField;
}
public static SubAggCollectionMode parse(String value) {
return parse(value, ParseField.EMPTY_FLAGS);
}
public static SubAggCollectionMode parse(String value, EnumSet<ParseField.Flag> flags) {
SubAggCollectionMode[] modes = SubAggCollectionMode.values();
for (SubAggCollectionMode mode : modes) {
if (mode.parseField.match(value, flags)) {
return mode;
}
}
throw new ElasticsearchParseException("No " + COLLECT_MODE.getPreferredName() + " found for value [" + value + "]");
}
}
// A scorer used for the deferred collection mode to handle any child aggs asking for scores that are not
// recorded.
static final Scorer unavailableScorer=new Scorer(null){
private final String MSG = "A limitation of the " + SubAggCollectionMode.BREADTH_FIRST.parseField.getPreferredName()
+ " collection mode is that scores cannot be buffered along with document IDs";
@Override
public float score() throws IOException {
throw new ElasticsearchParseException(MSG);
}
@Override
public int freq() throws IOException {
throw new ElasticsearchParseException(MSG);
}
@Override
public int advance(int arg0) throws IOException {
throw new ElasticsearchParseException(MSG);
}
@Override
public long cost() {
throw new ElasticsearchParseException(MSG);
}
@Override
public int docID() {
throw new ElasticsearchParseException(MSG);
}
@Override
public int nextDoc() throws IOException {
throw new ElasticsearchParseException(MSG);
}};
protected final String name;
protected final Aggregator parent;
protected final AggregationContext context;
protected final BigArrays bigArrays;
protected final int depth;
private final Map<String, Object> metaData;
protected final AggregatorFactories factories;
protected final Aggregator[] subAggregators;
protected BucketCollector collectableSubAggregators;
private Map<String, Aggregator> subAggregatorbyName;
private DeferringBucketCollector recordingWrapper;
/**
* Constructs a new Aggregator.
*
* @param name The name of the aggregation
* @param factories The factories for all the sub-aggregators under this aggregator
* @param context The aggregation context
* @param parent The parent aggregator (may be {@code null} for top level aggregators)
* @param metaData The metaData associated with this aggregator
*/
protected Aggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
this.name = name;
this.metaData = metaData;
this.parent = parent;
this.context = context;
this.bigArrays = context.bigArrays();
this.depth = parent == null ? 0 : 1 + parent.depth();
assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
this.factories = factories;
this.subAggregators = factories.createSubAggregators(this);
context.searchContext().addReleasable(this, Lifetime.PHASE);
// Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call)
collectableSubAggregators = new BucketCollector() {
void badState(){
throw new QueryPhaseExecutionException(Aggregator.this.context.searchContext(),
"preCollection not called on new Aggregator before use", null);
}
@Override
public void setNextReader(LeafReaderContext reader) {
badState();
}
@Override
public void preCollection() throws IOException {
badState();
}
@Override
public void postCollection() throws IOException {
badState();
}
@Override
public void collect(int docId, long bucketOrdinal) throws IOException {
badState();
}
@Override
public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) {
badState();
}
};
}
public Map<String, Object> metaData() {
return this.metaData;
}
/**
* Can be overriden by aggregator implementation to be called back when the collection phase starts.
*/
protected void doPreCollection() throws IOException {
}
public final void preCollection() throws IOException {
Iterable<Aggregator> collectables = Iterables.filter(Arrays.asList(subAggregators), COLLECTABLE_AGGREGATOR);
List<BucketCollector> nextPassCollectors = new ArrayList<>();
List<BucketCollector> thisPassCollectors = new ArrayList<>();
for (Aggregator aggregator : collectables) {
if (shouldDefer(aggregator)) {
nextPassCollectors.add(aggregator);
} else {
thisPassCollectors.add(aggregator);
}
}
if (nextPassCollectors.size() > 0) {
BucketCollector deferreds = BucketCollector.wrap(nextPassCollectors);
recordingWrapper = new DeferringBucketCollector(deferreds, context);
// TODO. Without line below we are dependent on subclass aggs
// delegating setNextReader calls on to child aggs
// which they don't seem to do as a matter of course. Need to move
// to a delegation model rather than broadcast
context.registerReaderContextAware(recordingWrapper);
thisPassCollectors.add(recordingWrapper);
}
collectableSubAggregators = BucketCollector.wrap(thisPassCollectors);
collectableSubAggregators.preCollection();
doPreCollection();
}
/**
* This method should be overidden by subclasses that want to defer calculation
* of a child aggregation until a first pass is complete and a set of buckets has
* been pruned.
* Deferring collection will require the recording of all doc/bucketIds from the first
* pass and then the sub class should call {@link #runDeferredCollections(long...)}
* for the selected set of buckets that survive the pruning.
* @param aggregator the child aggregator
* @return true if the aggregator should be deferred
* until a first pass at collection has completed
*/
protected boolean shouldDefer(Aggregator aggregator) {
return false;
}
protected void runDeferredCollections(long... bucketOrds){
// Being lenient here - ignore calls where there are no deferred collections to playback
if (recordingWrapper != null) {
context.setScorer(unavailableScorer);
recordingWrapper.prepareSelectedBuckets(bucketOrds);
}
}
/**
* @return The name of the aggregation.
*/
public String name() {
return name;
}
/** Return the depth of this aggregator in the aggregation tree. */
public final int depth() {
return depth;
}
/**
* @return The parent aggregator of this aggregator. The addAggregation are hierarchical in the sense that some can
* be composed out of others (more specifically, bucket addAggregation can define other addAggregation that will
* be aggregated per bucket). This method returns the direct parent aggregator that contains this aggregator, or
* {@code null} if there is none (meaning, this aggregator is a top level one)
*/
public Aggregator parent() {
return parent;
}
public Aggregator[] subAggregators() {
return subAggregators;
}
public Aggregator subAggregator(String aggName) {
if (subAggregatorbyName == null) {
subAggregatorbyName = new HashMap<>(subAggregators.length);
for (int i = 0; i < subAggregators.length; i++) {
subAggregatorbyName.put(subAggregators[i].name, subAggregators[i]);
}
}
return subAggregatorbyName.get(aggName);
}
/**
* @return The current aggregation context.
*/
public AggregationContext context() {
return context;
}
/**
* @return Whether this aggregator is in the state where it can collect documents. Some aggregators can do their aggregations without
* actually collecting documents, for example, an aggregator that computes stats over unmapped fields doesn't need to collect
* anything as it knows to just return "empty" stats as the aggregation result.
*/
public abstract boolean shouldCollect();
/**
* Called after collection of all document is done.
*/
public final void postCollection() throws IOException {
collectableSubAggregators.postCollection();
doPostCollection();
}
/** Called upon release of the aggregator. */
@Override
public void close() {
try (Releasable _ = recordingWrapper) {
doClose();
}
}
/** Release instance-specific data. */
protected void doClose() {}
/**
* Can be overriden by aggregator implementation to be called back when the collection phase ends.
*/
protected void doPostCollection() throws IOException {
}
/**
* @return The aggregated & built aggregation
*/
public abstract InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException;
@Override
public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException {
results.add(buildAggregation(bucketOrdinal));
}
public abstract InternalAggregation buildEmptyAggregation();
protected final InternalAggregations buildEmptySubAggregations() {
List<InternalAggregation> aggs = new ArrayList<>();
for (Aggregator aggregator : subAggregators) {
aggs.add(aggregator.buildEmptyAggregation());
}
return new InternalAggregations(aggs);
}
/** /**
* Parses the aggregation request and creates the appropriate aggregator factory for it. * Parses the aggregation request and creates the appropriate aggregator factory for it.
* *
@ -391,4 +63,89 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
} }
/**
* Returns whether one of the parents is a {@link BucketsAggregator}.
*/
public static boolean descendsFromBucketAggregator(Aggregator parent) {
while (parent != null) {
if (parent instanceof BucketsAggregator) {
return true;
}
parent = parent.parent();
}
return false;
}
/**
* Return the name of this aggregator.
*/
public abstract String name();
/**
* Return the {@link AggregationContext} attached with this {@link Aggregator}.
*/
public abstract AggregationContext context();
/**
* Return the parent aggregator.
*/
public abstract Aggregator parent();
/**
* Return the sub aggregator with the provided name.
*/
public abstract Aggregator subAggregator(String name);
/**
* Build an aggregation for data that has been collected into {@code bucket}.
*/
public abstract InternalAggregation buildAggregation(long bucket) throws IOException;
/**
* Build an empty aggregation.
*/
public abstract InternalAggregation buildEmptyAggregation();
/** Aggregation mode for sub aggregations. */
public enum SubAggCollectionMode {
/**
* Creates buckets and delegates to child aggregators in a single pass over
* the matching documents
*/
DEPTH_FIRST(new ParseField("depth_first")),
/**
* Creates buckets for all matching docs and then prunes to top-scoring buckets
* before a second pass over the data when child aggregators are called
* but only for docs from the top-scoring buckets
*/
BREADTH_FIRST(new ParseField("breadth_first"));
public static final ParseField KEY = new ParseField("collect_mode");
private final ParseField parseField;
SubAggCollectionMode(ParseField parseField) {
this.parseField = parseField;
}
public ParseField parseField() {
return parseField;
}
public static SubAggCollectionMode parse(String value) {
return parse(value, ParseField.EMPTY_FLAGS);
}
public static SubAggCollectionMode parse(String value, EnumSet<ParseField.Flag> flags) {
SubAggCollectionMode[] modes = SubAggCollectionMode.values();
for (SubAggCollectionMode mode : modes) {
if (mode.parseField.match(value, flags)) {
return mode;
}
}
throw new ElasticsearchParseException("No " + KEY.getPreferredName() + " found for value [" + value + "]");
}
}
} }

View File

@ -0,0 +1,247 @@
/*
* 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;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import org.elasticsearch.search.query.QueryPhaseExecutionException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Base implementation for concrete aggregators.
*/
public abstract class AggregatorBase extends Aggregator {
protected final String name;
protected final Aggregator parent;
protected final AggregationContext context;
private final Map<String, Object> metaData;
protected final Aggregator[] subAggregators;
protected BucketCollector collectableSubAggregators;
private Map<String, Aggregator> subAggregatorbyName;
private DeferringBucketCollector recordingWrapper;
/**
* Constructs a new Aggregator.
*
* @param name The name of the aggregation
* @param factories The factories for all the sub-aggregators under this aggregator
* @param context The aggregation context
* @param parent The parent aggregator (may be {@code null} for top level aggregators)
* @param metaData The metaData associated with this aggregator
*/
protected AggregatorBase(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
this.name = name;
this.metaData = metaData;
this.parent = parent;
this.context = context;
assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
this.subAggregators = factories.createSubAggregators(this);
context.searchContext().addReleasable(this, Lifetime.PHASE);
// Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call)
collectableSubAggregators = new BucketCollector() {
void badState(){
throw new QueryPhaseExecutionException(AggregatorBase.this.context.searchContext(),
"preCollection not called on new Aggregator before use", null);
}
@Override
public LeafBucketCollector getLeafCollector(LeafReaderContext reader) {
badState();
assert false;
return null; // unreachable but compiler does not agree
}
@Override
public void preCollection() throws IOException {
badState();
}
@Override
public void postCollection() throws IOException {
badState();
}
@Override
public boolean needsScores() {
badState();
return false; // unreachable
}
};
}
/**
* Most aggregators don't need scores, make sure to extend this method if
* your aggregator needs them.
*/
@Override
public boolean needsScores() {
return collectableSubAggregators.needsScores();
}
public Map<String, Object> metaData() {
return this.metaData;
}
/**
* Get a {@link LeafBucketCollector} for the given ctx, which should
* delegate to the given collector.
*/
protected abstract LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException;
@Override
public final LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException {
final LeafBucketCollector sub = collectableSubAggregators.getLeafCollector(ctx);
return getLeafCollector(ctx, sub);
}
/**
* Can be overriden by aggregator implementation to be called back when the collection phase starts.
*/
protected void doPreCollection() throws IOException {
}
@Override
public final void preCollection() throws IOException {
List<BucketCollector> collectors = new ArrayList<>();
List<BucketCollector> deferredCollectors = new ArrayList<>();
for (int i = 0; i < subAggregators.length; ++i) {
if (shouldDefer(subAggregators[i])) {
if (recordingWrapper == null) {
recordingWrapper = new DeferringBucketCollector();
}
deferredCollectors.add(subAggregators[i]);
subAggregators[i] = recordingWrapper.wrap(subAggregators[i]);
} else {
collectors.add(subAggregators[i]);
}
}
if (recordingWrapper != null) {
recordingWrapper.setDeferredCollector(deferredCollectors);
collectors.add(recordingWrapper);
}
collectableSubAggregators = BucketCollector.wrap(collectors);
doPreCollection();
}
/**
* This method should be overidden by subclasses that want to defer calculation
* of a child aggregation until a first pass is complete and a set of buckets has
* been pruned.
* Deferring collection will require the recording of all doc/bucketIds from the first
* pass and then the sub class should call {@link #runDeferredCollections(long...)}
* for the selected set of buckets that survive the pruning.
* @param aggregator the child aggregator
* @return true if the aggregator should be deferred
* until a first pass at collection has completed
*/
protected boolean shouldDefer(Aggregator aggregator) {
return false;
}
protected final void runDeferredCollections(long... bucketOrds) throws IOException{
// Being lenient here - ignore calls where there are no deferred collections to playback
if (recordingWrapper != null) {
recordingWrapper.replay(bucketOrds);
}
}
/**
* @return The name of the aggregation.
*/
public String name() {
return name;
}
/**
* @return The parent aggregator of this aggregator. The addAggregation are hierarchical in the sense that some can
* be composed out of others (more specifically, bucket addAggregation can define other addAggregation that will
* be aggregated per bucket). This method returns the direct parent aggregator that contains this aggregator, or
* {@code null} if there is none (meaning, this aggregator is a top level one)
*/
public Aggregator parent() {
return parent;
}
public Aggregator[] subAggregators() {
return subAggregators;
}
public Aggregator subAggregator(String aggName) {
if (subAggregatorbyName == null) {
subAggregatorbyName = new HashMap<>(subAggregators.length);
for (int i = 0; i < subAggregators.length; i++) {
subAggregatorbyName.put(subAggregators[i].name(), subAggregators[i]);
}
}
return subAggregatorbyName.get(aggName);
}
/**
* @return The current aggregation context.
*/
public AggregationContext context() {
return context;
}
/**
* Called after collection of all document is done.
*/
@Override
public final void postCollection() throws IOException {
// post-collect this agg before subs to make it possible to buffer and then replay in postCollection()
doPostCollection();
collectableSubAggregators.postCollection();
}
/** Called upon release of the aggregator. */
@Override
public void close() {
doClose();
}
/** Release instance-specific data. */
protected void doClose() {}
/**
* Can be overriden by aggregator implementation to be called back when the collection phase ends.
*/
protected void doPostCollection() throws IOException {
}
protected final InternalAggregations buildEmptySubAggregations() {
List<InternalAggregation> aggs = new ArrayList<>();
for (Aggregator aggregator : subAggregators) {
aggs.add(aggregator.buildEmptyAggregation());
}
return new InternalAggregations(aggs);
}
@Override
public String toString() {
return name;
}
}

View File

@ -46,11 +46,8 @@ public class AggregatorFactories {
private static Aggregator createAndRegisterContextAware(AggregationContext context, AggregatorFactory factory, Aggregator parent, boolean collectsFromSingleBucket) throws IOException { private static Aggregator createAndRegisterContextAware(AggregationContext context, AggregatorFactory factory, Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
final Aggregator aggregator = factory.create(context, parent, collectsFromSingleBucket); final Aggregator aggregator = factory.create(context, parent, collectsFromSingleBucket);
if (aggregator.shouldCollect()) {
context.registerReaderContextAware(aggregator);
}
// Once the aggregator is fully constructed perform any initialisation - // Once the aggregator is fully constructed perform any initialisation -
// can't do everything in constructors if Aggregator base class needs // can't do everything in constructors if Aggregator base class needs
// to delegate to subclasses as part of construction. // to delegate to subclasses as part of construction.
aggregator.preCollection(); aggregator.preCollection();
return aggregator; return aggregator;
@ -99,15 +96,6 @@ public class AggregatorFactories {
} }
} }
public boolean needsScores() {
for (AggregatorFactory factory : factories) {
if (factory.needsScores()) {
return true;
}
}
return false;
}
private final static class Empty extends AggregatorFactories { private final static class Empty extends AggregatorFactories {
private static final AggregatorFactory[] EMPTY_FACTORIES = new AggregatorFactory[0]; private static final AggregatorFactory[] EMPTY_FACTORIES = new AggregatorFactory[0];

View File

@ -19,10 +19,12 @@
package org.elasticsearch.search.aggregations; package org.elasticsearch.search.aggregations;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchIllegalStateException; import org.apache.lucene.search.Scorer;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
@ -90,8 +92,7 @@ public abstract class AggregatorFactory {
* @return The created aggregator * @return The created aggregator
*/ */
public final Aggregator create(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket) throws IOException { public final Aggregator create(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
Aggregator aggregator = createInternal(context, parent, collectsFromSingleBucket, this.metaData); return createInternal(context, parent, collectsFromSingleBucket, this.metaData);
return aggregator;
} }
public void doValidate() { public void doValidate() {
@ -101,33 +102,52 @@ public abstract class AggregatorFactory {
this.metaData = metaData; this.metaData = metaData;
} }
public boolean needsScores() {
return factories.needsScores();
}
/** /**
* Utility method. Given an {@link AggregatorFactory} that creates {@link Aggregator}s that only know how * Utility method. Given an {@link AggregatorFactory} that creates {@link Aggregator}s that only know how
* to collect bucket <tt>0</tt>, this returns an aggregator that can collect any bucket. * to collect bucket <tt>0</tt>, this returns an aggregator that can collect any bucket.
*/ */
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final AggregationContext context, Aggregator parent) throws IOException { protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final AggregationContext context, final Aggregator parent) throws IOException {
final Aggregator first = factory.create(context, parent, true); final Aggregator first = factory.create(context, parent, true);
return new Aggregator(first.name(), AggregatorFactories.EMPTY, first.context(), first.parent(), first.metaData()) { final BigArrays bigArrays = context.bigArrays();
return new Aggregator() {
ObjectArray<Aggregator> aggregators; ObjectArray<Aggregator> aggregators;
LeafReaderContext readerContext; ObjectArray<LeafBucketCollector> collectors;
{ {
context.searchContext().addReleasable(this, Lifetime.PHASE);
aggregators = bigArrays.newObjectArray(1); aggregators = bigArrays.newObjectArray(1);
aggregators.set(0, first); aggregators.set(0, first);
collectors = bigArrays.newObjectArray(1);
} }
@Override @Override
public boolean shouldCollect() { public String name() {
return first.shouldCollect(); return first.name();
} }
@Override @Override
protected void doPreCollection() throws IOException { public AggregationContext context() {
return first.context();
}
@Override
public Aggregator parent() {
return first.parent();
}
@Override
public boolean needsScores() {
return first.needsScores();
}
@Override
public Aggregator subAggregator(String name) {
throw new UnsupportedOperationException();
}
@Override
public void preCollection() throws IOException {
for (long i = 0; i < aggregators.size(); ++i) { for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i); final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) { if (aggregator != null) {
@ -137,7 +157,7 @@ public abstract class AggregatorFactory {
} }
@Override @Override
protected void doPostCollection() throws IOException { public void postCollection() throws IOException {
for (long i = 0; i < aggregators.size(); ++i) { for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i); final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) { if (aggregator != null) {
@ -147,32 +167,50 @@ public abstract class AggregatorFactory {
} }
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx) {
aggregators = bigArrays.grow(aggregators, owningBucketOrdinal + 1); for (long i = 0; i < collectors.size(); ++i) {
Aggregator aggregator = aggregators.get(owningBucketOrdinal); collectors.set(i, null);
if (aggregator == null) {
aggregator = factory.create(context, parent, true);
aggregator.preCollection();
aggregator.setNextReader(readerContext);
aggregators.set(owningBucketOrdinal, aggregator);
} }
aggregator.collect(doc, 0); return new LeafBucketCollector() {
Scorer scorer;
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
}
@Override
public void collect(int doc, long bucket) throws IOException {
aggregators = bigArrays.grow(aggregators, bucket + 1);
collectors = bigArrays.grow(collectors, bucket + 1);
LeafBucketCollector collector = collectors.get(bucket);
if (collector == null) {
Aggregator aggregator = aggregators.get(bucket);
if (aggregator == null) {
aggregator = factory.create(context, parent, true);
aggregator.preCollection();
aggregators.set(bucket, aggregator);
}
collector = aggregator.getLeafCollector(ctx);
collector.setScorer(scorer);
collectors.set(bucket, collector);
}
collector.collect(doc, 0);
}
};
} }
@Override @Override
public void setNextReader(LeafReaderContext context) throws IOException { public InternalAggregation buildAggregation(long bucket) throws IOException {
this.readerContext = context; if (bucket < aggregators.size()) {
for (long i = 0; i < aggregators.size(); ++i) { Aggregator aggregator = aggregators.get(bucket);
final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) { if (aggregator != null) {
aggregator.setNextReader(context); return aggregator.buildAggregation(0);
} }
} }
} return buildEmptyAggregation();
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
throw new ElasticsearchIllegalStateException("Invalid context - aggregation must use addResults() to collect child results");
} }
@Override @Override
@ -181,19 +219,8 @@ public abstract class AggregatorFactory {
} }
@Override @Override
public void doClose() { public void close() {
Releasables.close(aggregators); Releasables.close(aggregators, collectors);
}
@Override
public void gatherAnalysis(BucketAnalysisCollector results, long owningBucketOrdinal) throws IOException {
// The bucket ordinal may be out of range in case of eg. a terms/filter/terms where
// the filter matches no document in the highest buckets of the first terms agg
if (owningBucketOrdinal >= aggregators.size() || aggregators.get(owningBucketOrdinal) == null) {
results.add(first.buildEmptyAggregation());
} else {
aggregators.get(owningBucketOrdinal).gatherAnalysis(results,0);
}
} }
}; };
} }

View File

@ -19,17 +19,20 @@
package org.elasticsearch.search.aggregations; package org.elasticsearch.search.aggregations;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.lucene.ReaderContextAware; import org.elasticsearch.common.lucene.search.XCollector;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/** /**
* A Collector that can collect data in separate buckets. * A Collector that can collect data in separate buckets.
*/ */
public abstract class BucketCollector implements ReaderContextAware { public abstract class BucketCollector implements XCollector {
/** /**
* Used to gather a summary from a bucket * Used to gather a summary from a bucket
@ -45,12 +48,8 @@ public abstract class BucketCollector implements ReaderContextAware {
public final static BucketCollector NO_OP_COLLECTOR = new BucketCollector() { public final static BucketCollector NO_OP_COLLECTOR = new BucketCollector() {
@Override @Override
public void collect(int docId, long bucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext reader) {
// no-op return LeafBucketCollector.NO_OP_COLLECTOR;
}
@Override
public void setNextReader(LeafReaderContext reader) {
// no-op
} }
public void preCollection() throws IOException { public void preCollection() throws IOException {
// no-op // no-op
@ -59,9 +58,8 @@ public abstract class BucketCollector implements ReaderContextAware {
public void postCollection() throws IOException { public void postCollection() throws IOException {
// no-op // no-op
} }
@Override public boolean needsScores() {
public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) { return false;
// no-op
} }
}; };
@ -79,17 +77,12 @@ public abstract class BucketCollector implements ReaderContextAware {
return new BucketCollector() { return new BucketCollector() {
@Override @Override
public void collect(int docId, long bucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException {
for (BucketCollector collector : collectors) { List<LeafBucketCollector> leafCollectors = new ArrayList<>();
collector.collect(docId, bucketOrdinal); for (BucketCollector c : collectors) {
} leafCollectors.add(c.getLeafCollector(ctx));
}
@Override
public void setNextReader(LeafReaderContext reader) throws IOException {
for (BucketCollector collector : collectors) {
collector.setNextReader(reader);
} }
return LeafBucketCollector.wrap(leafCollectors);
} }
@Override @Override
@ -107,43 +100,27 @@ public abstract class BucketCollector implements ReaderContextAware {
} }
@Override @Override
public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException { public boolean needsScores() {
for (BucketCollector collector : collectors) { for (BucketCollector collector : collectors) {
collector.gatherAnalysis(results, bucketOrdinal); if (collector.needsScores()) {
return true;
}
} }
return false;
} }
}; };
} }
} }
/** @Override
* Called during the query phase, to collect & aggregate the given document. public abstract LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException;
*
* @param doc The document to be collected/aggregated
* @param bucketOrdinal The ordinal of the bucket this aggregator belongs to, assuming this aggregator is not a top level aggregator.
* Typically, aggregators with {@code #bucketAggregationMode} set to {@link BucketAggregationMode#MULTI_BUCKETS}
* will heavily depend on this ordinal. Other aggregators may or may not use it and can see this ordinal as just
* an extra information for the aggregation context. For top level aggregators, the ordinal will always be
* equal to 0.
* @throws IOException
*/
public abstract void collect(int docId, long bucketOrdinal) throws IOException;
/** /**
* Pre collection callback. * Pre collection callback.
*/ */
public abstract void preCollection() throws IOException; public abstract void preCollection() throws IOException;
/** @Override
* Post collection callback.
*/
public abstract void postCollection() throws IOException; public abstract void postCollection() throws IOException;
/**
* Called post-collection to gather the results from surviving buckets.
* @param analysisCollector
* @param bucketOrdinal
*/
public abstract void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException;
} }

View File

@ -1,91 +0,0 @@
/*
* 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;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.LongHash;
import java.io.IOException;
/**
* Filters a collection stream of docIds and related buckets using a sorted
* list of valid bucket ordinals.
*/
public class FilteringBucketCollector extends BucketCollector implements Releasable {
private final LongHash denseMap;
private final BucketCollector delegate;
/**
*
* @param the valid BucketOrds
* @param delegate The collector that will be called for any buckets listed in sortedBucketOrds
*/
public FilteringBucketCollector(long[] validBucketOrds, BucketCollector delegate, BigArrays bigArrays) {
denseMap = new LongHash(validBucketOrds.length, bigArrays);
for (int i = 0; i < validBucketOrds.length; i++) {
denseMap.add(validBucketOrds[i]);
}
this.delegate = delegate;
}
@Override
public final void setNextReader(LeafReaderContext reader) throws IOException {
delegate.setNextReader(reader);
}
@Override
public final void collect(int docId, long bucketOrdinal) throws IOException {
long ordinal = denseMap.find(bucketOrdinal);
if (ordinal >= 0) {
delegate.collect(docId, ordinal);
}
}
@Override
public void preCollection() throws IOException {
delegate.preCollection();
}
@Override
public final void postCollection() throws IOException {
delegate.postCollection();
}
@Override
public void close() throws ElasticsearchException {
Releasables.close(denseMap);
}
@Override
public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException{
long ordinal = denseMap.find(bucketOrdinal);
if (ordinal >= 0) {
delegate.gatherAnalysis(analysisCollector, ordinal);
} else {
throw new ElasticsearchIllegalArgumentException("Aggregation requested on a missing bucket #" + bucketOrdinal);
}
}
}

View File

@ -0,0 +1,95 @@
/*
* 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;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Scorer;
import java.io.IOException;
/**
* Per-leaf bucket collector.
*/
public abstract class LeafBucketCollector implements LeafCollector {
public static final LeafBucketCollector NO_OP_COLLECTOR = new LeafBucketCollector() {
@Override
public void setScorer(Scorer arg0) throws IOException {
// no-op
}
@Override
public void collect(int doc, long bucket) {
// no-op
}
};
public static LeafBucketCollector wrap(Iterable<LeafBucketCollector> collectors) {
final Iterable<LeafBucketCollector> actualCollectors = Iterables.filter(collectors,
new Predicate<LeafBucketCollector> () {
@Override
public boolean apply(LeafBucketCollector c) {
return c != NO_OP_COLLECTOR;
}
});
final LeafBucketCollector[] colls = Iterables.toArray(actualCollectors, LeafBucketCollector.class);
switch (colls.length) {
case 0:
return NO_OP_COLLECTOR;
case 1:
return colls[0];
default:
return new LeafBucketCollector() {
@Override
public void setScorer(Scorer s) throws IOException {
for (LeafBucketCollector c : colls) {
c.setScorer(s);
}
}
@Override
public void collect(int doc, long bucket) throws IOException {
for (LeafBucketCollector c : colls) {
c.collect(doc, bucket);
}
}
};
}
}
/**
* Collect the given doc in the given bucket.
*/
public abstract void collect(int doc, long bucket) throws IOException;
@Override
public final void collect(int doc) throws IOException {
collect(doc, 0);
}
@Override
public void setScorer(Scorer scorer) throws IOException {
// no-op by default
}
}

View File

@ -0,0 +1,63 @@
/*
* 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;
import org.apache.lucene.search.Scorer;
import org.elasticsearch.common.lucene.ScorerAware;
import java.io.IOException;
/**
* A {@link LeafBucketCollector} that delegates all calls to the sub leaf
* aggregator and sets the scorer on its source of values if it implements
* {@link ScorerAware}.
*/
public class LeafBucketCollectorBase extends LeafBucketCollector {
private final LeafBucketCollector sub;
private final ScorerAware values;
/**
* @param sub The leaf collector for sub aggregations.
* @param values The values. {@link ScorerAware#setScorer} will be called automatically on them if they implement {@link ScorerAware}.
*/
public LeafBucketCollectorBase(LeafBucketCollector sub, Object values) {
this.sub = sub;
if (values instanceof ScorerAware) {
this.values = (ScorerAware) values;
} else {
this.values = null;
}
}
@Override
public void setScorer(Scorer s) throws IOException {
sub.setScorer(s);
if (values != null) {
values.setScorer(s);
}
}
@Override
public void collect(int doc, long bucket) throws IOException {
sub.collect(doc, bucket);
}
}

View File

@ -29,7 +29,7 @@ import java.util.Map;
* An aggregator that is not collected, this can typically be used when running an aggregation over a field that doesn't have * An aggregator that is not collected, this can typically be used when running an aggregation over a field that doesn't have
* a mapping. * a mapping.
*/ */
public abstract class NonCollectingAggregator extends Aggregator { public abstract class NonCollectingAggregator extends AggregatorBase {
protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, Map<String, Object> metaData) throws IOException { protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, Map<String, Object> metaData) throws IOException {
super(name, subFactories, context, parent, metaData); super(name, subFactories, context, parent, metaData);
@ -39,28 +39,14 @@ public abstract class NonCollectingAggregator extends Aggregator {
this(name, context, parent, AggregatorFactories.EMPTY, metaData); this(name, context, parent, AggregatorFactories.EMPTY, metaData);
} }
private void fail() {
throw new IllegalStateException("This aggregator should not be collected");
}
@Override @Override
public final void setNextReader(LeafReaderContext reader) { public final LeafBucketCollector getLeafCollector(LeafReaderContext reader, LeafBucketCollector sub) {
fail(); // the framework will automatically eliminate it
} return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public final boolean shouldCollect() {
return false;
}
@Override
public final void collect(int doc, long owningBucketOrdinal) throws IOException {
fail();
} }
@Override @Override
public final InternalAggregation buildAggregation(long owningBucketOrdinal) { public final InternalAggregation buildAggregation(long owningBucketOrdinal) {
return buildEmptyAggregation(); return buildEmptyAggregation();
} }
} }

View File

@ -1,43 +0,0 @@
/*
* 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;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.lease.Releasable;
import java.io.IOException;
/**
* Abstraction for implementations that record a "collect" stream for subsequent play-back
*/
public abstract class RecordingBucketCollector extends BucketCollector implements Releasable {
/**
* Replay a previously executed set of calls to the {@link #collect(int, long)} method
* @param collector the object which will be called to handle the playback
* @throws IOException
*/
public abstract void replayCollection(BucketCollector collector) throws IOException;
@Override
public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) {
throw new ElasticsearchIllegalStateException("gatherAnalysis not supported");
}
}

View File

@ -1,160 +0,0 @@
/*
* 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;
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 java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
* Records a "collect" stream for subsequent play-back using a per-segment
* object to collect matches. Playback is fast because each collection
* contains only the required matches for the current reader.
*/
public class RecordingPerReaderBucketCollector extends RecordingBucketCollector {
final List<PerSegmentCollects> perSegmentCollections = new ArrayList<>();
private PerSegmentCollects currentCollection;
private boolean recordingComplete;
static class PerSegmentCollects {
LeafReaderContext readerContext;
PackedLongValues.Builder docs;
PackedLongValues.Builder buckets;
int lastDocId = 0;
PerSegmentCollects(LeafReaderContext readerContext) {
this.readerContext = readerContext;
}
void collect(int doc, long owningBucketOrdinal) throws IOException {
if (docs == null) {
// TODO unclear what might be reasonable constructor args to pass to this collection
// No way of accurately predicting how many docs will be collected
docs = PackedLongValues.packedBuilder(PackedInts.COMPACT);
}
// Store as delta-encoded for better compression
docs.add(doc - lastDocId);
lastDocId = doc;
if (buckets == null) {
if (owningBucketOrdinal != 0) {
// Store all of the prior bucketOrds (which up until now have
// all been zero based)
buckets = PackedLongValues.packedBuilder(PackedInts.COMPACT);
for (int i = 0; i < docs.size() - 1; i++) {
buckets.add(0);
}
// record the new non-zero bucketID
buckets.add(owningBucketOrdinal);
}
} else {
buckets.add(owningBucketOrdinal);
}
}
void endCollect() {
}
boolean hasItems() {
return docs != null;
}
void replay(BucketCollector collector) throws IOException {
lastDocId = 0;
collector.setNextReader(readerContext);
if (!hasItems()) {
return;
}
if (buckets == null) {
final PackedLongValues.Iterator docsIter = docs.build().iterator();
while (docsIter.hasNext()) {
lastDocId += (int) docsIter.next();
collector.collect(lastDocId, 0);
}
} else {
assert docs.size() == buckets.size();
final PackedLongValues.Iterator docsIter = docs.build().iterator();
final PackedLongValues.Iterator bucketsIter = buckets.build().iterator();
while (docsIter.hasNext()) {
lastDocId += (int) docsIter.next();
collector.collect(lastDocId, bucketsIter.next());
}
}
}
}
@Override
public void setNextReader(LeafReaderContext reader) {
if(recordingComplete){
// The way registration works for listening on reader changes we have the potential to be called > once
// TODO fixup the aggs framework so setNextReader calls are delegated to child aggs and not reliant on
// registering a listener.
return;
}
stowLastSegmentCollection();
currentCollection = new PerSegmentCollects(reader);
}
private void stowLastSegmentCollection() {
if (currentCollection != null) {
if (currentCollection.hasItems()) {
currentCollection.endCollect();
perSegmentCollections.add(currentCollection);
}
currentCollection = null;
}
}
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
currentCollection.collect(doc, owningBucketOrdinal);
}
/*
* Allows clients to replay a stream of collected items.
*
*/
@Override
public void replayCollection(BucketCollector collector) throws IOException{
for (PerSegmentCollects collection : perSegmentCollections) {
collection.replay(collector);
}
collector.postCollection();
}
@Override
public void preCollection() throws IOException {
// nothing to do
}
@Override
public void postCollection() throws IOException {
recordingComplete = true;
stowLastSegmentCollection();
}
@Override
public void close() throws ElasticsearchException {
}
}

View File

@ -19,73 +19,73 @@
package org.elasticsearch.search.aggregations.bucket; package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.IntArray; import org.elasticsearch.common.util.IntArray;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map; import java.util.Map;
/** /**
* *
*/ */
public abstract class BucketsAggregator extends Aggregator { public abstract class BucketsAggregator extends AggregatorBase {
private final BigArrays bigArrays;
private IntArray docCounts; private IntArray docCounts;
public BucketsAggregator(String name, AggregatorFactories factories, public BucketsAggregator(String name, AggregatorFactories factories,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, factories, context, parent, metaData); super(name, factories, context, parent, metaData);
bigArrays = context.bigArrays();
docCounts = bigArrays.newIntArray(1, true); docCounts = bigArrays.newIntArray(1, true);
} }
/** /**
* Return an upper bound of the maximum bucket ordinal seen so far. * Return an upper bound of the maximum bucket ordinal seen so far.
*/ */
protected final long maxBucketOrd() { public final long maxBucketOrd() {
return docCounts.size(); return docCounts.size();
} }
/** /**
* Ensure there are at least <code>maxBucketOrd</code> buckets available. * Ensure there are at least <code>maxBucketOrd</code> buckets available.
*/ */
protected final void grow(long maxBucketOrd) { public final void grow(long maxBucketOrd) {
docCounts = bigArrays.grow(docCounts, maxBucketOrd); docCounts = bigArrays.grow(docCounts, maxBucketOrd);
} }
/** /**
* Utility method to collect the given doc in the given bucket (identified by the bucket ordinal) * Utility method to collect the given doc in the given bucket (identified by the bucket ordinal)
*/ */
protected final void collectBucket(int doc, long bucketOrd) throws IOException { public final void collectBucket(LeafBucketCollector subCollector, int doc, long bucketOrd) throws IOException {
grow(bucketOrd + 1); grow(bucketOrd + 1);
collectExistingBucket(doc, bucketOrd); collectExistingBucket(subCollector, doc, bucketOrd);
} }
/** /**
* Same as {@link #collectBucket(int, long)}, but doesn't check if the docCounts needs to be re-sized. * Same as {@link #collectBucket(int, long)}, but doesn't check if the docCounts needs to be re-sized.
*/ */
protected final void collectExistingBucket(int doc, long bucketOrd) throws IOException { public final void collectExistingBucket(LeafBucketCollector subCollector, int doc, long bucketOrd) throws IOException {
docCounts.increment(bucketOrd, 1); docCounts.increment(bucketOrd, 1);
collectBucketNoCounts(doc, bucketOrd); subCollector.collect(doc, bucketOrd);
} }
public IntArray getDocCounts() { public IntArray getDocCounts() {
return docCounts; return docCounts;
} }
/**
* Utility method to collect the given doc in the given bucket but not to update the doc counts of the bucket
*/
protected final void collectBucketNoCounts(int doc, long bucketOrd) throws IOException {
collectableSubAggregators.collect(doc, bucketOrd);
}
/** /**
* Utility method to increment the doc counts of the given bucket (identified by the bucket ordinal) * Utility method to increment the doc counts of the given bucket (identified by the bucket ordinal)
*/ */
protected final void incrementBucketDocCount(long bucketOrd, int inc) throws IOException { public final void incrementBucketDocCount(long bucketOrd, int inc) {
docCounts = bigArrays.grow(docCounts, bucketOrd + 1); docCounts = bigArrays.grow(docCounts, bucketOrd + 1);
docCounts.increment(bucketOrd, inc); docCounts.increment(bucketOrd, inc);
} }
@ -108,41 +108,13 @@ public abstract class BucketsAggregator extends Aggregator {
/** /**
* Required method to build the child aggregations of the given bucket (identified by the bucket ordinal). * Required method to build the child aggregations of the given bucket (identified by the bucket ordinal).
*/ */
protected final InternalAggregations bucketAggregations(long bucketOrd) throws IOException { protected final InternalAggregations bucketAggregations(long bucket) throws IOException {
final ArrayList<InternalAggregation> childAggs = new ArrayList<>(); final InternalAggregation[] aggregations = new InternalAggregation[subAggregators.length];
final long bucketDocCount = bucketDocCount(bucketOrd); for (int i = 0; i < subAggregators.length; i++) {
if (bucketDocCount == 0L) { aggregations[i] = subAggregators[i].buildAggregation(bucket);
// All child aggs marked as empty
for (int i = 0; i < subAggregators.length; i++) {
childAggs.add(subAggregators[i].buildEmptyAggregation());
}
} else {
BucketAnalysisCollector analysisCollector = new BucketAnalysisCollector() {
@Override
public void add(Aggregation analysis) {
childAggs.add((InternalAggregation) analysis);
}
};
// Add the collectable sub aggs by asking the collect tree to gather
// results using ordinals that may have undergone transformation as the
// result of the collection process e.g. filtering
// to a subset of buckets then rebasing the numbers in the deferred collection
collectableSubAggregators.gatherAnalysis(analysisCollector, bucketOrd);
// Also add the results of any non-collecting sub aggs using the top-level ordinals
for (int i = 0; i < subAggregators.length; i++) {
if (!subAggregators[i].shouldCollect()) {
// Agg is not part of the collect tree - call directly
childAggs.add(subAggregators[i].buildAggregation(bucketOrd));
}
}
} }
return new InternalAggregations(Arrays.asList(aggregations));
return new InternalAggregations(childAggs);
} }
/** /**
* Utility method to build empty aggregations of the sub aggregators. * Utility method to build empty aggregations of the sub aggregators.

View File

@ -20,119 +20,215 @@
package org.elasticsearch.search.aggregations.bucket; package org.elasticsearch.search.aggregations.bucket;
import org.apache.lucene.index.LeafReaderContext; 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.ElasticsearchException;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.search.aggregations.FilteringBucketCollector; import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.search.aggregations.RecordingBucketCollector; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.RecordingPerReaderBucketCollector; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.query.QueryPhaseExecutionException; import org.elasticsearch.search.aggregations.BucketCollector;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/** /**
* Buffers the matches in a collect stream and can replay a subset of the collected buckets * A {@link BucketCollector} that records collected doc IDs and buckets and
* to a deferred set of collectors. * allows to replay a subset of the collected buckets.
* The rationale for not bundling all this logic into {@link RecordingBucketCollector} is to allow
* the possibility of alternative recorder impl choices while keeping the logic in here for
* setting {@link AggregationContext}'s setNextReader method and preparing the appropriate choice
* of filtering logic for stream replay. These felt like agg-specific functions that should be kept away
* from the {@link RecordingBucketCollector} impl which is concentrated on efficient storage of doc and bucket IDs
*/ */
public class DeferringBucketCollector extends BucketCollector implements Releasable { public final class DeferringBucketCollector extends BucketCollector {
private final BucketCollector deferred;
private final RecordingBucketCollector recording;
private final AggregationContext context;
private FilteringBucketCollector filteredCollector;
private static class Entry {
final LeafReaderContext context;
final PackedLongValues docDeltas;
final PackedLongValues buckets;
public DeferringBucketCollector(BucketCollector deferred, AggregationContext context) { public Entry(LeafReaderContext context, PackedLongValues docDeltas, PackedLongValues buckets) {
this.deferred = deferred; this.context = context;
this.recording = new RecordingPerReaderBucketCollector(); this.docDeltas = docDeltas;
this.context = context; this.buckets = buckets;
}
}
final List<Entry> entries = new ArrayList<>();
BucketCollector collector;
LeafReaderContext context;
PackedLongValues.Builder docDeltas;
PackedLongValues.Builder buckets;
long maxBucket = -1;
boolean finished = false;
LongHash selectedBuckets;
/** Sole constructor. */
public DeferringBucketCollector() {}
@Override
public boolean needsScores() {
if (collector == null) {
throw new ElasticsearchIllegalStateException();
}
return collector.needsScores();
}
/** Set the deferred collectors. */
public void setDeferredCollector(Iterable<BucketCollector> deferredCollectors) {
this.collector = BucketCollector.wrap(deferredCollectors);
}
private void finishLeaf() {
if (context != null) {
entries.add(new Entry(context, docDeltas.build(), buckets.build()));
}
context = null;
docDeltas = null;
buckets = null;
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException {
recording.setNextReader(reader); finishLeaf();
}
@Override context = ctx;
public void collect(int docId, long bucketOrdinal) throws IOException { docDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT);
recording.collect(docId, bucketOrdinal); buckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT);
return new LeafBucketCollector() {
int lastDoc = 0;
@Override
public void collect(int doc, long bucket) throws IOException {
docDeltas.add(doc - lastDoc);
buckets.add(bucket);
lastDoc = doc;
maxBucket = Math.max(maxBucket, bucket);
}
};
} }
@Override @Override
public void preCollection() throws IOException { public void preCollection() throws IOException {
recording.preCollection();
} }
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
recording.postCollection(); finishLeaf();
finished = true;
} }
/** /**
* Plays a selection of the data cached from previous collect calls to the * Replay the wrapped collector, but only on a selection of buckets.
* deferred collector.
*
* @param survivingBucketOrds
* the valid bucket ords for which deferred collection should be
* attempted
*/ */
public void prepareSelectedBuckets(long... survivingBucketOrds) { public void replay(long... selectedBuckets) throws IOException {
if (!finished) {
BucketCollector subs = new BucketCollector() { throw new ElasticsearchIllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called");
}
if (this.selectedBuckets != null) {
throw new ElasticsearchIllegalStateException("Alerady been replayed");
}
final LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE);
for (long bucket : selectedBuckets) {
hash.add(bucket);
}
this.selectedBuckets = hash;
collector.preCollection();
for (Entry entry : entries) {
final LeafBucketCollector leafCollector = collector.getLeafCollector(entry.context);
leafCollector.setScorer(Lucene.illegalScorer("A limitation of the " + SubAggCollectionMode.BREADTH_FIRST
+ " collection mode is that scores cannot be buffered along with document IDs"));
final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator();
final PackedLongValues.Iterator buckets = entry.buckets.iterator();
int doc = 0;
for (long i = 0, end = entry.docDeltas.size(); i < end; ++i) {
doc += docDeltaIterator.next();
final long bucket = buckets.next();
final long rebasedBucket = hash.find(bucket);
if (rebasedBucket != -1) {
leafCollector.collect(doc, rebasedBucket);
}
}
}
collector.postCollection();
}
/**
* Wrap the provided aggregator so that it behaves (almost) as if it had been
* collected directly.
*/
public Aggregator wrap(final Aggregator in) {
return new Aggregator() {
@Override @Override
public void setNextReader(LeafReaderContext reader) throws IOException { public boolean needsScores() {
// Need to set AggregationContext otherwise ValueSources in aggs return in.needsScores();
// don't read any values
context.setNextReader(reader);
deferred.setNextReader(reader);
} }
@Override @Override
public void collect(int docId, long bucketOrdinal) throws IOException { public void close() throws ElasticsearchException {
deferred.collect(docId, bucketOrdinal); in.close();
}
@Override
public String name() {
return in.name();
}
@Override
public Aggregator parent() {
return in.parent();
}
@Override
public AggregationContext context() {
return in.context();
}
@Override
public Aggregator subAggregator(String name) {
return in.subAggregator(name);
}
@Override
public InternalAggregation buildAggregation(long bucket) throws IOException {
if (selectedBuckets == null) {
throw new ElasticsearchIllegalStateException("Collection has not been replayed yet.");
}
final long rebasedBucket = selectedBuckets.find(bucket);
if (rebasedBucket == -1) {
throw new ElasticsearchIllegalStateException("Cannot build for a bucket which has not been collected");
}
return in.buildAggregation(rebasedBucket);
}
@Override
public InternalAggregation buildEmptyAggregation() {
return in.buildEmptyAggregation();
}
@Override
public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException {
throw new ElasticsearchIllegalStateException("Deferred collectors cannot be collected directly. They must be collected through the recording wrapper.");
} }
@Override @Override
public void preCollection() throws IOException { public void preCollection() throws IOException {
deferred.preCollection(); throw new ElasticsearchIllegalStateException("Deferred collectors cannot be collected directly. They must be collected through the recording wrapper.");
} }
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
deferred.postCollection(); throw new ElasticsearchIllegalStateException("Deferred collectors cannot be collected directly. They must be collected through the recording wrapper.");
} }
@Override
public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException {
deferred.gatherAnalysis(results, bucketOrdinal);
}
}; };
filteredCollector = new FilteringBucketCollector(survivingBucketOrds, subs, context.bigArrays());
try {
recording.replayCollection(filteredCollector);
} catch (IOException e) {
throw new QueryPhaseExecutionException(context.searchContext(), "Failed to replay deferred set of matching docIDs", e);
}
} }
}
@Override
public void close() throws ElasticsearchException {
Releasables.close(recording, filteredCollector);
}
@Override
public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException {
filteredCollector.gatherAnalysis(analysisCollector, bucketOrdinal);
}
}

View File

@ -35,9 +35,4 @@ public abstract class SingleBucketAggregator extends BucketsAggregator {
super(name, factories, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
} }
@Override
public boolean shouldCollect() {
return true;
}
} }

View File

@ -24,9 +24,8 @@ import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.ReaderContextAware;
import org.elasticsearch.common.lucene.docset.DocIdSets; import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.common.util.LongArray; import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.common.util.LongObjectPagedHashMap; import org.elasticsearch.common.util.LongObjectPagedHashMap;
@ -34,6 +33,7 @@ import org.elasticsearch.index.search.child.ConstantScorer;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -49,7 +49,7 @@ import java.util.Map;
// The RecordingPerReaderBucketCollector assumes per segment recording which isn't the case for this // The RecordingPerReaderBucketCollector assumes per segment recording which isn't the case for this
// aggregation, for this reason that collector can't be used // aggregation, for this reason that collector can't be used
public class ParentToChildrenAggregator extends SingleBucketAggregator implements ReaderContextAware { public class ParentToChildrenAggregator extends SingleBucketAggregator {
private final String parentType; private final String parentType;
private final Filter childFilter; private final Filter childFilter;
@ -67,8 +67,6 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
private boolean multipleBucketsPerParentOrd = false; private boolean multipleBucketsPerParentOrd = false;
private List<LeafReaderContext> replay = new ArrayList<>(); private List<LeafReaderContext> replay = new ArrayList<>();
private SortedDocValues globalOrdinals;
private Bits parentDocs;
public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext, public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext,
Aggregator parent, String parentType, Filter childFilter, Filter parentFilter, Aggregator parent, String parentType, Filter childFilter, Filter parentFilter,
@ -95,60 +93,62 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
} }
@Override @Override
public void collect(int docId, long bucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
if (parentDocs.get(docId)) { final LeafBucketCollector sub) throws IOException {
long globalOrdinal = globalOrdinals.getOrd(docId); if (valuesSource == null) {
if (globalOrdinal != -1) { return LeafBucketCollector.NO_OP_COLLECTOR;
if (parentOrdToBuckets.get(globalOrdinal) == -1) { }
parentOrdToBuckets.set(globalOrdinal, bucketOrdinal); if (replay == null) {
} else { throw new ElasticsearchIllegalStateException();
long[] bucketOrds = parentOrdToOtherBuckets.get(globalOrdinal); }
if (bucketOrds != null) {
bucketOrds = Arrays.copyOf(bucketOrds, bucketOrds.length + 1); final SortedDocValues globalOrdinals = valuesSource.globalOrdinalsValues(parentType, ctx);
bucketOrds[bucketOrds.length - 1] = bucketOrdinal; assert globalOrdinals != null;
parentOrdToOtherBuckets.put(globalOrdinal, bucketOrds); DocIdSet parentDocIdSet = parentFilter.getDocIdSet(ctx, null);
} else { // The DocIdSets.toSafeBits(...) can convert to FixedBitSet, but this
parentOrdToOtherBuckets.put(globalOrdinal, new long[]{bucketOrdinal}); // will only happen if the none filter cache is used. (which only happens in tests)
// Otherwise the filter cache will produce a bitset based filter.
final Bits parentDocs = DocIdSets.asSequentialAccessBits(ctx.reader().maxDoc(), parentDocIdSet);
DocIdSet childDocIdSet = childFilter.getDocIdSet(ctx, null);
if (DocIdSets.isEmpty(childDocIdSet) == false) {
replay.add(ctx);
}
return new LeafBucketCollector() {
@Override
public void collect(int docId, long bucket) throws IOException {
if (parentDocs.get(docId)) {
long globalOrdinal = globalOrdinals.getOrd(docId);
if (globalOrdinal != -1) {
if (parentOrdToBuckets.get(globalOrdinal) == -1) {
parentOrdToBuckets.set(globalOrdinal, bucket);
} else {
long[] bucketOrds = parentOrdToOtherBuckets.get(globalOrdinal);
if (bucketOrds != null) {
bucketOrds = Arrays.copyOf(bucketOrds, bucketOrds.length + 1);
bucketOrds[bucketOrds.length - 1] = bucket;
parentOrdToOtherBuckets.put(globalOrdinal, bucketOrds);
} else {
parentOrdToOtherBuckets.put(globalOrdinal, new long[]{bucket});
}
multipleBucketsPerParentOrd = true;
}
} }
multipleBucketsPerParentOrd = true;
} }
} }
} };
}
@Override
public void setNextReader(LeafReaderContext reader) {
if (replay == null) {
return;
}
globalOrdinals = valuesSource.globalOrdinalsValues(parentType);
assert globalOrdinals != null;
try {
DocIdSet parentDocIdSet = parentFilter.getDocIdSet(reader, null);
// The DocIdSets.toSafeBits(...) can convert to FixedBitSet, but this
// will only happen if the none filter cache is used. (which only happens in tests)
// Otherwise the filter cache will produce a bitset based filter.
parentDocs = DocIdSets.asSequentialAccessBits(reader.reader().maxDoc(), parentDocIdSet);
DocIdSet childDocIdSet = childFilter.getDocIdSet(reader, null);
if (globalOrdinals != null && !DocIdSets.isEmpty(childDocIdSet)) {
replay.add(reader);
}
} catch (IOException e) {
throw ExceptionsHelper.convertToElastic(e);
}
} }
@Override @Override
protected void doPostCollection() throws IOException { protected void doPostCollection() throws IOException {
List<LeafReaderContext> replay = this.replay; final List<LeafReaderContext> replay = this.replay;
this.replay = null; this.replay = null;
for (LeafReaderContext atomicReaderContext : replay) { for (LeafReaderContext ctx : replay) {
context.setNextReader(atomicReaderContext); final LeafBucketCollector sub = collectableSubAggregators.getLeafCollector(ctx);
SortedDocValues globalOrdinals = valuesSource.globalOrdinalsValues(parentType); final SortedDocValues globalOrdinals = valuesSource.globalOrdinalsValues(parentType, ctx);
DocIdSet childDocIdSet = childFilter.getDocIdSet(atomicReaderContext, atomicReaderContext.reader().getLiveDocs()); DocIdSet childDocIdSet = childFilter.getDocIdSet(ctx, ctx.reader().getLiveDocs());
if (childDocIdSet == null) { if (childDocIdSet == null) {
continue; continue;
} }
@ -158,19 +158,19 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
} }
// Set the scorer, since we now replay only the child docIds // Set the scorer, since we now replay only the child docIds
context.setScorer(ConstantScorer.create(childDocsIter, null, 1f)); sub.setScorer(ConstantScorer.create(childDocsIter, null, 1f));
for (int docId = childDocsIter.nextDoc(); docId != DocIdSetIterator.NO_MORE_DOCS; docId = childDocsIter.nextDoc()) { for (int docId = childDocsIter.nextDoc(); docId != DocIdSetIterator.NO_MORE_DOCS; docId = childDocsIter.nextDoc()) {
long globalOrdinal = globalOrdinals.getOrd(docId); long globalOrdinal = globalOrdinals.getOrd(docId);
if (globalOrdinal != -1) { if (globalOrdinal != -1) {
long bucketOrd = parentOrdToBuckets.get(globalOrdinal); long bucketOrd = parentOrdToBuckets.get(globalOrdinal);
if (bucketOrd != -1) { if (bucketOrd != -1) {
collectBucket(docId, bucketOrd); collectBucket(sub, docId, bucketOrd);
if (multipleBucketsPerParentOrd) { if (multipleBucketsPerParentOrd) {
long[] otherBucketOrds = parentOrdToOtherBuckets.get(globalOrdinal); long[] otherBucketOrds = parentOrdToOtherBuckets.get(globalOrdinal);
if (otherBucketOrds != null) { if (otherBucketOrds != null) {
for (long otherBucketOrd : otherBucketOrds) { for (long otherBucketOrd : otherBucketOrds) {
collectBucket(docId, otherBucketOrd); collectBucket(sub, docId, otherBucketOrd);
} }
} }
} }
@ -178,10 +178,6 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
} }
} }
} }
// Need to invoke post collection on all aggs that the children agg is wrapping,
// otherwise any post work that is required, because we started to collect buckets
// in the method will not be performed.
collectableSubAggregators.postCollection();
} }
@Override @Override

View File

@ -22,7 +22,12 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.elasticsearch.common.lucene.docset.DocIdSets; import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -36,8 +41,6 @@ public class FilterAggregator extends SingleBucketAggregator {
private final Filter filter; private final Filter filter;
private Bits bits;
public FilterAggregator(String name, public FilterAggregator(String name,
org.apache.lucene.search.Filter filter, org.apache.lucene.search.Filter filter,
AggregatorFactories factories, AggregatorFactories factories,
@ -49,19 +52,19 @@ public class FilterAggregator extends SingleBucketAggregator {
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
try { final LeafBucketCollector sub) throws IOException {
bits = DocIdSets.asSequentialAccessBits(reader.reader().maxDoc(), filter.getDocIdSet(reader, null)); // TODO: use the iterator if the filter does not support random access
} catch (IOException ioe) { // no need to provide deleted docs to the filter
throw new AggregationExecutionException("Failed to aggregate filter aggregator [" + name + "]", ioe); final Bits bits = DocIdSets.asSequentialAccessBits(ctx.reader().maxDoc(), filter.getDocIdSet(ctx, null));
} return new LeafBucketCollectorBase(sub, null) {
} @Override
public void collect(int doc, long bucket) throws IOException {
@Override if (bits.get(doc)) {
public void collect(int doc, long owningBucketOrdinal) throws IOException { collectBucket(sub, doc, bucket);
if (bits.get(doc)) { }
collectBucket(doc, owningBucketOrdinal); }
} };
} }
@Override @Override

View File

@ -20,11 +20,18 @@
package org.elasticsearch.search.aggregations.bucket.filters; package org.elasticsearch.search.aggregations.bucket.filters;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.elasticsearch.common.lucene.docset.DocIdSets; import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -49,40 +56,34 @@ public class FiltersAggregator extends BucketsAggregator {
} }
private final KeyedFilter[] filters; private final KeyedFilter[] filters;
private final Bits[] bits; private final boolean keyed;
private boolean keyed;
public FiltersAggregator(String name, AggregatorFactories factories, List<KeyedFilter> filters, boolean keyed, AggregationContext aggregationContext, public FiltersAggregator(String name, AggregatorFactories factories, List<KeyedFilter> filters, boolean keyed, AggregationContext aggregationContext,
Aggregator parent, Map<String, Object> metaData) throws IOException { Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
this.keyed = keyed; this.keyed = keyed;
this.filters = filters.toArray(new KeyedFilter[filters.size()]); this.filters = filters.toArray(new KeyedFilter[filters.size()]);
this.bits = new Bits[this.filters.length];
} }
@Override @Override
public boolean shouldCollect() { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
return true; final LeafBucketCollector sub) throws IOException {
} // TODO: use the iterator if the filter does not support random access
// no need to provide deleted docs to the filter
@Override final Bits[] bits = new Bits[filters.length];
public void setNextReader(LeafReaderContext reader) { for (int i = 0; i < filters.length; ++i) {
try { bits[i] = DocIdSets.asSequentialAccessBits(ctx.reader().maxDoc(), filters[i].filter.getDocIdSet(ctx, null));
for (int i = 0; i < filters.length; i++) {
bits[i] = DocIdSets.asSequentialAccessBits(reader.reader().maxDoc(), filters[i].filter.getDocIdSet(reader, null));
}
} catch (IOException ioe) {
throw new AggregationExecutionException("Failed to aggregate filter aggregator [" + name + "]", ioe);
} }
} return new LeafBucketCollectorBase(sub, null) {
@Override
@Override public void collect(int doc, long bucket) throws IOException {
public void collect(int doc, long owningBucketOrdinal) throws IOException { for (int i = 0; i < bits.length; i++) {
for (int i = 0; i < bits.length; i++) { if (bits[i].get(doc)) {
if (bits[i].get(doc)) { collectBucket(sub, doc, bucketOrd(bucket, i));
collectBucket(doc, bucketOrd(owningBucketOrdinal, i)); }
}
} }
} };
} }
@Override @Override
@ -108,7 +109,7 @@ public class FiltersAggregator extends BucketsAggregator {
return new InternalFilters(name, buckets, keyed, metaData()); return new InternalFilters(name, buckets, keyed, metaData());
} }
private final long bucketOrd(long owningBucketOrdinal, int filterOrd) { final long bucketOrd(long owningBucketOrdinal, int filterOrd) {
return owningBucketOrdinal * filters.length + filterOrd; return owningBucketOrdinal * filters.length + filterOrd;
} }

View File

@ -24,7 +24,9 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -45,7 +47,6 @@ public class GeoHashGridAggregator extends BucketsAggregator {
private final int shardSize; private final int shardSize;
private final ValuesSource.Numeric valuesSource; private final ValuesSource.Numeric valuesSource;
private final LongHash bucketOrds; private final LongHash bucketOrds;
private SortedNumericDocValues values;
public GeoHashGridAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, public GeoHashGridAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource,
int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException { int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
@ -57,35 +58,37 @@ public class GeoHashGridAggregator extends BucketsAggregator {
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return true; return (valuesSource != null && valuesSource.needsScores()) || super.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.longValues(); final LeafBucketCollector sub) throws IOException {
} final SortedNumericDocValues values = valuesSource.longValues(ctx);
return new LeafBucketCollectorBase(sub, null) {
@Override
public void collect(int doc, long bucket) throws IOException {
assert bucket == 0;
values.setDocument(doc);
final int valuesCount = values.count();
@Override long previous = Long.MAX_VALUE;
public void collect(int doc, long owningBucketOrdinal) throws IOException { for (int i = 0; i < valuesCount; ++i) {
assert owningBucketOrdinal == 0; final long val = values.valueAt(i);
values.setDocument(doc); if (previous != val || i == 0) {
final int valuesCount = values.count(); long bucketOrdinal = bucketOrds.add(val);
if (bucketOrdinal < 0) { // already seen
long previous = Long.MAX_VALUE; bucketOrdinal = - 1 - bucketOrdinal;
for (int i = 0; i < valuesCount; ++i) { collectExistingBucket(sub, doc, bucketOrdinal);
final long val = values.valueAt(i); } else {
if (previous != val || i == 0) { collectBucket(sub, doc, bucketOrdinal);
long bucketOrdinal = bucketOrds.add(val); }
if (bucketOrdinal < 0) { // already seen previous = val;
bucketOrdinal = - 1 - bucketOrdinal; }
collectExistingBucket(doc, bucketOrdinal);
} else {
collectBucket(doc, bucketOrdinal);
} }
previous = val;
} }
} };
} }
// private impl that stores a bucket ord. This allows for computing the aggregations lazily. // private impl that stores a bucket ord. This allows for computing the aggregations lazily.

View File

@ -18,6 +18,7 @@
*/ */
package org.elasticsearch.search.aggregations.bucket.geogrid; package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedNumericDocValues;
import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoPoint;
@ -28,11 +29,16 @@ import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.SortingNumericDocValues; import org.elasticsearch.index.fielddata.SortingNumericDocValues;
import org.elasticsearch.index.query.GeoBoundingBoxFilterBuilder; import org.elasticsearch.index.query.GeoBoundingBoxFilterBuilder;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BucketUtils; import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.support.*; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -131,26 +137,23 @@ public class GeoHashGridParser implements Aggregator.Parser {
if (collectsFromSingleBucket == false) { if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent); return asMultiBucketAggregator(this, aggregationContext, parent);
} }
final CellValues cellIdValues = new CellValues(valuesSource, precision); ValuesSource.Numeric cellIdSource = new CellIdSource(valuesSource, precision);
ValuesSource.Numeric cellIdSource = new CellIdSource(cellIdValues, valuesSource.metaData());
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, metaData); return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, metaData);
} }
private static class CellValues extends SortingNumericDocValues { private static class CellValues extends SortingNumericDocValues {
private ValuesSource.GeoPoint geoPointValues;
private MultiGeoPointValues geoValues; private MultiGeoPointValues geoValues;
private int precision; private int precision;
protected CellValues(ValuesSource.GeoPoint geoPointValues, int precision) { protected CellValues(MultiGeoPointValues geoValues, int precision) {
this.geoPointValues = geoPointValues; this.geoValues = geoValues;
this.precision = precision; this.precision = precision;
} }
@Override @Override
public void setDocument(int docId) { public void setDocument(int docId) {
geoValues = geoPointValues.geoPointValues();
geoValues.setDocument(docId); geoValues.setDocument(docId);
resize(geoValues.count()); resize(geoValues.count());
for (int i = 0; i < count(); ++i) { for (int i = 0; i < count(); ++i) {
@ -163,13 +166,13 @@ public class GeoHashGridParser implements Aggregator.Parser {
} }
private static class CellIdSource extends ValuesSource.Numeric { private static class CellIdSource extends ValuesSource.Numeric {
private final SortedNumericDocValues values; private final ValuesSource.GeoPoint valuesSource;
private MetaData metaData; private final int precision;
public CellIdSource(SortedNumericDocValues values, MetaData delegate) { public CellIdSource(ValuesSource.GeoPoint valuesSource, int precision) {
this.values = values; this.valuesSource = valuesSource;
//different GeoPoints could map to the same or different geohash cells. //different GeoPoints could map to the same or different geohash cells.
this.metaData = MetaData.builder(delegate).uniqueness(MetaData.Uniqueness.UNKNOWN).build(); this.precision = precision;
} }
@Override @Override
@ -178,25 +181,20 @@ public class GeoHashGridParser implements Aggregator.Parser {
} }
@Override @Override
public SortedNumericDocValues longValues() { public SortedNumericDocValues longValues(LeafReaderContext ctx) {
return values; return new CellValues(valuesSource.geoPointValues(ctx), precision);
} }
@Override @Override
public SortedNumericDoubleValues doubleValues() { public SortedNumericDoubleValues doubleValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override @Override
public SortedBinaryDocValues bytesValues() { public SortedBinaryDocValues bytesValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override
public MetaData metaData() {
return metaData;
}
} }
} }

View File

@ -20,7 +20,13 @@ package org.elasticsearch.search.aggregations.bucket.global;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -37,13 +43,15 @@ public class GlobalAggregator extends SingleBucketAggregator {
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
} final LeafBucketCollector sub) throws IOException {
return new LeafBucketCollectorBase(sub, null) {
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public void collect(int doc, long bucket) throws IOException {
assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator"; assert bucket == 0 : "global aggregator can only be a top level aggregator";
collectBucket(doc, owningBucketOrdinal); collectBucket(sub, doc, bucket);
}
};
} }
@Override @Override

View File

@ -28,6 +28,8 @@ import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -76,38 +78,43 @@ public class HistogramAggregator extends BucketsAggregator {
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return (valuesSource != null && valuesSource.needsScores()) || super.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.longValues(); final LeafBucketCollector sub) throws IOException {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0;
values.setDocument(doc);
final int valuesCount = values.count();
long previousKey = Long.MIN_VALUE;
for (int i = 0; i < valuesCount; ++i) {
long value = values.valueAt(i);
long key = rounding.roundKey(value);
assert key >= previousKey;
if (key == previousKey) {
continue;
}
long bucketOrd = bucketOrds.add(key);
if (bucketOrd < 0) { // already seen
bucketOrd = -1 - bucketOrd;
collectExistingBucket(doc, bucketOrd);
} else {
collectBucket(doc, bucketOrd);
}
previousKey = key;
} }
final SortedNumericDocValues values = valuesSource.longValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
assert bucket == 0;
values.setDocument(doc);
final int valuesCount = values.count();
long previousKey = Long.MIN_VALUE;
for (int i = 0; i < valuesCount; ++i) {
long value = values.valueAt(i);
long key = rounding.roundKey(value);
assert key >= previousKey;
if (key == previousKey) {
continue;
}
long bucketOrd = bucketOrds.add(key);
if (bucketOrd < 0) { // already seen
bucketOrd = -1 - bucketOrd;
collectExistingBucket(sub, doc, bucketOrd);
} else {
collectBucket(sub, doc, bucketOrd);
}
previousKey = key;
}
}
};
} }
@Override @Override

View File

@ -23,6 +23,8 @@ import org.apache.lucene.util.Bits;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -38,7 +40,6 @@ import java.util.Map;
public class MissingAggregator extends SingleBucketAggregator { public class MissingAggregator extends SingleBucketAggregator {
private final ValuesSource valuesSource; private final ValuesSource valuesSource;
private Bits docsWithValue;
public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
@ -47,19 +48,23 @@ public class MissingAggregator extends SingleBucketAggregator {
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
if (valuesSource != null) { final LeafBucketCollector sub) throws IOException {
docsWithValue = valuesSource.docsWithValue(reader.reader().maxDoc());
} else {
docsWithValue = new Bits.MatchNoBits(reader.reader().maxDoc());
}
}
@Override final Bits docsWithValue;
public void collect(int doc, long owningBucketOrdinal) throws IOException { if (valuesSource != null) {
if (docsWithValue != null && !docsWithValue.get(doc)) { docsWithValue = valuesSource.docsWithValue(ctx);
collectBucket(doc, owningBucketOrdinal); } else {
docsWithValue = new Bits.MatchNoBits(ctx.reader().maxDoc());
} }
return new LeafBucketCollectorBase(sub, docsWithValue) {
@Override
public void collect(int doc, long bucket) throws IOException {
if (docsWithValue != null && !docsWithValue.get(doc)) {
collectBucket(sub, doc, bucket);
}
}
};
} }
@Override @Override

View File

@ -26,12 +26,18 @@ import org.apache.lucene.search.FilterCachingPolicy;
import org.apache.lucene.search.join.BitDocIdSetFilter; import org.apache.lucene.search.join.BitDocIdSetFilter;
import org.apache.lucene.util.BitDocIdSet; import org.apache.lucene.util.BitDocIdSet;
import org.apache.lucene.util.BitSet; import org.apache.lucene.util.BitSet;
import org.elasticsearch.common.lucene.ReaderContextAware;
import org.elasticsearch.common.lucene.docset.DocIdSets; import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.object.ObjectMapper; import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.search.nested.NonNestedDocsFilter; import org.elasticsearch.index.search.nested.NonNestedDocsFilter;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -41,85 +47,75 @@ import java.util.Map;
/** /**
* *
*/ */
public class NestedAggregator extends SingleBucketAggregator implements ReaderContextAware { public class NestedAggregator extends SingleBucketAggregator {
private final Aggregator parentAggregator;
private BitDocIdSetFilter parentFilter; private BitDocIdSetFilter parentFilter;
private final Filter childFilter; private final Filter childFilter;
private DocIdSetIterator childDocs; private DocIdSetIterator childDocs;
private BitSet parentDocs; private BitSet parentDocs;
private LeafReaderContext reader;
public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map<String, Object> metaData, FilterCachingPolicy filterCachingPolicy) throws IOException { public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map<String, Object> metaData, FilterCachingPolicy filterCachingPolicy) throws IOException {
super(name, factories, aggregationContext, parentAggregator, metaData); super(name, factories, aggregationContext, parentAggregator, metaData);
this.parentAggregator = parentAggregator;
childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy); childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy);
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
// Reset parentFilter, so we resolve the parentDocs for each new segment being searched // Reset parentFilter, so we resolve the parentDocs for each new segment being searched
this.parentFilter = null; this.parentFilter = null;
this.reader = reader; // In ES if parent is deleted, then also the children are deleted. Therefore acceptedDocs can also null here.
try { DocIdSet childDocIdSet = childFilter.getDocIdSet(ctx, null);
// In ES if parent is deleted, then also the children are deleted. Therefore acceptedDocs can also null here. if (DocIdSets.isEmpty(childDocIdSet)) {
DocIdSet childDocIdSet = childFilter.getDocIdSet(reader, null); childDocs = null;
if (DocIdSets.isEmpty(childDocIdSet)) { } else {
childDocs = null; childDocs = childDocIdSet.iterator();
} else { }
childDocs = childDocIdSet.iterator();
return new LeafBucketCollectorBase(sub, null) {
@Override
public void collect(int parentDoc, long bucket) throws IOException {
// here we translate the parent doc to a list of its nested docs, and then call super.collect for evey one of them so they'll be collected
// if parentDoc is 0 then this means that this parent doesn't have child docs (b/c these appear always before the parent doc), so we can skip:
if (parentDoc == 0 || childDocs == null) {
return;
}
if (parentFilter == null) {
// The aggs are instantiated in reverse, first the most inner nested aggs and lastly the top level aggs
// So at the time a nested 'nested' aggs is parsed its closest parent nested aggs hasn't been constructed.
// So the trick is to set at the last moment just before needed and we can use its child filter as the
// parent filter.
// Additional NOTE: Before this logic was performed in the setNextReader(...) method, but the the assumption
// that aggs instances are constructed in reverse doesn't hold when buckets are constructed lazily during
// aggs execution
Filter parentFilterNotCached = findClosestNestedPath(parent());
if (parentFilterNotCached == null) {
parentFilterNotCached = NonNestedDocsFilter.INSTANCE;
}
parentFilter = context.searchContext().bitsetFilterCache().getBitDocIdSetFilter(parentFilterNotCached);
BitDocIdSet parentSet = parentFilter.getDocIdSet(ctx);
if (DocIdSets.isEmpty(parentSet)) {
// There are no parentDocs in the segment, so return and set childDocs to null, so we exit early for future invocations.
childDocs = null;
return;
} else {
parentDocs = parentSet.bits();
}
}
final int prevParentDoc = parentDocs.prevSetBit(parentDoc - 1);
int childDocId = childDocs.docID();
if (childDocId <= prevParentDoc) {
childDocId = childDocs.advance(prevParentDoc + 1);
}
for (; childDocId < parentDoc; childDocId = childDocs.nextDoc()) {
collectBucket(sub, childDocId, bucket);
}
} }
} catch (IOException ioe) { };
throw new AggregationExecutionException("Failed to aggregate [" + name + "]", ioe);
}
}
@Override
public void collect(int parentDoc, long bucketOrd) throws IOException {
assert bucketOrd == 0;
// here we translate the parent doc to a list of its nested docs, and then call super.collect for evey one of them so they'll be collected
// if parentDoc is 0 then this means that this parent doesn't have child docs (b/c these appear always before the parent doc), so we can skip:
if (parentDoc == 0 || childDocs == null) {
return;
}
if (parentFilter == null) {
// The aggs are instantiated in reverse, first the most inner nested aggs and lastly the top level aggs
// So at the time a nested 'nested' aggs is parsed its closest parent nested aggs hasn't been constructed.
// So the trick is to set at the last moment just before needed and we can use its child filter as the
// parent filter.
// Additional NOTE: Before this logic was performed in the setNextReader(...) method, but the the assumption
// that aggs instances are constructed in reverse doesn't hold when buckets are constructed lazily during
// aggs execution
Filter parentFilterNotCached = findClosestNestedPath(parentAggregator);
if (parentFilterNotCached == null) {
parentFilterNotCached = NonNestedDocsFilter.INSTANCE;
}
parentFilter = context.searchContext().bitsetFilterCache().getBitDocIdSetFilter(parentFilterNotCached);
BitDocIdSet parentSet = parentFilter.getDocIdSet(reader);
if (DocIdSets.isEmpty(parentSet)) {
// There are no parentDocs in the segment, so return and set childDocs to null, so we exit early for future invocations.
childDocs = null;
return;
} else {
parentDocs = parentSet.bits();
}
}
final int prevParentDoc = parentDocs.prevSetBit(parentDoc - 1);
int childDocId = childDocs.docID();
if (childDocId <= prevParentDoc) {
childDocId = childDocs.advance(prevParentDoc + 1);
}
int numChildren = 0;
for (; childDocId < parentDoc; childDocId = childDocs.nextDoc()) {
collectBucketNoCounts(childDocId, bucketOrd);
numChildren += 1;
}
incrementBucketDocCount(bucketOrd, numChildren);
} }
@Override @Override

View File

@ -19,19 +19,26 @@
package org.elasticsearch.search.aggregations.bucket.nested; package org.elasticsearch.search.aggregations.bucket.nested;
import com.carrotsearch.hppc.LongIntOpenHashMap; import com.carrotsearch.hppc.LongIntOpenHashMap;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.apache.lucene.search.join.BitDocIdSetFilter; import org.apache.lucene.search.join.BitDocIdSetFilter;
import org.apache.lucene.util.BitDocIdSet; import org.apache.lucene.util.BitDocIdSet;
import org.apache.lucene.util.BitSet; import org.apache.lucene.util.BitSet;
import org.elasticsearch.common.lucene.ReaderContextAware;
import org.elasticsearch.common.lucene.docset.DocIdSets; import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.object.ObjectMapper; import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.search.nested.NonNestedDocsFilter; import org.elasticsearch.index.search.nested.NonNestedDocsFilter;
import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -41,13 +48,9 @@ import java.util.Map;
/** /**
* *
*/ */
public class ReverseNestedAggregator extends SingleBucketAggregator implements ReaderContextAware { public class ReverseNestedAggregator extends SingleBucketAggregator {
private final BitDocIdSetFilter parentFilter; private final BitDocIdSetFilter parentFilter;
private BitSet parentDocs;
// TODO: Add LongIntPagedHashMap?
private final LongIntOpenHashMap bucketOrdToLastCollectedParentDoc;
public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException { public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
@ -56,49 +59,39 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
} else { } else {
parentFilter = context.searchContext().bitsetFilterCache().getBitDocIdSetFilter(objectMapper.nestedTypeFilter()); parentFilter = context.searchContext().bitsetFilterCache().getBitDocIdSetFilter(objectMapper.nestedTypeFilter());
} }
bucketOrdToLastCollectedParentDoc = new LongIntOpenHashMap(32);
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
bucketOrdToLastCollectedParentDoc.clear(); // In ES if parent is deleted, then also the children are deleted, so the child docs this agg receives
try { // must belong to parent docs that is alive. For this reason acceptedDocs can be null here.
// In ES if parent is deleted, then also the children are deleted, so the child docs this agg receives BitDocIdSet docIdSet = parentFilter.getDocIdSet(ctx);
// must belong to parent docs that is alive. For this reason acceptedDocs can be null here. final BitSet parentDocs;
BitDocIdSet docIdSet = parentFilter.getDocIdSet(reader); if (DocIdSets.isEmpty(docIdSet)) {
if (DocIdSets.isEmpty(docIdSet)) { return LeafBucketCollector.NO_OP_COLLECTOR;
parentDocs = null;
} else {
parentDocs = docIdSet.bits();
}
} catch (IOException ioe) {
throw new AggregationExecutionException("Failed to aggregate [" + name + "]", ioe);
}
}
@Override
public void collect(int childDoc, long bucketOrd) throws IOException {
if (parentDocs == null) {
return;
}
// fast forward to retrieve the parentDoc this childDoc belongs to
final int parentDoc = parentDocs.nextSetBit(childDoc);
assert childDoc <= parentDoc && parentDoc != DocIdSetIterator.NO_MORE_DOCS;
if (bucketOrdToLastCollectedParentDoc.containsKey(bucketOrd)) {
int lastCollectedParentDoc = bucketOrdToLastCollectedParentDoc.lget();
if (parentDoc > lastCollectedParentDoc) {
innerCollect(parentDoc, bucketOrd);
bucketOrdToLastCollectedParentDoc.lset(parentDoc);
}
} else { } else {
innerCollect(parentDoc, bucketOrd); parentDocs = docIdSet.bits();
bucketOrdToLastCollectedParentDoc.put(bucketOrd, parentDoc);
} }
} final LongIntOpenHashMap bucketOrdToLastCollectedParentDoc = new LongIntOpenHashMap(32);
return new LeafBucketCollectorBase(sub, null) {
private void innerCollect(int parentDoc, long bucketOrd) throws IOException { @Override
collectBucket(parentDoc, bucketOrd); public void collect(int childDoc, long bucket) throws IOException {
// fast forward to retrieve the parentDoc this childDoc belongs to
final int parentDoc = parentDocs.nextSetBit(childDoc);
assert childDoc <= parentDoc && parentDoc != DocIdSetIterator.NO_MORE_DOCS;
if (bucketOrdToLastCollectedParentDoc.containsKey(bucket)) {
int lastCollectedParentDoc = bucketOrdToLastCollectedParentDoc.lget();
if (parentDoc > lastCollectedParentDoc) {
collectBucket(sub, parentDoc, bucket);
bucketOrdToLastCollectedParentDoc.lset(parentDoc);
}
} else {
collectBucket(sub, parentDoc, bucket);
bucketOrdToLastCollectedParentDoc.put(bucket, parentDoc);
}
}
};
} }
private static NestedAggregator findClosestNestedAggregator(Aggregator parent) { private static NestedAggregator findClosestNestedAggregator(Aggregator parent) {

View File

@ -19,11 +19,19 @@
package org.elasticsearch.search.aggregations.bucket.range; package org.elasticsearch.search.aggregations.bucket.range;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.InPlaceMergeSorter; import org.apache.lucene.util.InPlaceMergeSorter;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -80,12 +88,11 @@ public class RangeAggregator extends BucketsAggregator {
} }
} }
private final ValuesSource.Numeric valuesSource; final ValuesSource.Numeric valuesSource;
private final @Nullable ValueFormatter formatter; final @Nullable ValueFormatter formatter;
private final Range[] ranges; final Range[] ranges;
private final boolean keyed; final boolean keyed;
private final InternalRange.Factory rangeFactory; final InternalRange.Factory rangeFactory;
private SortedNumericDoubleValues values;
final double[] maxTo; final double[] maxTo;
@ -123,78 +130,80 @@ public class RangeAggregator extends BucketsAggregator {
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return true; return (valuesSource != null && valuesSource.needsScores()) || super.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
values.setDocument(doc);
final int valuesCount = values.count();
for (int i = 0, lo = 0; i < valuesCount; ++i) {
final double value = values.valueAt(i);
lo = collect(doc, value, bucket, lo);
}
}
private int collect(int doc, double value, long owningBucketOrdinal, int lowBound) throws IOException {
int lo = lowBound, hi = ranges.length - 1; // all candidates are between these indexes
int mid = (lo + hi) >>> 1;
while (lo <= hi) {
if (value < ranges[mid].from) {
hi = mid - 1;
} else if (value >= maxTo[mid]) {
lo = mid + 1;
} else {
break;
}
mid = (lo + hi) >>> 1;
}
if (lo > hi) return lo; // no potential candidate
// binary search the lower bound
int startLo = lo, startHi = mid;
while (startLo <= startHi) {
final int startMid = (startLo + startHi) >>> 1;
if (value >= maxTo[startMid]) {
startLo = startMid + 1;
} else {
startHi = startMid - 1;
}
}
// binary search the upper bound
int endLo = mid, endHi = hi;
while (endLo <= endHi) {
final int endMid = (endLo + endHi) >>> 1;
if (value < ranges[endMid].from) {
endHi = endMid - 1;
} else {
endLo = endMid + 1;
}
}
assert startLo == lowBound || value >= maxTo[startLo - 1];
assert endHi == ranges.length - 1 || value < ranges[endHi + 1].from;
for (int i = startLo; i <= endHi; ++i) {
if (ranges[i].matches(value)) {
collectBucket(sub, doc, subBucketOrdinal(owningBucketOrdinal, i));
}
}
return endHi + 1;
}
};
} }
private final long subBucketOrdinal(long owningBucketOrdinal, int rangeOrd) { private final long subBucketOrdinal(long owningBucketOrdinal, int rangeOrd) {
return owningBucketOrdinal * ranges.length + rangeOrd; return owningBucketOrdinal * ranges.length + rangeOrd;
} }
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
values.setDocument(doc);
final int valuesCount = values.count();
for (int i = 0, lo = 0; i < valuesCount; ++i) {
final double value = values.valueAt(i);
lo = collect(doc, value, owningBucketOrdinal, lo);
}
}
private int collect(int doc, double value, long owningBucketOrdinal, int lowBound) throws IOException {
int lo = lowBound, hi = ranges.length - 1; // all candidates are between these indexes
int mid = (lo + hi) >>> 1;
while (lo <= hi) {
if (value < ranges[mid].from) {
hi = mid - 1;
} else if (value >= maxTo[mid]) {
lo = mid + 1;
} else {
break;
}
mid = (lo + hi) >>> 1;
}
if (lo > hi) return lo; // no potential candidate
// binary search the lower bound
int startLo = lo, startHi = mid;
while (startLo <= startHi) {
final int startMid = (startLo + startHi) >>> 1;
if (value >= maxTo[startMid]) {
startLo = startMid + 1;
} else {
startHi = startMid - 1;
}
}
// binary search the upper bound
int endLo = mid, endHi = hi;
while (endLo <= endHi) {
final int endMid = (endLo + endHi) >>> 1;
if (value < ranges[endMid].from) {
endHi = endMid - 1;
} else {
endLo = endMid + 1;
}
}
assert startLo == lowBound || value >= maxTo[startLo - 1];
assert endHi == ranges.length - 1 || value < ranges[endHi + 1].from;
for (int i = startLo; i <= endHi; ++i) {
if (ranges[i].matches(value)) {
collectBucket(doc, subBucketOrdinal(owningBucketOrdinal, i));
}
}
return endHi + 1;
}
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
List<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length); List<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length);

View File

@ -24,7 +24,6 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.geo.GeoDistance; import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoDistance.FixedSourceDistance; import org.elasticsearch.common.geo.GeoDistance.FixedSourceDistance;
import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.lucene.ReaderContextAware;
import org.elasticsearch.common.unit.DistanceUnit; import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.MultiGeoPointValues; import org.elasticsearch.index.fielddata.MultiGeoPointValues;
@ -36,7 +35,12 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange; import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator; import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped; import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.support.*; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.GeoPointParser;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -182,57 +186,43 @@ public class GeoDistanceParser implements Aggregator.Parser {
@Override @Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException { protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit); DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit);
aggregationContext.registerReaderContextAware(distanceSource);
return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, metaData); return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, metaData);
} }
private static class DistanceSource extends ValuesSource.Numeric implements ReaderContextAware { private static class DistanceSource extends ValuesSource.Numeric {
private final ValuesSource.GeoPoint source; private final ValuesSource.GeoPoint source;
private final GeoDistance distanceType; private final GeoDistance distanceType;
private final DistanceUnit unit; private final DistanceUnit unit;
private final org.elasticsearch.common.geo.GeoPoint origin; private final org.elasticsearch.common.geo.GeoPoint origin;
private final MetaData metaData;
private SortedNumericDoubleValues distanceValues;
public DistanceSource(ValuesSource.GeoPoint source, GeoDistance distanceType, org.elasticsearch.common.geo.GeoPoint origin, DistanceUnit unit) { public DistanceSource(ValuesSource.GeoPoint source, GeoDistance distanceType, org.elasticsearch.common.geo.GeoPoint origin, DistanceUnit unit) {
this.source = source; this.source = source;
// even if the geo points are unique, there's no guarantee the distances are // even if the geo points are unique, there's no guarantee the distances are
this.metaData = MetaData.builder(source.metaData()).uniqueness(MetaData.Uniqueness.UNKNOWN).build();
this.distanceType = distanceType; this.distanceType = distanceType;
this.unit = unit; this.unit = unit;
this.origin = origin; this.origin = origin;
} }
@Override
public void setNextReader(LeafReaderContext reader) {
final MultiGeoPointValues geoValues = source.geoPointValues();
final FixedSourceDistance distance = distanceType.fixedSourceDistance(origin.getLat(), origin.getLon(), unit);
distanceValues = GeoDistance.distanceValues(geoValues, distance);
}
@Override
public MetaData metaData() {
return metaData;
}
@Override @Override
public boolean isFloatingPoint() { public boolean isFloatingPoint() {
return true; return true;
} }
@Override @Override
public SortedNumericDocValues longValues() { public SortedNumericDocValues longValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override @Override
public SortedNumericDoubleValues doubleValues() { public SortedNumericDoubleValues doubleValues(LeafReaderContext ctx) {
return distanceValues; final MultiGeoPointValues geoValues = source.geoPointValues(ctx);
final FixedSourceDistance distance = distanceType.fixedSourceDistance(origin.getLat(), origin.getLon(), unit);
return GeoDistance.distanceValues(geoValues, distance);
} }
@Override @Override
public SortedBinaryDocValues bytesValues() { public SortedBinaryDocValues bytesValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }

View File

@ -19,11 +19,14 @@
package org.elasticsearch.search.aggregations.bucket.significant; package org.elasticsearch.search.aggregations.bucket.significant;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -44,20 +47,27 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
protected final SignificantTermsAggregatorFactory termsAggFactory; protected final SignificantTermsAggregatorFactory termsAggFactory;
public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long maxOrd, BucketCountThresholds bucketCountThresholds, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) throws IOException { SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, maxOrd, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData); super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData);
this.termsAggFactory = termsAggFactory; this.termsAggFactory = termsAggFactory;
} }
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
super.collect(doc, owningBucketOrdinal); final LeafBucketCollector sub) throws IOException {
numCollectedDocs++; return new LeafBucketCollectorBase(super.getLeafCollector(ctx, sub), null) {
@Override
public void collect(int doc, long bucket) throws IOException {
super.collect(doc, bucket);
numCollectedDocs++;
}
};
} }
@Override @Override
public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException { public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
@ -136,25 +146,32 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
private final LongHash bucketOrds; private final LongHash bucketOrds;
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) throws IOException { public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, 1, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData); super(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData);
bucketOrds = new LongHash(1, aggregationContext.bigArrays()); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
} }
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
numCollectedDocs++; final LeafBucketCollector sub) throws IOException {
globalOrds.setDocument(doc); return new LeafBucketCollectorBase(super.getLeafCollector(ctx, sub), null) {
final int numOrds = globalOrds.cardinality(); @Override
for (int i = 0; i < numOrds; i++) { public void collect(int doc, long bucket) throws IOException {
final long globalOrd = globalOrds.ordAt(i); assert bucket == 0;
long bucketOrd = bucketOrds.add(globalOrd); numCollectedDocs++;
if (bucketOrd < 0) { globalOrds.setDocument(doc);
bucketOrd = -1 - bucketOrd; final int numOrds = globalOrds.cardinality();
collectExistingBucket(doc, bucketOrd); for (int i = 0; i < numOrds; i++) {
} else { final long globalOrd = globalOrds.ordAt(i);
collectBucket(doc, bucketOrd); long bucketOrd = bucketOrds.add(globalOrd);
if (bucketOrd < 0) {
bucketOrd = -1 - bucketOrd;
collectExistingBucket(sub, doc, bucketOrd);
} else {
collectBucket(sub, doc, bucketOrd);
}
}
} }
} };
} }
@Override @Override

View File

@ -19,10 +19,13 @@
package org.elasticsearch.search.aggregations.bucket.significant; package org.elasticsearch.search.aggregations.bucket.significant;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -52,9 +55,15 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
private final SignificantTermsAggregatorFactory termsAggFactory; private final SignificantTermsAggregatorFactory termsAggFactory;
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
super.collect(doc, owningBucketOrdinal); final LeafBucketCollector sub) throws IOException {
numCollectedDocs++; return new LeafBucketCollectorBase(super.getLeafCollector(ctx, sub), null) {
@Override
public void collect(int doc, long bucket) throws IOException {
super.collect(doc, bucket);
numCollectedDocs++;
}
};
} }
@Override @Override

View File

@ -19,10 +19,13 @@
package org.elasticsearch.search.aggregations.bucket.significant; package org.elasticsearch.search.aggregations.bucket.significant;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.terms.StringTermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.StringTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -52,9 +55,15 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
} }
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
super.collect(doc, owningBucketOrdinal); final LeafBucketCollector sub) throws IOException {
numCollectedDocs++; return new LeafBucketCollectorBase(super.getLeafCollector(ctx, sub), null) {
@Override
public void collect(int doc, long bucket) throws IOException {
super.collect(doc, bucket);
numCollectedDocs++;
}
};
} }
@Override @Override

View File

@ -32,6 +32,7 @@ import org.elasticsearch.common.lucene.index.FreqTermsEnum;
import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.NonCollectingAggregator;
@ -67,11 +68,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
} }
@Override
boolean needsGlobalOrdinals() {
return false;
}
}, },
GLOBAL_ORDINALS(new ParseField("global_ordinals")) { GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
@ -81,13 +77,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource; ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = aggregationContext.searchContext().searcher(); IndexSearcher indexSearcher = aggregationContext.searchContext().searcher();
long maxOrd = valueSourceWithOrdinals.globalMaxOrd(indexSearcher); return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
}
@Override
boolean needsGlobalOrdinals() {
return true;
} }
}, },
@ -99,11 +89,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException {
return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
} }
@Override
boolean needsGlobalOrdinals() {
return true;
}
}; };
public static ExecutionMode fromString(String value) { public static ExecutionMode fromString(String value) {
@ -125,8 +110,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException; AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException;
abstract boolean needsGlobalOrdinals();
@Override @Override
public String toString() { public String toString() {
return parseField.getPreferredName(); return parseField.getPreferredName();
@ -196,7 +179,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
} }
} }
assert execution != null; assert execution != null;
valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals());
return execution.create(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, this, metaData); return execution.create(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, this, metaData);
} }

View File

@ -30,7 +30,6 @@ import java.util.Map;
abstract class AbstractStringTermsAggregator extends TermsAggregator { abstract class AbstractStringTermsAggregator extends TermsAggregator {
protected final boolean showTermDocCountError; protected final boolean showTermDocCountError;
public AbstractStringTermsAggregator(String name, AggregatorFactories factories, public AbstractStringTermsAggregator(String name, AggregatorFactories factories,
@ -41,11 +40,6 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
this.showTermDocCountError = showTermDocCountError; this.showTermDocCountError = showTermDocCountError;
} }
@Override
public boolean shouldCollect() {
return true;
}
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, metaData()); return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, metaData());

View File

@ -21,7 +21,6 @@
package org.elasticsearch.search.aggregations.bucket.terms; package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
@ -80,7 +79,7 @@ public abstract class AbstractTermsParametersParser {
} else if (token == XContentParser.Token.VALUE_STRING) { } else if (token == XContentParser.Token.VALUE_STRING) {
if (EXECUTION_HINT_FIELD_NAME.match(currentFieldName)) { if (EXECUTION_HINT_FIELD_NAME.match(currentFieldName)) {
executionHint = parser.text(); executionHint = parser.text();
} else if(Aggregator.COLLECT_MODE.match(currentFieldName)){ } else if(SubAggCollectionMode.KEY.match(currentFieldName)){
collectMode = SubAggCollectionMode.parse(parser.text()); collectMode = SubAggCollectionMode.parse(parser.text());
} else if (REQUIRED_SIZE_FIELD_NAME.match(currentFieldName)) { } else if (REQUIRED_SIZE_FIELD_NAME.match(currentFieldName)) {
bucketCountThresholds.setRequiredSize(parser.intValue()); bucketCountThresholds.setRequiredSize(parser.intValue());

View File

@ -18,6 +18,7 @@
*/ */
package org.elasticsearch.search.aggregations.bucket.terms; package org.elasticsearch.search.aggregations.bucket.terms;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.util.NumericUtils; import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -45,8 +46,8 @@ public class DoubleTermsAggregator extends LongTermsAggregator {
} }
@Override @Override
protected SortedNumericDocValues getValues(Numeric valuesSource) { protected SortedNumericDocValues getValues(Numeric valuesSource, LeafReaderContext ctx) throws IOException {
return FieldData.toSortableLongBits(valuesSource.doubleValues()); return FieldData.toSortableLongBits(valuesSource.doubleValues(ctx));
} }
@Override @Override

View File

@ -27,7 +27,6 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LongBitSet; import org.apache.lucene.util.LongBitSet;
import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
@ -38,8 +37,10 @@ import org.elasticsearch.index.fielddata.AbstractRandomAccessOrds;
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalMapping; import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalMapping;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.Bucket;
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue; import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
@ -58,8 +59,6 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
protected final ValuesSource.Bytes.WithOrdinals.FieldData valuesSource; protected final ValuesSource.Bytes.WithOrdinals.FieldData valuesSource;
protected final IncludeExclude includeExclude; protected final IncludeExclude includeExclude;
protected RandomAccessOrds globalOrds;
// TODO: cache the acceptedglobalValues per aggregation definition. // TODO: cache the acceptedglobalValues per aggregation definition.
// We can't cache this yet in ValuesSource, since ValuesSource is reused per field for aggs during the execution. // We can't cache this yet in ValuesSource, since ValuesSource is reused per field for aggs during the execution.
// If aggs with same field, but different include/exclude are defined, then the last defined one will override the // If aggs with same field, but different include/exclude are defined, then the last defined one will override the
@ -68,15 +67,14 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
// especially if this agg is on a second layer or deeper. // especially if this agg is on a second layer or deeper.
protected LongBitSet acceptedGlobalOrdinals; protected LongBitSet acceptedGlobalOrdinals;
protected Collector collector; protected RandomAccessOrds globalOrds;
public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, Terms.Order order, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException { IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData); super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;
grow(maxOrd);
} }
protected long getBucketOrd(long termOrd) { protected long getBucketOrd(long termOrd) {
@ -84,54 +82,52 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
} }
@Override @Override
public boolean shouldCollect() { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
return true; final LeafBucketCollector sub) throws IOException {
globalOrds = valuesSource.globalOrdinalsValues(ctx);
if (acceptedGlobalOrdinals == null && includeExclude != null) {
acceptedGlobalOrdinals = includeExclude.acceptedGlobalOrdinals(globalOrds, valuesSource);
}
if (acceptedGlobalOrdinals != null) {
globalOrds = new FilteredOrdinals(globalOrds, acceptedGlobalOrdinals);
}
return newCollector(globalOrds, sub);
} }
protected Collector newCollector(final RandomAccessOrds ords) { protected LeafBucketCollector newCollector(final RandomAccessOrds ords, final LeafBucketCollector sub) {
grow(ords.getValueCount());
final SortedDocValues singleValues = DocValues.unwrapSingleton(ords); final SortedDocValues singleValues = DocValues.unwrapSingleton(ords);
if (singleValues != null) { if (singleValues != null) {
return new Collector() { return new LeafBucketCollectorBase(sub, ords) {
@Override @Override
public void collect(int doc) throws IOException { public void collect(int doc, long bucket) throws IOException {
assert bucket == 0;
final int ord = singleValues.getOrd(doc); final int ord = singleValues.getOrd(doc);
if (ord >= 0) { if (ord >= 0) {
collectExistingBucket(doc, ord); collectExistingBucket(sub, doc, ord);
} }
} }
}; };
} else { } else {
return new Collector() { return new LeafBucketCollectorBase(sub, ords) {
public void collect(int doc) throws IOException { @Override
public void collect(int doc, long bucket) throws IOException {
assert bucket == 0;
ords.setDocument(doc); ords.setDocument(doc);
final int numOrds = ords.cardinality(); final int numOrds = ords.cardinality();
for (int i = 0; i < numOrds; i++) { for (int i = 0; i < numOrds; i++) {
final long globalOrd = ords.ordAt(i); final long globalOrd = ords.ordAt(i);
collectExistingBucket(doc, globalOrd); collectExistingBucket(sub, doc, globalOrd);
} }
} }
}; };
} }
} }
@Override
public void setNextReader(LeafReaderContext reader) {
globalOrds = valuesSource.globalOrdinalsValues();
if (acceptedGlobalOrdinals != null) {
globalOrds = new FilteredOrdinals(globalOrds, acceptedGlobalOrdinals);
} else if (includeExclude != null) {
acceptedGlobalOrdinals = includeExclude.acceptedGlobalOrdinals(globalOrds, valuesSource);
globalOrds = new FilteredOrdinals(globalOrds, acceptedGlobalOrdinals);
}
collector = newCollector(globalOrds);
}
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0;
collector.collect(doc);
}
protected static void copy(BytesRef from, BytesRef to) { protected static void copy(BytesRef from, BytesRef to) {
if (to.bytes.length < from.length) { if (to.bytes.length < from.length) {
to.bytes = new byte[ArrayUtil.oversize(from.length, RamUsageEstimator.NUM_BYTES_BYTE)]; to.bytes = new byte[ArrayUtil.oversize(from.length, RamUsageEstimator.NUM_BYTES_BYTE)];
@ -255,10 +251,6 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
} }
} }
private static interface Collector {
void collect(int doc) throws IOException;
}
/** /**
* Variant of {@link GlobalOrdinalsStringTermsAggregator} that rebases hashes in order to make them dense. Might be * Variant of {@link GlobalOrdinalsStringTermsAggregator} that rebases hashes in order to make them dense. Might be
* useful in case few hashes are visited. * useful in case few hashes are visited.
@ -268,33 +260,34 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
private final LongHash bucketOrds; private final LongHash bucketOrds;
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Terms.Order order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext,
Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException { Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, 1, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); super(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData);
bucketOrds = new LongHash(1, aggregationContext.bigArrays()); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
} }
protected Collector newCollector(final RandomAccessOrds ords) { protected LeafBucketCollector newCollector(final RandomAccessOrds ords, final LeafBucketCollector sub) {
final SortedDocValues singleValues = DocValues.unwrapSingleton(ords); final SortedDocValues singleValues = DocValues.unwrapSingleton(ords);
if (singleValues != null) { if (singleValues != null) {
return new Collector() { return new LeafBucketCollectorBase(sub, ords) {
@Override @Override
public void collect(int doc) throws IOException { public void collect(int doc, long bucket) throws IOException {
final int globalOrd = singleValues.getOrd(doc); final int globalOrd = singleValues.getOrd(doc);
if (globalOrd >= 0) { if (globalOrd >= 0) {
long bucketOrd = bucketOrds.add(globalOrd); long bucketOrd = bucketOrds.add(globalOrd);
if (bucketOrd < 0) { if (bucketOrd < 0) {
bucketOrd = -1 - bucketOrd; bucketOrd = -1 - bucketOrd;
collectExistingBucket(doc, bucketOrd); collectExistingBucket(sub, doc, bucketOrd);
} else { } else {
collectBucket(doc, bucketOrd); collectBucket(sub, doc, bucketOrd);
} }
} }
} }
}; };
} else { } else {
return new Collector() { return new LeafBucketCollectorBase(sub, ords) {
public void collect(int doc) throws IOException { @Override
public void collect(int doc, long bucket) throws IOException {
ords.setDocument(doc); ords.setDocument(doc);
final int numOrds = ords.cardinality(); final int numOrds = ords.cardinality();
for (int i = 0; i < numOrds; i++) { for (int i = 0; i < numOrds; i++) {
@ -302,9 +295,9 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
long bucketOrd = bucketOrds.add(globalOrd); long bucketOrd = bucketOrds.add(globalOrd);
if (bucketOrd < 0) { if (bucketOrd < 0) {
bucketOrd = -1 - bucketOrd; bucketOrd = -1 - bucketOrd;
collectExistingBucket(doc, bucketOrd); collectExistingBucket(sub, doc, bucketOrd);
} else { } else {
collectBucket(doc, bucketOrd); collectBucket(sub, doc, bucketOrd);
} }
} }
} }
@ -331,35 +324,40 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
*/ */
public static class LowCardinality extends GlobalOrdinalsStringTermsAggregator { public static class LowCardinality extends GlobalOrdinalsStringTermsAggregator {
private final IntArray segmentDocCounts; private IntArray segmentDocCounts;
private RandomAccessOrds segmentOrds; private RandomAccessOrds segmentOrds;
public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException { Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); super(name, factories, valuesSource, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData);
assert factories == null || factories.count() == 0; assert factories == null || factories.count() == 0;
this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true); this.segmentDocCounts = context.bigArrays().newIntArray(1, true);
} }
// bucketOrd is ord + 1 to avoid a branch to deal with the missing ord // bucketOrd is ord + 1 to avoid a branch to deal with the missing ord
protected Collector newCollector(final RandomAccessOrds ords) { protected LeafBucketCollector newCollector(final RandomAccessOrds ords, LeafBucketCollector sub) {
final SortedDocValues singleValues = DocValues.unwrapSingleton(segmentOrds); segmentDocCounts = context.bigArrays().grow(segmentDocCounts, 1 + ords.getValueCount());
assert sub == LeafBucketCollector.NO_OP_COLLECTOR;
final SortedDocValues singleValues = DocValues.unwrapSingleton(ords);
if (singleValues != null) { if (singleValues != null) {
return new Collector() { return new LeafBucketCollectorBase(sub, ords) {
@Override @Override
public void collect(int doc) throws IOException { public void collect(int doc, long bucket) throws IOException {
assert bucket == 0;
final int ord = singleValues.getOrd(doc); final int ord = singleValues.getOrd(doc);
segmentDocCounts.increment(ord + 1, 1); segmentDocCounts.increment(ord + 1, 1);
} }
}; };
} else { } else {
return new Collector() { return new LeafBucketCollectorBase(sub, ords) {
public void collect(int doc) throws IOException { @Override
segmentOrds.setDocument(doc); public void collect(int doc, long bucket) throws IOException {
final int numOrds = segmentOrds.cardinality(); assert bucket == 0;
ords.setDocument(doc);
final int numOrds = ords.cardinality();
for (int i = 0; i < numOrds; i++) { for (int i = 0; i < numOrds; i++) {
final long segmentOrd = segmentOrds.ordAt(i); final long segmentOrd = ords.ordAt(i);
segmentDocCounts.increment(segmentOrd + 1, 1); segmentDocCounts.increment(segmentOrd + 1, 1);
} }
} }
@ -368,14 +366,15 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
final LeafBucketCollector sub) throws IOException {
if (segmentOrds != null) { if (segmentOrds != null) {
mapSegmentCountsToGlobalCounts(); mapSegmentCountsToGlobalCounts();
} }
globalOrds = valuesSource.globalOrdinalsValues(); globalOrds = valuesSource.globalOrdinalsValues(ctx);
segmentOrds = valuesSource.ordinalsValues(); segmentOrds = valuesSource.ordinalsValues(ctx);
collector = newCollector(segmentOrds); return newCollector(segmentOrds, sub);
} }
@Override @Override
@ -410,11 +409,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
} }
final long ord = i - 1; // remember we do +1 when counting final long ord = i - 1; // remember we do +1 when counting
final long globalOrd = mapping == null ? ord : mapping.getGlobalOrd(ord); final long globalOrd = mapping == null ? ord : mapping.getGlobalOrd(ord);
try { incrementBucketDocCount(globalOrd, inc);
incrementBucketDocCount(globalOrd, inc);
} catch (IOException e) {
throw ExceptionsHelper.convertToElastic(e);
}
} }
} }
} }

View File

@ -18,7 +18,6 @@
*/ */
package org.elasticsearch.search.aggregations.bucket.terms; package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.Version;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.util.Comparators; import org.elasticsearch.common.util.Comparators;
@ -220,7 +219,7 @@ class InternalOrder extends Terms.Order {
} }
} }
static class CompoundOrder extends Terms.Order{ static class CompoundOrder extends Terms.Order {
static final byte ID = -1; static final byte ID = -1;

View File

@ -25,7 +25,9 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue; import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude.LongFilter; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude.LongFilter;
@ -48,7 +50,6 @@ public class LongTermsAggregator extends TermsAggregator {
protected final @Nullable ValueFormatter formatter; protected final @Nullable ValueFormatter formatter;
protected final LongHash bucketOrds; protected final LongHash bucketOrds;
private boolean showTermDocCountError; private boolean showTermDocCountError;
private SortedNumericDocValues values;
private LongFilter longFilter; private LongFilter longFilter;
public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format,
@ -60,46 +61,46 @@ public class LongTermsAggregator extends TermsAggregator {
this.longFilter = longFilter; this.longFilter = longFilter;
bucketOrds = new LongHash(1, aggregationContext.bigArrays()); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return true; return (valuesSource != null && valuesSource.needsScores()) || super.needsScores();
} }
protected SortedNumericDocValues getValues(ValuesSource.Numeric valuesSource) { protected SortedNumericDocValues getValues(ValuesSource.Numeric valuesSource, LeafReaderContext ctx) throws IOException {
return valuesSource.longValues(); return valuesSource.longValues(ctx);
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = getValues(valuesSource); final LeafBucketCollector sub) throws IOException {
} final SortedNumericDocValues values = getValues(valuesSource, ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0;
values.setDocument(doc);
final int valuesCount = values.count();
@Override long previous = Long.MAX_VALUE;
public void collect(int doc, long owningBucketOrdinal) throws IOException { for (int i = 0; i < valuesCount; ++i) {
assert owningBucketOrdinal == 0; final long val = values.valueAt(i);
values.setDocument(doc); if (previous != val || i == 0) {
final int valuesCount = values.count(); if ((longFilter == null) || (longFilter.accept(val))) {
long bucketOrdinal = bucketOrds.add(val);
if (bucketOrdinal < 0) { // already seen
bucketOrdinal = - 1 - bucketOrdinal;
collectExistingBucket(sub, doc, bucketOrdinal);
} else {
collectBucket(sub, doc, bucketOrdinal);
}
}
long previous = Long.MAX_VALUE; previous = val;
for (int i = 0; i < valuesCount; ++i) { }
final long val = values.valueAt(i);
if (previous != val || i == 0) {
if ((longFilter == null) || (longFilter.accept(val))) {
long bucketOrdinal = bucketOrds.add(val);
if (bucketOrdinal < 0) { // already seen
bucketOrdinal = - 1 - bucketOrdinal;
collectExistingBucket(doc, bucketOrdinal);
} else {
collectBucket(doc, bucketOrdinal);
}
} }
previous = val;
} }
} };
} }
@Override @Override
@ -109,8 +110,7 @@ public class LongTermsAggregator extends TermsAggregator {
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
// we need to fill-in the blanks // we need to fill-in the blanks
for (LeafReaderContext ctx : context.searchContext().searcher().getTopReaderContext().leaves()) { for (LeafReaderContext ctx : context.searchContext().searcher().getTopReaderContext().leaves()) {
context.setNextReader(ctx); final SortedNumericDocValues values = getValues(valuesSource, ctx);
final SortedNumericDocValues values = getValues(valuesSource);
for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) { for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) {
values.setDocument(docId); values.setDocument(docId);
final int valueCount = values.count(); final int valueCount = values.count();
@ -148,7 +148,7 @@ public class LongTermsAggregator extends TermsAggregator {
list[i] = bucket; list[i] = bucket;
otherDocCount -= bucket.docCount; otherDocCount -= bucket.docCount;
} }
runDeferredCollections(survivingBucketOrds); runDeferredCollections(survivingBucketOrds);
//Now build the aggs //Now build the aggs
@ -156,11 +156,11 @@ public class LongTermsAggregator extends TermsAggregator {
list[i].aggregations = bucketAggregations(list[i].bucketOrd); list[i].aggregations = bucketAggregations(list[i].bucketOrd);
list[i].docCountError = 0; list[i].docCountError = 0;
} }
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData()); return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, metaData()); return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, metaData());

View File

@ -26,7 +26,9 @@ import org.elasticsearch.common.util.BytesRefHash;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue; import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -44,8 +46,6 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
private final ValuesSource valuesSource; private final ValuesSource valuesSource;
protected final BytesRefHash bucketOrds; protected final BytesRefHash bucketOrds;
private final IncludeExclude includeExclude; private final IncludeExclude includeExclude;
private SortedBinaryDocValues values;
private final BytesRefBuilder previous;
public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
Terms.Order order, BucketCountThresholds bucketCountThresholds, Terms.Order order, BucketCountThresholds bucketCountThresholds,
@ -55,44 +55,47 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;
bucketOrds = new BytesRefHash(1, aggregationContext.bigArrays()); bucketOrds = new BytesRefHash(1, aggregationContext.bigArrays());
previous = new BytesRefBuilder();
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return true; return (valuesSource != null && valuesSource.needsScores()) || super.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.bytesValues(); final LeafBucketCollector sub) throws IOException {
} final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
final BytesRefBuilder previous = new BytesRefBuilder();
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public void collect(int doc, long bucket) throws IOException {
assert owningBucketOrdinal == 0; assert bucket == 0;
values.setDocument(doc); values.setDocument(doc);
final int valuesCount = values.count(); final int valuesCount = values.count();
// SortedBinaryDocValues don't guarantee uniqueness so we need to take care of dups // SortedBinaryDocValues don't guarantee uniqueness so we need to take care of dups
previous.clear(); previous.clear();
for (int i = 0; i < valuesCount; ++i) { for (int i = 0; i < valuesCount; ++i) {
final BytesRef bytes = values.valueAt(i); final BytesRef bytes = values.valueAt(i);
if (includeExclude != null && !includeExclude.accept(bytes)) { if (includeExclude != null && !includeExclude.accept(bytes)) {
continue; continue;
}
if (previous.get().equals(bytes)) {
continue;
}
long bucketOrdinal = bucketOrds.add(bytes);
if (bucketOrdinal < 0) { // already seen
bucketOrdinal = - 1 - bucketOrdinal;
collectExistingBucket(sub, doc, bucketOrdinal);
} else {
collectBucket(sub, doc, bucketOrdinal);
}
previous.copyBytes(bytes);
}
} }
if (previous.get().equals(bytes)) { };
continue;
}
long bucketOrdinal = bucketOrds.add(bytes);
if (bucketOrdinal < 0) { // already seen
bucketOrdinal = - 1 - bucketOrdinal;
collectExistingBucket(doc, bucketOrdinal);
} else {
collectBucket(doc, bucketOrdinal);
}
previous.copyBytes(bytes);
}
} }
@Override @Override
@ -102,8 +105,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
// we need to fill-in the blanks // we need to fill-in the blanks
for (LeafReaderContext ctx : context.searchContext().searcher().getTopReaderContext().leaves()) { for (LeafReaderContext ctx : context.searchContext().searcher().getTopReaderContext().leaves()) {
context.setNextReader(ctx); final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
final SortedBinaryDocValues values = valuesSource.bytesValues();
// brute force // brute force
for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) { for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) {
values.setDocument(docId); values.setDocument(docId);

View File

@ -131,15 +131,15 @@ public abstract class TermsAggregator extends BucketsAggregator {
} }
protected final BucketCountThresholds bucketCountThresholds; protected final BucketCountThresholds bucketCountThresholds;
protected Terms.Order order; protected final Terms.Order order;
protected Set<Aggregator> aggsUsedForSorting = new HashSet<>(); protected final Set<Aggregator> aggsUsedForSorting = new HashSet<>();
protected SubAggCollectionMode subAggCollectMode; protected final SubAggCollectionMode collectMode;
public TermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode subAggCollectMode, Map<String, Object> metaData) throws IOException { public TermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode collectMode, Map<String, Object> metaData) throws IOException {
super(name, factories, context, parent, metaData); super(name, factories, context, parent, metaData);
this.bucketCountThresholds = bucketCountThresholds; this.bucketCountThresholds = bucketCountThresholds;
this.order = InternalOrder.validate(order, this); this.order = InternalOrder.validate(order, this);
this.subAggCollectMode = subAggCollectMode; this.collectMode = collectMode;
// Don't defer any child agg if we are dependent on it for pruning results // Don't defer any child agg if we are dependent on it for pruning results
if (order instanceof Aggregation){ if (order instanceof Aggregation){
AggregationPath path = ((Aggregation) order).path(); AggregationPath path = ((Aggregation) order).path();
@ -157,7 +157,7 @@ public abstract class TermsAggregator extends BucketsAggregator {
@Override @Override
protected boolean shouldDefer(Aggregator aggregator) { protected boolean shouldDefer(Aggregator aggregator) {
return (subAggCollectMode == SubAggCollectionMode.BREADTH_FIRST) && (!aggsUsedForSorting.contains(aggregator)); return (collectMode == SubAggCollectionMode.BREADTH_FIRST) && (!aggsUsedForSorting.contains(aggregator));
} }
} }

View File

@ -21,8 +21,13 @@ package org.elasticsearch.search.aggregations.bucket.terms;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -43,7 +48,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
return new StringTermsAggregator(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new StringTermsAggregator(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@ -58,9 +63,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@Override @Override
@ -73,9 +78,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@Override @Override
@ -87,12 +92,12 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
if (includeExclude != null || factories.count() > 0) { if (includeExclude != null || factories.count() > 0) {
return GLOBAL_ORDINALS.create(name, factories, valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return GLOBAL_ORDINALS.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@Override @Override
@ -117,8 +122,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
} }
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException; IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException;
abstract boolean needsGlobalOrdinals(); abstract boolean needsGlobalOrdinals();
@ -131,17 +137,17 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
private final Terms.Order order; private final Terms.Order order;
private final IncludeExclude includeExclude; private final IncludeExclude includeExclude;
private final String executionHint; private final String executionHint;
private SubAggCollectionMode subAggCollectMode; private final SubAggCollectionMode collectMode;
private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
private boolean showTermDocCountError; private final boolean showTermDocCountError;
public TermsAggregatorFactory(String name, ValuesSourceConfig config, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint,SubAggCollectionMode executionMode, boolean showTermDocCountError) { public TermsAggregatorFactory(String name, ValuesSourceConfig config, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint, SubAggCollectionMode executionMode, boolean showTermDocCountError) {
super(name, StringTerms.TYPE.name(), config); super(name, StringTerms.TYPE.name(), config);
this.order = order; this.order = order;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;
this.executionHint = executionHint; this.executionHint = executionHint;
this.bucketCountThresholds = bucketCountThresholds; this.bucketCountThresholds = bucketCountThresholds;
this.subAggCollectMode = executionMode; this.collectMode = executionMode;
this.showTermDocCountError = showTermDocCountError; this.showTermDocCountError = showTermDocCountError;
} }
@ -211,8 +217,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
} }
assert execution != null; assert execution != null;
valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals()); return execution.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectMode, showTermDocCountError, metaData);
return execution.create(name, factories, valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
if ((includeExclude != null) && (includeExclude.isRegexBased())) { if ((includeExclude != null) && (includeExclude.isRegexBased())) {
@ -227,14 +232,14 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
longFilter = includeExclude.convertToDoubleFilter(); longFilter = includeExclude.convertToDoubleFilter();
} }
return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, order, bucketCountThresholds, aggregationContext, parent, collectMode,
showTermDocCountError, longFilter, metaData); showTermDocCountError, longFilter, metaData);
} }
if (includeExclude != null) { if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter(); longFilter = includeExclude.convertToLongFilter();
} }
return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, longFilter, metaData); order, bucketCountThresholds, aggregationContext, parent, collectMode, showTermDocCountError, longFilter, metaData);
} }
throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field() + throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field() +

View File

@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.ValuesSourceAggregationBuilder; import org.elasticsearch.search.aggregations.ValuesSourceAggregationBuilder;
@ -282,7 +281,7 @@ public class TermsBuilder extends ValuesSourceAggregationBuilder<TermsBuilder> {
order.toXContent(builder, params); order.toXContent(builder, params);
} }
if (collectionMode != null) { if (collectionMode != null) {
builder.field(Aggregator.COLLECT_MODE.getPreferredName(), collectionMode.parseField().getPreferredName()); builder.field(SubAggCollectionMode.KEY.getPreferredName(), collectionMode.parseField().getPreferredName());
} }
if (includeTerms != null) { if (includeTerms != null) {
builder.array("include", includeTerms); builder.array("include", includeTerms);

View File

@ -175,7 +175,7 @@ public class IncludeExclude {
if(hasRegexTest) { if(hasRegexTest) {
// We have includeVals that are a regex or only regex excludes - we need to do the potentially // We have includeVals that are a regex or only regex excludes - we need to do the potentially
// slow option of hitting termsEnum for every term in the index. // slow option of hitting termsEnum for every term in the index.
TermsEnum globalTermsEnum = valueSource.globalOrdinalsValues().termsEnum(); TermsEnum globalTermsEnum = globalOrdinals.termsEnum();
try { try {
for (BytesRef term = globalTermsEnum.next(); term != null; term = globalTermsEnum.next()) { for (BytesRef term = globalTermsEnum.next(); term != null; term = globalTermsEnum.next()) {
if (accept(term)) { if (accept(term)) {

View File

@ -20,13 +20,14 @@
package org.elasticsearch.search.aggregations.metrics; package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
public abstract class MetricsAggregator extends Aggregator { public abstract class MetricsAggregator extends AggregatorBase {
protected MetricsAggregator(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { protected MetricsAggregator(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, AggregatorFactories.EMPTY, context, parent, metaData); super(name, AggregatorFactories.EMPTY, context, parent, metaData);

View File

@ -21,11 +21,14 @@ package org.elasticsearch.search.aggregations.metrics.avg;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.common.util.LongArray; import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -41,12 +44,11 @@ import java.util.Map;
*/ */
public class AvgAggregator extends NumericMetricsAggregator.SingleValue { public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
private final ValuesSource.Numeric valuesSource; final ValuesSource.Numeric valuesSource;
private SortedNumericDoubleValues values;
private LongArray counts; LongArray counts;
private DoubleArray sums; DoubleArray sums;
private ValueFormatter formatter; ValueFormatter formatter;
public AvgAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public AvgAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
@ -54,34 +56,41 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays();
counts = bigArrays.newLongArray(1, true); counts = bigArrays.newLongArray(1, true);
sums = bigArrays.newDoubleArray(1, true); sums = bigArrays.newDoubleArray(1, true);
} }
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return valuesSource != null && valuesSource.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
counts = bigArrays.grow(counts, owningBucketOrdinal + 1);
sums = bigArrays.grow(sums, owningBucketOrdinal + 1);
values.setDocument(doc);
final int valueCount = values.count();
counts.increment(owningBucketOrdinal, valueCount);
double sum = 0;
for (int i = 0; i < valueCount; i++) {
sum += values.valueAt(i);
} }
sums.increment(owningBucketOrdinal, sum); final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
counts = bigArrays.grow(counts, bucket + 1);
sums = bigArrays.grow(sums, bucket + 1);
values.setDocument(doc);
final int valueCount = values.count();
counts.increment(bucket, valueCount);
double sum = 0;
for (int i = 0; i < valueCount; i++) {
sum += values.valueAt(i);
}
sums.increment(bucket, sum);
}
};
} }
@Override @Override
@ -90,11 +99,11 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long bucket) {
if (valuesSource == null || owningBucketOrdinal >= counts.size()) { if (valuesSource == null || bucket >= sums.size()) {
return new InternalAvg(name, 0l, 0, formatter, metaData()); return buildEmptyAggregation();
} }
return new InternalAvg(name, sums.get(owningBucketOrdinal), counts.get(owningBucketOrdinal), formatter, metaData()); return new InternalAvg(name, sums.get(bucket), counts.get(bucket), formatter, metaData());
} }
@Override @Override

View File

@ -40,6 +40,7 @@ import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -70,35 +71,36 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.rehash = rehash; this.rehash = rehash;
this.precision = precision; this.precision = precision;
this.counts = valuesSource == null ? null : new HyperLogLogPlusPlus(precision, bigArrays, 1); this.counts = valuesSource == null ? null : new HyperLogLogPlusPlus(precision, context.bigArrays(), 1);
this.formatter = formatter; this.formatter = formatter;
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public boolean needsScores() {
postCollectLastCollector(); return valuesSource != null && valuesSource.needsScores();
collector = createCollector(reader);
} }
private Collector createCollector(LeafReaderContext reader) { private Collector pickCollector(LeafReaderContext ctx) throws IOException {
if (valuesSource == null) {
return new EmptyCollector();
}
// if rehash is false then the value source is either already hashed, or the user explicitly // if rehash is false then the value source is either already hashed, or the user explicitly
// requested not to hash the values (perhaps they already hashed the values themselves before indexing the doc) // requested not to hash the values (perhaps they already hashed the values themselves before indexing the doc)
// so we can just work with the original value source as is // so we can just work with the original value source as is
if (!rehash) { if (!rehash) {
MurmurHash3Values hashValues = MurmurHash3Values.cast(((ValuesSource.Numeric) valuesSource).longValues()); MurmurHash3Values hashValues = MurmurHash3Values.cast(((ValuesSource.Numeric) valuesSource).longValues(ctx));
return new DirectCollector(counts, hashValues); return new DirectCollector(counts, hashValues);
} }
if (valuesSource instanceof ValuesSource.Numeric) { if (valuesSource instanceof ValuesSource.Numeric) {
ValuesSource.Numeric source = (ValuesSource.Numeric) valuesSource; ValuesSource.Numeric source = (ValuesSource.Numeric) valuesSource;
MurmurHash3Values hashValues = source.isFloatingPoint() ? MurmurHash3Values.hash(source.doubleValues()) : MurmurHash3Values.hash(source.longValues()); MurmurHash3Values hashValues = source.isFloatingPoint() ? MurmurHash3Values.hash(source.doubleValues(ctx)) : MurmurHash3Values.hash(source.longValues(ctx));
return new DirectCollector(counts, hashValues); return new DirectCollector(counts, hashValues);
} }
if (valuesSource instanceof ValuesSource.Bytes.WithOrdinals) { if (valuesSource instanceof ValuesSource.Bytes.WithOrdinals) {
ValuesSource.Bytes.WithOrdinals source = (ValuesSource.Bytes.WithOrdinals) valuesSource; ValuesSource.Bytes.WithOrdinals source = (ValuesSource.Bytes.WithOrdinals) valuesSource;
final RandomAccessOrds ordinalValues = source.ordinalsValues(); final RandomAccessOrds ordinalValues = source.ordinalsValues(ctx);
final long maxOrd = ordinalValues.getValueCount(); final long maxOrd = ordinalValues.getValueCount();
if (maxOrd == 0) { if (maxOrd == 0) {
return new EmptyCollector(); return new EmptyCollector();
@ -108,22 +110,20 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
final long countsMemoryUsage = HyperLogLogPlusPlus.memoryUsage(precision); final long countsMemoryUsage = HyperLogLogPlusPlus.memoryUsage(precision);
// only use ordinals if they don't increase memory usage by more than 25% // only use ordinals if they don't increase memory usage by more than 25%
if (ordinalsMemoryUsage < countsMemoryUsage / 4) { if (ordinalsMemoryUsage < countsMemoryUsage / 4) {
return new OrdinalsCollector(counts, ordinalValues, bigArrays); return new OrdinalsCollector(counts, ordinalValues, context.bigArrays());
} }
} }
return new DirectCollector(counts, MurmurHash3Values.hash(valuesSource.bytesValues())); return new DirectCollector(counts, MurmurHash3Values.hash(valuesSource.bytesValues(ctx)));
}
@Override
public boolean shouldCollect() {
return valuesSource != null;
} }
@Override @Override
public void collect(int doc, long owningBucketOrdinal) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
collector.collect(doc, owningBucketOrdinal); final LeafBucketCollector sub) throws IOException {
postCollectLastCollector();
collector = pickCollector(ctx);
return collector;
} }
private void postCollectLastCollector() { private void postCollectLastCollector() {
@ -169,15 +169,13 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
Releasables.close(counts, collector); Releasables.close(counts, collector);
} }
private static interface Collector extends Releasable { private static abstract class Collector extends LeafBucketCollector implements Releasable {
void collect(int doc, long bucketOrd); public abstract void postCollect();
void postCollect();
} }
private static class EmptyCollector implements Collector { private static class EmptyCollector extends Collector {
@Override @Override
public void collect(int doc, long bucketOrd) { public void collect(int doc, long bucketOrd) {
@ -195,7 +193,7 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
} }
} }
private static class DirectCollector implements Collector { private static class DirectCollector extends Collector {
private final MurmurHash3Values hashes; private final MurmurHash3Values hashes;
private final HyperLogLogPlusPlus counts; private final HyperLogLogPlusPlus counts;
@ -226,7 +224,7 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
} }
private static class OrdinalsCollector implements Collector { private static class OrdinalsCollector extends Collector {
private static final long SHALLOW_FIXEDBITSET_SIZE = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class); private static final long SHALLOW_FIXEDBITSET_SIZE = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class);

View File

@ -63,7 +63,7 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<V
* because otherwise it might be memory-intensive. On the other hand, for top-level aggregators * because otherwise it might be memory-intensive. On the other hand, for top-level aggregators
* we try to focus on accuracy. * we try to focus on accuracy.
*/ */
private int defaultPrecision(Aggregator parent) { private static int defaultPrecision(Aggregator parent) {
int precision = HyperLogLogPlusPlus.DEFAULT_PRECISION; int precision = HyperLogLogPlusPlus.DEFAULT_PRECISION;
while (parent != null) { while (parent != null) {
if (parent instanceof SingleBucketAggregator == false) { if (parent instanceof SingleBucketAggregator == false) {

View File

@ -22,10 +22,13 @@ package org.elasticsearch.search.aggregations.metrics.geobounds;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.MultiGeoPointValues; import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -39,13 +42,12 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
private final ValuesSource.GeoPoint valuesSource; private final ValuesSource.GeoPoint valuesSource;
private final boolean wrapLongitude; private final boolean wrapLongitude;
private DoubleArray tops; DoubleArray tops;
private DoubleArray bottoms; DoubleArray bottoms;
private DoubleArray posLefts; DoubleArray posLefts;
private DoubleArray posRights; DoubleArray posRights;
private DoubleArray negLefts; DoubleArray negLefts;
private DoubleArray negRights; DoubleArray negRights;
private MultiGeoPointValues values;
protected GeoBoundsAggregator(String name, AggregationContext aggregationContext, protected GeoBoundsAggregator(String name, AggregationContext aggregationContext,
Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, Map<String, Object> metaData) throws IOException { Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, Map<String, Object> metaData) throws IOException {
@ -53,6 +55,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.wrapLongitude = wrapLongitude; this.wrapLongitude = wrapLongitude;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays();
tops = bigArrays.newDoubleArray(1, false); tops = bigArrays.newDoubleArray(1, false);
tops.fill(0, tops.size(), Double.NEGATIVE_INFINITY); tops.fill(0, tops.size(), Double.NEGATIVE_INFINITY);
bottoms = bigArrays.newDoubleArray(1, false); bottoms = bigArrays.newDoubleArray(1, false);
@ -69,13 +72,70 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
} }
@Override @Override
public boolean shouldCollect() { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
return valuesSource != null; LeafBucketCollector sub) {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
}
final BigArrays bigArrays = context.bigArrays();
final MultiGeoPointValues values = valuesSource.geoPointValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
if (bucket >= tops.size()) {
long from = tops.size();
tops = bigArrays.grow(tops, bucket + 1);
tops.fill(from, tops.size(), Double.NEGATIVE_INFINITY);
bottoms = bigArrays.resize(bottoms, tops.size());
bottoms.fill(from, bottoms.size(), Double.NEGATIVE_INFINITY);
posLefts = bigArrays.resize(posLefts, tops.size());
posLefts.fill(from, posLefts.size(), Double.NEGATIVE_INFINITY);
posRights = bigArrays.resize(posRights, tops.size());
posRights.fill(from, posRights.size(), Double.NEGATIVE_INFINITY);
negLefts = bigArrays.resize(negLefts, tops.size());
negLefts.fill(from, negLefts.size(), Double.NEGATIVE_INFINITY);
negRights = bigArrays.resize(negRights, tops.size());
negRights.fill(from, negRights.size(), Double.NEGATIVE_INFINITY);
}
@Override values.setDocument(doc);
public void setNextReader(LeafReaderContext reader) { final int valuesCount = values.count();
this.values = this.valuesSource.geoPointValues();
for (int i = 0; i < valuesCount; ++i) {
GeoPoint value = values.valueAt(i);
double top = tops.get(bucket);
if (value.lat() > top) {
top = value.lat();
}
double bottom = bottoms.get(bucket);
if (value.lat() < bottom) {
bottom = value.lat();
}
double posLeft = posLefts.get(bucket);
if (value.lon() > 0 && value.lon() < posLeft) {
posLeft = value.lon();
}
double posRight = posRights.get(bucket);
if (value.lon() > 0 && value.lon() > posRight) {
posRight = value.lon();
}
double negLeft = negLefts.get(bucket);
if (value.lon() < 0 && value.lon() < negLeft) {
negLeft = value.lon();
}
double negRight = negRights.get(bucket);
if (value.lon() < 0 && value.lon() > negRight) {
negRight = value.lon();
}
tops.set(bucket, top);
bottoms.set(bucket, bottom);
posLefts.set(bucket, posLeft);
posRights.set(bucket, posRight);
negLefts.set(bucket, negLeft);
negRights.set(bucket, negRight);
}
}
};
} }
@Override @Override
@ -97,62 +157,6 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
return new InternalGeoBounds(name, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, return new InternalGeoBounds(name, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY,
Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, metaData()); Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, metaData());
} }
@Override
public void collect(int docId, long owningBucketOrdinal) throws IOException {
if (owningBucketOrdinal >= tops.size()) {
long from = tops.size();
tops = bigArrays.grow(tops, owningBucketOrdinal + 1);
tops.fill(from, tops.size(), Double.NEGATIVE_INFINITY);
bottoms = bigArrays.resize(bottoms, tops.size());
bottoms.fill(from, bottoms.size(), Double.NEGATIVE_INFINITY);
posLefts = bigArrays.resize(posLefts, tops.size());
posLefts.fill(from, posLefts.size(), Double.NEGATIVE_INFINITY);
posRights = bigArrays.resize(posRights, tops.size());
posRights.fill(from, posRights.size(), Double.NEGATIVE_INFINITY);
negLefts = bigArrays.resize(negLefts, tops.size());
negLefts.fill(from, negLefts.size(), Double.NEGATIVE_INFINITY);
negRights = bigArrays.resize(negRights, tops.size());
negRights.fill(from, negRights.size(), Double.NEGATIVE_INFINITY);
}
values.setDocument(docId);
final int valuesCount = values.count();
for (int i = 0; i < valuesCount; ++i) {
GeoPoint value = values.valueAt(i);
double top = tops.get(owningBucketOrdinal);
if (value.lat() > top) {
top = value.lat();
}
double bottom = bottoms.get(owningBucketOrdinal);
if (value.lat() < bottom) {
bottom = value.lat();
}
double posLeft = posLefts.get(owningBucketOrdinal);
if (value.lon() > 0 && value.lon() < posLeft) {
posLeft = value.lon();
}
double posRight = posRights.get(owningBucketOrdinal);
if (value.lon() > 0 && value.lon() > posRight) {
posRight = value.lon();
}
double negLeft = negLefts.get(owningBucketOrdinal);
if (value.lon() < 0 && value.lon() < negLeft) {
negLeft = value.lon();
}
double negRight = negRights.get(owningBucketOrdinal);
if (value.lon() < 0 && value.lon() > negRight) {
negRight = value.lon();
}
tops.set(owningBucketOrdinal, top);
bottoms.set(owningBucketOrdinal, bottom);
posLefts.set(owningBucketOrdinal, posLeft);
posRights.set(owningBucketOrdinal, posRight);
negLefts.set(owningBucketOrdinal, negLeft);
negRights.set(owningBucketOrdinal, negRight);
}
}
@Override @Override
public void doClose() { public void doClose() {

View File

@ -21,12 +21,15 @@ package org.elasticsearch.search.aggregations.metrics.max;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.NumericDoubleValues; import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.MultiValueMode; import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -42,11 +45,10 @@ import java.util.Map;
*/ */
public class MaxAggregator extends NumericMetricsAggregator.SingleValue { public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
private final ValuesSource.Numeric valuesSource; final ValuesSource.Numeric valuesSource;
private NumericDoubleValues values; final ValueFormatter formatter;
private DoubleArray maxes; DoubleArray maxes;
private ValueFormatter formatter;
public MaxAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public MaxAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
@ -54,33 +56,41 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
maxes = bigArrays.newDoubleArray(1, false); maxes = context.bigArrays().newDoubleArray(1, false);
maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY);
} }
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return valuesSource != null && valuesSource.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
final SortedNumericDoubleValues values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
this.values = MultiValueMode.MAX.select(values, Double.NEGATIVE_INFINITY); if (valuesSource == null) {
} return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
if (owningBucketOrdinal >= maxes.size()) {
long from = maxes.size();
maxes = bigArrays.grow(maxes, owningBucketOrdinal + 1);
maxes.fill(from, maxes.size(), Double.NEGATIVE_INFINITY);
} }
final double value = values.get(doc); final BigArrays bigArrays = context.bigArrays();
double max = maxes.get(owningBucketOrdinal); final SortedNumericDoubleValues allValues = valuesSource.doubleValues(ctx);
max = Math.max(max, value); final NumericDoubleValues values = MultiValueMode.MAX.select(allValues, Double.NEGATIVE_INFINITY);
maxes.set(owningBucketOrdinal, max); return new LeafBucketCollectorBase(sub, allValues) {
@Override
public void collect(int doc, long bucket) throws IOException {
if (bucket >= maxes.size()) {
long from = maxes.size();
maxes = bigArrays.grow(maxes, bucket + 1);
maxes.fill(from, maxes.size(), Double.NEGATIVE_INFINITY);
}
final double value = values.get(doc);
double max = maxes.get(bucket);
max = Math.max(max, value);
maxes.set(bucket, max);
}
};
} }
@Override @Override
@ -89,12 +99,11 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long bucket) {
if (valuesSource == null) { if (valuesSource == null || bucket >= maxes.size()) {
return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, metaData()); return buildEmptyAggregation();
} }
assert owningBucketOrdinal < maxes.size(); return new InternalMax(name, maxes.get(bucket), formatter, metaData());
return new InternalMax(name, maxes.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override

View File

@ -21,12 +21,15 @@ package org.elasticsearch.search.aggregations.metrics.min;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.NumericDoubleValues; import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.MultiValueMode; import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -42,45 +45,52 @@ import java.util.Map;
*/ */
public class MinAggregator extends NumericMetricsAggregator.SingleValue { public class MinAggregator extends NumericMetricsAggregator.SingleValue {
private final ValuesSource.Numeric valuesSource; final ValuesSource.Numeric valuesSource;
private NumericDoubleValues values; final ValueFormatter formatter;
private DoubleArray mins; DoubleArray mins;
private ValueFormatter formatter;
public MinAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public MinAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
if (valuesSource != null) { if (valuesSource != null) {
mins = bigArrays.newDoubleArray(1, false); mins = context.bigArrays().newDoubleArray(1, false);
mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY);
} }
this.formatter = formatter; this.formatter = formatter;
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return valuesSource != null && valuesSource.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
final SortedNumericDoubleValues values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
this.values = MultiValueMode.MIN.select(values, Double.POSITIVE_INFINITY); if (valuesSource == null) {
} return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
if (owningBucketOrdinal >= mins.size()) {
long from = mins.size();
mins = bigArrays.grow(mins, owningBucketOrdinal + 1);
mins.fill(from, mins.size(), Double.POSITIVE_INFINITY);
} }
final double value = values.get(doc); final BigArrays bigArrays = context.bigArrays();
double min = mins.get(owningBucketOrdinal); final SortedNumericDoubleValues allValues = valuesSource.doubleValues(ctx);
min = Math.min(min, value); final NumericDoubleValues values = MultiValueMode.MIN.select(allValues, Double.POSITIVE_INFINITY);
mins.set(owningBucketOrdinal, min); return new LeafBucketCollectorBase(sub, allValues) {
@Override
public void collect(int doc, long bucket) throws IOException {
if (bucket >= mins.size()) {
long from = mins.size();
mins = bigArrays.grow(mins, bucket + 1);
mins.fill(from, mins.size(), Double.POSITIVE_INFINITY);
}
final double value = values.get(doc);
double min = mins.get(bucket);
min = Math.min(min, value);
mins.set(bucket, min);
}
};
} }
@Override @Override
@ -89,12 +99,11 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long bucket) {
if (valuesSource == null) { if (valuesSource == null || bucket >= mins.size()) {
return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, metaData()); return buildEmptyAggregation();
} }
assert owningBucketOrdinal < mins.size(); return new InternalMin(name, mins.get(bucket), formatter, metaData());
return new InternalMin(name, mins.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override

View File

@ -23,9 +23,12 @@ import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.ArrayUtils; import org.elasticsearch.common.util.ArrayUtils;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -43,11 +46,10 @@ public abstract class AbstractPercentilesAggregator extends NumericMetricsAggreg
protected final double[] keys; protected final double[] keys;
protected final ValuesSource.Numeric valuesSource; protected final ValuesSource.Numeric valuesSource;
private SortedNumericDoubleValues values; protected final ValueFormatter formatter;
protected ObjectArray<TDigestState> states; protected ObjectArray<TDigestState> states;
protected final double compression; protected final double compression;
protected final boolean keyed; protected final boolean keyed;
protected ValueFormatter formatter;
public AbstractPercentilesAggregator(String name, ValuesSource.Numeric valuesSource, AggregationContext context, public AbstractPercentilesAggregator(String name, ValuesSource.Numeric valuesSource, AggregationContext context,
Aggregator parent, double[] keys, double compression, boolean keyed, Aggregator parent, double[] keys, double compression, boolean keyed,
@ -56,36 +58,42 @@ public abstract class AbstractPercentilesAggregator extends NumericMetricsAggreg
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.keyed = keyed; this.keyed = keyed;
this.formatter = formatter; this.formatter = formatter;
this.states = bigArrays.newObjectArray(1); this.states = context.bigArrays().newObjectArray(1);
this.keys = keys; this.keys = keys;
this.compression = compression; this.compression = compression;
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return valuesSource != null && valuesSource.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
}
final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
states = bigArrays.grow(states, bucket + 1);
@Override TDigestState state = states.get(bucket);
public void collect(int doc, long bucketOrd) throws IOException { if (state == null) {
states = bigArrays.grow(states, bucketOrd + 1); state = new TDigestState(compression);
states.set(bucket, state);
TDigestState state = states.get(bucketOrd); }
if (state == null) {
state = new TDigestState(compression); values.setDocument(doc);
states.set(bucketOrd, state); final int valueCount = values.count();
} for (int i = 0; i < valueCount; i++) {
state.add(values.valueAt(i));
values.setDocument(doc); }
final int valueCount = values.count(); }
for (int i = 0; i < valueCount; i++) { };
state.add(values.valueAt(i));
}
} }
@Override @Override

View File

@ -27,13 +27,18 @@ import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
import java.util.*; import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
public class ScriptedMetricAggregator extends MetricsAggregator { public class ScriptedMetricAggregator extends MetricsAggregator {
@ -47,8 +52,8 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
private final Map<String, Object> params; private final Map<String, Object> params;
// initial parameters for {reduce} // initial parameters for {reduce}
private final Map<String, Object> reduceParams; private final Map<String, Object> reduceParams;
private ScriptService scriptService; private final ScriptService scriptService;
private ScriptType reduceScriptType; private final ScriptType reduceScriptType;
protected ScriptedMetricAggregator(String name, String scriptLang, ScriptType initScriptType, String initScript, protected ScriptedMetricAggregator(String name, String scriptLang, ScriptType initScriptType, String initScript,
ScriptType mapScriptType, String mapScript, ScriptType combineScriptType, String combineScript, ScriptType reduceScriptType, ScriptType mapScriptType, String mapScript, ScriptType combineScriptType, String combineScript, ScriptType reduceScriptType,
@ -81,20 +86,22 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return true; return true; // TODO: how can we know if the script relies on scores?
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) throws IOException { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
mapScript.setNextReader(reader); final LeafBucketCollector sub) throws IOException {
} mapScript.setNextReader(ctx);
return new LeafBucketCollectorBase(sub, mapScript) {
@Override @Override
public void collect(int docId, long bucketOrdinal) throws IOException { public void collect(int doc, long bucket) throws IOException {
assert bucketOrdinal == 0 : bucketOrdinal; assert bucket == 0 : bucket;
mapScript.setNextDocId(docId); mapScript.setNextDocId(doc);
mapScript.run(); mapScript.run();
}
};
} }
@Override @Override

View File

@ -28,6 +28,8 @@ import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -43,20 +45,21 @@ import java.util.Map;
*/ */
public class StatsAggegator extends NumericMetricsAggregator.MultiValue { public class StatsAggegator extends NumericMetricsAggregator.MultiValue {
private final ValuesSource.Numeric valuesSource; final ValuesSource.Numeric valuesSource;
private SortedNumericDoubleValues values; final ValueFormatter formatter;
LongArray counts;
DoubleArray sums;
DoubleArray mins;
DoubleArray maxes;
private LongArray counts;
private DoubleArray sums;
private DoubleArray mins;
private DoubleArray maxes;
private ValueFormatter formatter;
public StatsAggegator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public StatsAggegator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays();
counts = bigArrays.newLongArray(1, true); counts = bigArrays.newLongArray(1, true);
sums = bigArrays.newDoubleArray(1, true); sums = bigArrays.newDoubleArray(1, true);
mins = bigArrays.newDoubleArray(1, false); mins = bigArrays.newDoubleArray(1, false);
@ -68,43 +71,49 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue {
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return valuesSource != null && valuesSource.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
if (owningBucketOrdinal >= counts.size()) {
final long from = counts.size();
final long overSize = BigArrays.overSize(owningBucketOrdinal + 1);
counts = bigArrays.resize(counts, overSize);
sums = bigArrays.resize(sums, overSize);
mins = bigArrays.resize(mins, overSize);
maxes = bigArrays.resize(maxes, overSize);
mins.fill(from, overSize, Double.POSITIVE_INFINITY);
maxes.fill(from, overSize, Double.NEGATIVE_INFINITY);
} }
final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
if (bucket >= counts.size()) {
final long from = counts.size();
final long overSize = BigArrays.overSize(bucket + 1);
counts = bigArrays.resize(counts, overSize);
sums = bigArrays.resize(sums, overSize);
mins = bigArrays.resize(mins, overSize);
maxes = bigArrays.resize(maxes, overSize);
mins.fill(from, overSize, Double.POSITIVE_INFINITY);
maxes.fill(from, overSize, Double.NEGATIVE_INFINITY);
}
values.setDocument(doc); values.setDocument(doc);
final int valuesCount = values.count(); final int valuesCount = values.count();
counts.increment(owningBucketOrdinal, valuesCount); counts.increment(bucket, valuesCount);
double sum = 0; double sum = 0;
double min = mins.get(owningBucketOrdinal); double min = mins.get(bucket);
double max = maxes.get(owningBucketOrdinal); double max = maxes.get(bucket);
for (int i = 0; i < valuesCount; i++) { for (int i = 0; i < valuesCount; i++) {
double value = values.valueAt(i); double value = values.valueAt(i);
sum += value; sum += value;
min = Math.min(min, value); min = Math.min(min, value);
max = Math.max(max, value); max = Math.max(max, value);
} }
sums.increment(owningBucketOrdinal, sum); sums.increment(bucket, sum);
mins.set(owningBucketOrdinal, min); mins.set(bucket, min);
maxes.set(owningBucketOrdinal, max); maxes.set(bucket, max);
}
};
} }
@Override @Override
@ -131,13 +140,12 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long bucket) {
if (valuesSource == null) { if (valuesSource == null || bucket >= sums.size()) {
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, metaData()); return buildEmptyAggregation();
} }
assert owningBucketOrdinal < counts.size(); return new InternalStats(name, counts.get(bucket), sums.get(bucket), mins.get(bucket),
return new InternalStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), mins.get(owningBucketOrdinal), maxes.get(bucket), formatter, metaData());
maxes.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override

View File

@ -28,6 +28,8 @@ import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -43,16 +45,15 @@ import java.util.Map;
*/ */
public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue { public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue {
private final ValuesSource.Numeric valuesSource; final ValuesSource.Numeric valuesSource;
private SortedNumericDoubleValues values; final ValueFormatter formatter;
final double sigma;
private LongArray counts; LongArray counts;
private DoubleArray sums; DoubleArray sums;
private DoubleArray mins; DoubleArray mins;
private DoubleArray maxes; DoubleArray maxes;
private DoubleArray sumOfSqrs; DoubleArray sumOfSqrs;
private ValueFormatter formatter;
private double sigma;
public ExtendedStatsAggregator(String name, ValuesSource.Numeric valuesSource, public ExtendedStatsAggregator(String name, ValuesSource.Numeric valuesSource,
@Nullable ValueFormatter formatter, AggregationContext context, @Nullable ValueFormatter formatter, AggregationContext context,
@ -60,7 +61,9 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
super(name, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
this.sigma = sigma;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays();
counts = bigArrays.newLongArray(1, true); counts = bigArrays.newLongArray(1, true);
sums = bigArrays.newDoubleArray(1, true); sums = bigArrays.newDoubleArray(1, true);
mins = bigArrays.newDoubleArray(1, false); mins = bigArrays.newDoubleArray(1, false);
@ -68,52 +71,59 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
maxes = bigArrays.newDoubleArray(1, false); maxes = bigArrays.newDoubleArray(1, false);
maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY);
sumOfSqrs = bigArrays.newDoubleArray(1, true); sumOfSqrs = bigArrays.newDoubleArray(1, true);
this.sigma = sigma;
} }
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return valuesSource != null && valuesSource.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
if (owningBucketOrdinal >= counts.size()) {
final long from = counts.size();
final long overSize = BigArrays.overSize(owningBucketOrdinal + 1);
counts = bigArrays.resize(counts, overSize);
sums = bigArrays.resize(sums, overSize);
mins = bigArrays.resize(mins, overSize);
maxes = bigArrays.resize(maxes, overSize);
sumOfSqrs = bigArrays.resize(sumOfSqrs, overSize);
mins.fill(from, overSize, Double.POSITIVE_INFINITY);
maxes.fill(from, overSize, Double.NEGATIVE_INFINITY);
} }
final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
values.setDocument(doc); @Override
final int valuesCount = values.count(); public void collect(int doc, long bucket) throws IOException {
counts.increment(owningBucketOrdinal, valuesCount); if (bucket >= counts.size()) {
double sum = 0; final long from = counts.size();
double sumOfSqr = 0; final long overSize = BigArrays.overSize(bucket + 1);
double min = mins.get(owningBucketOrdinal); counts = bigArrays.resize(counts, overSize);
double max = maxes.get(owningBucketOrdinal); sums = bigArrays.resize(sums, overSize);
for (int i = 0; i < valuesCount; i++) { mins = bigArrays.resize(mins, overSize);
double value = values.valueAt(i); maxes = bigArrays.resize(maxes, overSize);
sum += value; sumOfSqrs = bigArrays.resize(sumOfSqrs, overSize);
sumOfSqr += value * value; mins.fill(from, overSize, Double.POSITIVE_INFINITY);
min = Math.min(min, value); maxes.fill(from, overSize, Double.NEGATIVE_INFINITY);
max = Math.max(max, value); }
}
sums.increment(owningBucketOrdinal, sum); values.setDocument(doc);
sumOfSqrs.increment(owningBucketOrdinal, sumOfSqr); final int valuesCount = values.count();
mins.set(owningBucketOrdinal, min); counts.increment(bucket, valuesCount);
maxes.set(owningBucketOrdinal, max); double sum = 0;
double sumOfSqr = 0;
double min = mins.get(bucket);
double max = maxes.get(bucket);
for (int i = 0; i < valuesCount; i++) {
double value = values.valueAt(i);
sum += value;
sumOfSqr += value * value;
min = Math.min(min, value);
max = Math.max(max, value);
}
sums.increment(bucket, sum);
sumOfSqrs.increment(bucket, sumOfSqr);
mins.set(bucket, min);
maxes.set(bucket, max);
}
};
} }
@Override @Override

View File

@ -21,10 +21,13 @@ package org.elasticsearch.search.aggregations.metrics.sum;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -40,11 +43,10 @@ import java.util.Map;
*/ */
public class SumAggregator extends NumericMetricsAggregator.SingleValue { public class SumAggregator extends NumericMetricsAggregator.SingleValue {
private final ValuesSource.Numeric valuesSource; final ValuesSource.Numeric valuesSource;
private SortedNumericDoubleValues values; final ValueFormatter formatter;
private DoubleArray sums; DoubleArray sums;
private ValueFormatter formatter;
public SumAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public SumAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
@ -52,30 +54,36 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
sums = bigArrays.newDoubleArray(1, true); sums = context.bigArrays().newDoubleArray(1, true);
} }
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return valuesSource != null; return valuesSource != null && valuesSource.needsScores();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
values = valuesSource.doubleValues(); final LeafBucketCollector sub) throws IOException {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
sums = bigArrays.grow(sums, owningBucketOrdinal + 1);
values.setDocument(doc);
final int valuesCount = values.count();
double sum = 0;
for (int i = 0; i < valuesCount; i++) {
sum += values.valueAt(i);
} }
sums.increment(owningBucketOrdinal, sum); final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
sums = bigArrays.grow(sums, bucket + 1);
values.setDocument(doc);
final int valuesCount = values.count();
double sum = 0;
for (int i = 0; i < valuesCount; i++) {
sum += values.valueAt(i);
}
sums.increment(bucket, sum);
}
};
} }
@Override @Override
@ -84,11 +92,11 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long bucket) {
if (valuesSource == null) { if (valuesSource == null || bucket >= sums.size()) {
return new InternalSum(name, 0, formatter, metaData()); return buildEmptyAggregation();
} }
return new InternalSum(name, sums.get(owningBucketOrdinal), formatter, metaData()); return new InternalSum(name, sums.get(bucket), formatter, metaData());
} }
@Override @Override

View File

@ -20,13 +20,25 @@
package org.elasticsearch.search.aggregations.metrics.tophits; package org.elasticsearch.search.aggregations.metrics.tophits;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.*; import org.apache.lucene.search.FieldDoc;
import org.elasticsearch.ExceptionsHelper; import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopDocsCollector;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TopScoreDocCollector;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.ScorerAware;
import org.elasticsearch.common.util.LongObjectPagedHashMap; import org.elasticsearch.common.util.LongObjectPagedHashMap;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchPhase;
@ -40,7 +52,7 @@ import java.util.Map;
/** /**
*/ */
public class TopHitsAggregator extends MetricsAggregator implements ScorerAware { public class TopHitsAggregator extends MetricsAggregator {
/** Simple wrapper around a top-level collector and the current leaf collector. */ /** Simple wrapper around a top-level collector and the current leaf collector. */
private static class TopDocsAndLeafCollector { private static class TopDocsAndLeafCollector {
@ -52,24 +64,64 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
} }
} }
private final FetchPhase fetchPhase; final FetchPhase fetchPhase;
private final SubSearchContext subSearchContext; final SubSearchContext subSearchContext;
private final LongObjectPagedHashMap<TopDocsAndLeafCollector> topDocsCollectors; final LongObjectPagedHashMap<TopDocsAndLeafCollector> topDocsCollectors;
private Scorer currentScorer;
private LeafReaderContext currentContext;
public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException { public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, context, parent, metaData); super(name, context, parent, metaData);
this.fetchPhase = fetchPhase; this.fetchPhase = fetchPhase;
topDocsCollectors = new LongObjectPagedHashMap<>(1, context.bigArrays()); topDocsCollectors = new LongObjectPagedHashMap<>(1, context.bigArrays());
this.subSearchContext = subSearchContext; this.subSearchContext = subSearchContext;
context.registerScorerAware(this);
} }
@Override @Override
public boolean shouldCollect() { public boolean needsScores() {
return true; Sort sort = subSearchContext.sort();
if (sort != null) {
return sort.needsScores() || subSearchContext.trackScores();
} else {
// sort by score
return true;
}
}
@Override
public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx,
final LeafBucketCollector sub) throws IOException {
for (LongObjectPagedHashMap.Cursor<TopDocsAndLeafCollector> cursor : topDocsCollectors) {
cursor.value.leafCollector = cursor.value.topLevelCollector.getLeafCollector(ctx);
}
return new LeafBucketCollectorBase(sub, null) {
Scorer scorer;
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
for (LongObjectPagedHashMap.Cursor<TopDocsAndLeafCollector> cursor : topDocsCollectors) {
cursor.value.leafCollector.setScorer(scorer);
}
super.setScorer(scorer);
}
@Override
public void collect(int docId, long bucket) throws IOException {
TopDocsAndLeafCollector collectors = topDocsCollectors.get(bucket);
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()) : TopScoreDocCollector.create(topN);
collectors = new TopDocsAndLeafCollector(topLevelCollector);
collectors.leafCollector = collectors.topLevelCollector.getLeafCollector(ctx);
collectors.leafCollector.setScorer(scorer);
topDocsCollectors.put(bucket, collectors);
}
collectors.leafCollector.collect(docId);
}
};
} }
@Override @Override
@ -111,45 +163,6 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
return new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), Lucene.EMPTY_TOP_DOCS, InternalSearchHits.empty()); return new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), Lucene.EMPTY_TOP_DOCS, InternalSearchHits.empty());
} }
@Override
public void collect(int docId, long bucketOrdinal) throws IOException {
TopDocsAndLeafCollector collectors = topDocsCollectors.get(bucketOrdinal);
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()) : TopScoreDocCollector.create(topN);
collectors = new TopDocsAndLeafCollector(topLevelCollector);
collectors.leafCollector = collectors.topLevelCollector.getLeafCollector(currentContext);
collectors.leafCollector.setScorer(currentScorer);
topDocsCollectors.put(bucketOrdinal, collectors);
}
collectors.leafCollector.collect(docId);
}
@Override
public void setNextReader(LeafReaderContext context) {
this.currentContext = context;
for (LongObjectPagedHashMap.Cursor<TopDocsAndLeafCollector> cursor : topDocsCollectors) {
try {
cursor.value.leafCollector = cursor.value.topLevelCollector.getLeafCollector(currentContext);
} catch (IOException e) {
throw ExceptionsHelper.convertToElastic(e);
}
}
}
@Override
public void setScorer(Scorer scorer) {
this.currentScorer = scorer;
for (LongObjectPagedHashMap.Cursor<TopDocsAndLeafCollector> cursor : topDocsCollectors) {
try {
cursor.value.leafCollector.setScorer(scorer);
} catch (IOException e) {
throw ExceptionsHelper.convertToElastic(e);
}
}
}
@Override @Override
protected void doClose() { protected void doClose() {
Releasables.close(topDocsCollectors); Releasables.close(topDocsCollectors);
@ -175,16 +188,5 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
public AggregatorFactory subFactories(AggregatorFactories subFactories) { public AggregatorFactory subFactories(AggregatorFactories subFactories) {
throw new AggregationInitializationException("Aggregator [" + name + "] of type [" + type + "] cannot accept sub-aggregations"); throw new AggregationInitializationException("Aggregator [" + name + "] of type [" + type + "] cannot accept sub-aggregations");
} }
@Override
public boolean needsScores() {
Sort sort = subSearchContext.sort();
if (sort != null) {
return sort.needsScores() || subSearchContext.trackScores();
} else {
// sort by score
return true;
}
}
} }
} }

View File

@ -21,10 +21,13 @@ package org.elasticsearch.search.aggregations.metrics.valuecount;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.LongArray; import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -43,12 +46,11 @@ import java.util.Map;
*/ */
public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue { public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
private final ValuesSource valuesSource; final ValuesSource valuesSource;
private SortedBinaryDocValues values; final ValueFormatter formatter;
// a count per bucket // a count per bucket
LongArray counts; LongArray counts;
private ValueFormatter formatter;
public ValueCountAggregator(String name, ValuesSource valuesSource, @Nullable ValueFormatter formatter, public ValueCountAggregator(String name, ValuesSource valuesSource, @Nullable ValueFormatter formatter,
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException { AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
@ -56,25 +58,28 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
counts = bigArrays.newLongArray(1, true); counts = context.bigArrays().newLongArray(1, true);
} }
} }
@Override @Override
public boolean shouldCollect() { public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
return valuesSource != null; final LeafBucketCollector sub) throws IOException {
} if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
}
final BigArrays bigArrays = context.bigArrays();
final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override @Override
public void setNextReader(LeafReaderContext reader) { public void collect(int doc, long bucket) throws IOException {
values = valuesSource.bytesValues(); counts = bigArrays.grow(counts, bucket + 1);
} values.setDocument(doc);
counts.increment(bucket, values.count());
}
@Override };
public void collect(int doc, long owningBucketOrdinal) throws IOException {
counts = bigArrays.grow(counts, owningBucketOrdinal + 1);
values.setDocument(doc);
counts.increment(owningBucketOrdinal, values.count());
} }
@Override @Override
@ -83,12 +88,11 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long bucket) {
if (valuesSource == null) { if (valuesSource == null || bucket >= counts.size()) {
return new InternalValueCount(name, 0, formatter, metaData()); return buildEmptyAggregation();
} }
assert owningBucketOrdinal < counts.size(); return new InternalValueCount(name, counts.get(bucket), formatter, metaData());
return new InternalValueCount(name, counts.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override

View File

@ -18,17 +18,7 @@
*/ */
package org.elasticsearch.search.aggregations.support; package org.elasticsearch.search.aggregations.support;
import com.carrotsearch.hppc.ObjectObjectOpenHashMap;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.lucene.ReaderContextAware;
import org.elasticsearch.common.lucene.ScorerAware;
import org.elasticsearch.common.lucene.TopReaderContextAware;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData; import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
@ -39,37 +29,18 @@ import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/** /**
* *
*/ */
@SuppressWarnings({"unchecked"}) public class AggregationContext {
public class AggregationContext implements ReaderContextAware, ScorerAware {
private final SearchContext searchContext; private final SearchContext searchContext;
private ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource>[] perDepthFieldDataSources = new ObjectObjectOpenHashMap[4];
private List<ReaderContextAware> readerAwares = new ArrayList<>();
private List<TopReaderContextAware> topReaderAwares = new ArrayList<TopReaderContextAware>();
private List<ScorerAware> scorerAwares = new ArrayList<>();
private LeafReaderContext reader;
private Scorer scorer;
public AggregationContext(SearchContext searchContext) { public AggregationContext(SearchContext searchContext) {
this.searchContext = searchContext; this.searchContext = searchContext;
} }
/**
* Whether aggregators which are attached to this context need scores.
*/
public boolean needsScores() {
return searchContext.aggregations().factories().needsScores();
}
public SearchContext searchContext() { public SearchContext searchContext() {
return searchContext; return searchContext;
} }
@ -82,47 +53,11 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
return searchContext.bigArrays(); return searchContext.bigArrays();
} }
public LeafReaderContext currentReader() {
return reader;
}
public Scorer currentScorer() {
return scorer;
}
public void setNextReader(IndexReaderContext reader) {
for (TopReaderContextAware topReaderAware : topReaderAwares) {
topReaderAware.setNextReader(reader);
}
}
public void setNextReader(LeafReaderContext reader) throws IOException {
this.reader = reader;
for (ReaderContextAware aware : readerAwares) {
aware.setNextReader(reader);
}
}
public void setScorer(Scorer scorer) {
this.scorer = scorer;
for (ScorerAware scorerAware : scorerAwares) {
scorerAware.setScorer(scorer);
}
}
/** Get a value source given its configuration and the depth of the aggregator in the aggregation tree. */ /** 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 { public <VS extends ValuesSource> VS valuesSource(ValuesSourceConfig<VS> config) throws IOException {
assert config.valid() : "value source config is invalid - must have either a field context or a script or marked as unmapped"; assert config.valid() : "value source config is invalid - must have either a field context or a script or marked as unmapped";
assert !config.unmapped : "value source should not be created for unmapped fields"; assert !config.unmapped : "value source should not be created for unmapped fields";
if (perDepthFieldDataSources.length <= depth) {
perDepthFieldDataSources = Arrays.copyOf(perDepthFieldDataSources, ArrayUtil.oversize(1 + depth, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
}
if (perDepthFieldDataSources[depth] == null) {
perDepthFieldDataSources[depth] = new ObjectObjectOpenHashMap<>();
}
final ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources = perDepthFieldDataSources[depth];
if (config.fieldContext == null) { if (config.fieldContext == null) {
if (ValuesSource.Numeric.class.isAssignableFrom(config.valueSourceType)) { if (ValuesSource.Numeric.class.isAssignableFrom(config.valueSourceType)) {
return (VS) numericScript(config); return (VS) numericScript(config);
@ -134,146 +69,49 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
} }
if (ValuesSource.Numeric.class.isAssignableFrom(config.valueSourceType)) { if (ValuesSource.Numeric.class.isAssignableFrom(config.valueSourceType)) {
return (VS) numericField(fieldDataSources, config); return (VS) numericField(config);
} }
if (ValuesSource.GeoPoint.class.isAssignableFrom(config.valueSourceType)) { if (ValuesSource.GeoPoint.class.isAssignableFrom(config.valueSourceType)) {
return (VS) geoPointField(fieldDataSources, config); return (VS) geoPointField(config);
} }
// falling back to bytes values // falling back to bytes values
return (VS) bytesField(fieldDataSources, config); return (VS) bytesField(config);
} }
private ValuesSource.Numeric numericScript(ValuesSourceConfig<?> config) throws IOException { private ValuesSource.Numeric numericScript(ValuesSourceConfig<?> config) throws IOException {
setScorerIfNeeded(config.script); return new ValuesSource.Numeric.Script(config.script, config.scriptValueType);
setReaderIfNeeded(config.script);
scorerAwares.add(config.script);
readerAwares.add(config.script);
ValuesSource.Numeric source = new ValuesSource.Numeric.Script(config.script, config.scriptValueType);
return source;
} }
private ValuesSource.Numeric numericField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) throws IOException { private ValuesSource.Numeric numericField(ValuesSourceConfig<?> config) throws IOException {
final ConfigCacheKey cacheKey = new ConfigCacheKey(config); ValuesSource.Numeric dataSource = new ValuesSource.Numeric.FieldData((IndexNumericFieldData) config.fieldContext.indexFieldData());
ValuesSource.Numeric dataSource = (ValuesSource.Numeric) fieldDataSources.get(cacheKey);
if (dataSource == null) {
ValuesSource.MetaData metaData = ValuesSource.MetaData.load(config.fieldContext.indexFieldData(), searchContext);
dataSource = new ValuesSource.Numeric.FieldData((IndexNumericFieldData) config.fieldContext.indexFieldData(), metaData);
setReaderIfNeeded((ReaderContextAware) dataSource);
readerAwares.add((ReaderContextAware) dataSource);
fieldDataSources.put(cacheKey, dataSource);
}
if (config.script != null) { if (config.script != null) {
setScorerIfNeeded(config.script);
setReaderIfNeeded(config.script);
scorerAwares.add(config.script);
readerAwares.add(config.script);
dataSource = new ValuesSource.Numeric.WithScript(dataSource, config.script); dataSource = new ValuesSource.Numeric.WithScript(dataSource, config.script);
} }
return dataSource; return dataSource;
} }
private ValuesSource bytesField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) throws IOException { private ValuesSource bytesField(ValuesSourceConfig<?> config) throws IOException {
final ConfigCacheKey cacheKey = new ConfigCacheKey(config); final IndexFieldData<?> indexFieldData = config.fieldContext.indexFieldData();
ValuesSource dataSource = fieldDataSources.get(cacheKey); ValuesSource dataSource;
if (dataSource == null) { if (indexFieldData instanceof ParentChildIndexFieldData) {
final IndexFieldData<?> indexFieldData = config.fieldContext.indexFieldData(); dataSource = new ValuesSource.Bytes.WithOrdinals.ParentChild((ParentChildIndexFieldData) indexFieldData);
ValuesSource.MetaData metaData = ValuesSource.MetaData.load(config.fieldContext.indexFieldData(), searchContext); } else if (indexFieldData instanceof IndexOrdinalsFieldData) {
if (indexFieldData instanceof ParentChildIndexFieldData) { dataSource = new ValuesSource.Bytes.WithOrdinals.FieldData((IndexOrdinalsFieldData) indexFieldData);
dataSource = new ValuesSource.Bytes.WithOrdinals.ParentChild((ParentChildIndexFieldData) indexFieldData, metaData); } else {
} else if (indexFieldData instanceof IndexOrdinalsFieldData) { dataSource = new ValuesSource.Bytes.FieldData(indexFieldData);
dataSource = new ValuesSource.Bytes.WithOrdinals.FieldData((IndexOrdinalsFieldData) indexFieldData, metaData);
} else {
dataSource = new ValuesSource.Bytes.FieldData(indexFieldData, metaData);
}
setReaderIfNeeded((ReaderContextAware) dataSource);
readerAwares.add((ReaderContextAware) dataSource);
if (dataSource instanceof TopReaderContextAware) {
topReaderAwares.add((TopReaderContextAware) dataSource);
}
fieldDataSources.put(cacheKey, dataSource);
} }
if (config.script != null) { if (config.script != null) {
setScorerIfNeeded(config.script);
setReaderIfNeeded(config.script);
scorerAwares.add(config.script);
readerAwares.add(config.script);
dataSource = new ValuesSource.WithScript(dataSource, config.script); dataSource = new ValuesSource.WithScript(dataSource, config.script);
} }
return dataSource; return dataSource;
} }
private ValuesSource.Bytes bytesScript(ValuesSourceConfig<?> config) throws IOException { private ValuesSource.Bytes bytesScript(ValuesSourceConfig<?> config) throws IOException {
setScorerIfNeeded(config.script); return new ValuesSource.Bytes.Script(config.script);
setReaderIfNeeded(config.script);
scorerAwares.add(config.script);
readerAwares.add(config.script);
ValuesSource.Bytes source = new ValuesSource.Bytes.Script(config.script);
return source;
} }
private ValuesSource.GeoPoint geoPointField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) throws IOException { private ValuesSource.GeoPoint geoPointField(ValuesSourceConfig<?> config) throws IOException {
final ConfigCacheKey cacheKey = new ConfigCacheKey(config); return new ValuesSource.GeoPoint((IndexGeoPointFieldData) config.fieldContext.indexFieldData());
ValuesSource.GeoPoint dataSource = (ValuesSource.GeoPoint) fieldDataSources.get(cacheKey);
if (dataSource == null) {
ValuesSource.MetaData metaData = ValuesSource.MetaData.load(config.fieldContext.indexFieldData(), searchContext);
dataSource = new ValuesSource.GeoPoint((IndexGeoPointFieldData) config.fieldContext.indexFieldData(), metaData);
setReaderIfNeeded(dataSource);
readerAwares.add(dataSource);
fieldDataSources.put(cacheKey, dataSource);
}
return dataSource;
} }
public void registerReaderContextAware(ReaderContextAware readerContextAware) throws IOException {
setReaderIfNeeded(readerContextAware);
readerAwares.add(readerContextAware);
}
public void registerScorerAware(ScorerAware scorerAware) {
setScorerIfNeeded(scorerAware);
scorerAwares.add(scorerAware);
}
private void setReaderIfNeeded(ReaderContextAware readerAware) throws IOException {
if (reader != null) {
readerAware.setNextReader(reader);
}
}
private void setScorerIfNeeded(ScorerAware scorerAware) {
if (scorer != null) {
scorerAware.setScorer(scorer);
}
}
private static class ConfigCacheKey {
private final String field;
private final Class<? extends ValuesSource> valueSourceType;
private ConfigCacheKey(ValuesSourceConfig config) {
this.field = config.fieldContext.field();
this.valueSourceType = config.valueSourceType;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ConfigCacheKey that = (ConfigCacheKey) o;
if (!field.equals(that.field)) return false;
if (!valueSourceType.equals(that.valueSourceType)) return false;
return true;
}
@Override
public int hashCode() {
int result = field.hashCode();
result = 31 * result + valueSourceType.hashCode();
return result;
}
}
} }

View File

@ -18,367 +18,169 @@
*/ */
package org.elasticsearch.search.aggregations.support; package org.elasticsearch.search.aggregations.support;
import org.apache.lucene.index.*; import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lucene.ReaderContextAware; import org.elasticsearch.common.lucene.ScorerAware;
import org.elasticsearch.common.lucene.TopReaderContextAware; import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.*; import org.elasticsearch.index.fielddata.AtomicParentChildFieldData;
import org.elasticsearch.index.fielddata.plain.ParentChildAtomicFieldData; import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
import org.elasticsearch.index.fielddata.IndexParentChildFieldData;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.SortingBinaryDocValues;
import org.elasticsearch.index.fielddata.SortingNumericDocValues;
import org.elasticsearch.index.fielddata.SortingNumericDoubleValues;
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData; import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
import org.elasticsearch.script.SearchScript; import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric.WithScript.DoubleValues;
import org.elasticsearch.search.aggregations.support.ValuesSource.WithScript.BytesValues; import org.elasticsearch.search.aggregations.support.ValuesSource.WithScript.BytesValues;
import org.elasticsearch.search.aggregations.support.values.ScriptBytesValues; import org.elasticsearch.search.aggregations.support.values.ScriptBytesValues;
import org.elasticsearch.search.aggregations.support.values.ScriptDoubleValues; import org.elasticsearch.search.aggregations.support.values.ScriptDoubleValues;
import org.elasticsearch.search.aggregations.support.values.ScriptLongValues; import org.elasticsearch.search.aggregations.support.values.ScriptLongValues;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
public abstract class ValuesSource { public abstract class ValuesSource {
public static class MetaData {
public static final MetaData UNKNOWN = new MetaData();
public enum Uniqueness {
UNIQUE,
NOT_UNIQUE,
UNKNOWN;
public boolean unique() {
return this == UNIQUE;
}
}
private long maxAtomicUniqueValuesCount = -1;
private boolean multiValued = true;
private Uniqueness uniqueness = Uniqueness.UNKNOWN;
private MetaData() {}
private MetaData(MetaData other) {
this.maxAtomicUniqueValuesCount = other.maxAtomicUniqueValuesCount;
this.multiValued = other.multiValued;
this.uniqueness = other.uniqueness;
}
private MetaData(long maxAtomicUniqueValuesCount, boolean multiValued, Uniqueness uniqueness) {
this.maxAtomicUniqueValuesCount = maxAtomicUniqueValuesCount;
this.multiValued = multiValued;
this.uniqueness = uniqueness;
}
public long maxAtomicUniqueValuesCount() {
return maxAtomicUniqueValuesCount;
}
public boolean multiValued() {
return multiValued;
}
public Uniqueness uniqueness() {
return uniqueness;
}
public static MetaData load(IndexFieldData<?> indexFieldData, SearchContext context) {
MetaData metaData = new MetaData();
metaData.uniqueness = Uniqueness.UNIQUE;
for (LeafReaderContext readerContext : context.searcher().getTopReaderContext().leaves()) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
if (fieldData instanceof AtomicOrdinalsFieldData) {
AtomicOrdinalsFieldData fd = (AtomicOrdinalsFieldData) fieldData;
RandomAccessOrds values = fd.getOrdinalsValues();
metaData.multiValued |= FieldData.isMultiValued(values);
metaData.maxAtomicUniqueValuesCount = Math.max(metaData.maxAtomicUniqueValuesCount, values.getValueCount());
} else if (fieldData instanceof AtomicNumericFieldData) {
AtomicNumericFieldData fd = (AtomicNumericFieldData) fieldData;
SortedNumericDoubleValues values = fd.getDoubleValues();
metaData.multiValued |= FieldData.isMultiValued(values);
metaData.maxAtomicUniqueValuesCount = Long.MAX_VALUE;
} else if (fieldData instanceof AtomicGeoPointFieldData) {
AtomicGeoPointFieldData fd = (AtomicGeoPointFieldData) fieldData;
MultiGeoPointValues values = fd.getGeoPointValues();
metaData.multiValued |= FieldData.isMultiValued(values);
metaData.maxAtomicUniqueValuesCount = Long.MAX_VALUE;
} else {
metaData.multiValued = true;
metaData.maxAtomicUniqueValuesCount = Long.MAX_VALUE;
}
}
return metaData;
}
public static Builder builder() {
return new Builder();
}
public static Builder builder(MetaData other) {
return new Builder(other);
}
public static class Builder {
private final MetaData metaData;
private Builder() {
metaData = new MetaData();
}
private Builder(MetaData metaData) {
this.metaData = new MetaData(metaData);
}
public Builder maxAtomicUniqueValuesCount(long maxAtomicUniqueValuesCount) {
metaData.maxAtomicUniqueValuesCount = maxAtomicUniqueValuesCount;
return this;
}
public Builder multiValued(boolean multiValued) {
metaData.multiValued = multiValued;
return this;
}
public Builder uniqueness(Uniqueness uniqueness) {
metaData.uniqueness = uniqueness;
return this;
}
public MetaData build() {
return metaData;
}
}
}
/** /**
* Get the current {@link BytesValues}. * Get the current {@link BytesValues}.
*/ */
public abstract SortedBinaryDocValues bytesValues(); public abstract SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException;
public abstract Bits docsWithValue(int maxDoc); public abstract Bits docsWithValue(LeafReaderContext context) throws IOException;
public void setNeedsGlobalOrdinals(boolean needsGlobalOrdinals) {} /** Whether this values source needs scores. */
public boolean needsScores() {
public abstract MetaData metaData(); return false;
}
public static abstract class Bytes extends ValuesSource { public static abstract class Bytes extends ValuesSource {
public Bits docsWithValue(int maxDoc) { public Bits docsWithValue(LeafReaderContext context) throws IOException {
final SortedBinaryDocValues bytes = bytesValues(); final SortedBinaryDocValues bytes = bytesValues(context);
if (org.elasticsearch.index.fielddata.FieldData.unwrapSingleton(bytes) != null) { if (org.elasticsearch.index.fielddata.FieldData.unwrapSingleton(bytes) != null) {
return org.elasticsearch.index.fielddata.FieldData.unwrapSingletonBits(bytes); return org.elasticsearch.index.fielddata.FieldData.unwrapSingletonBits(bytes);
} else { } else {
return org.elasticsearch.index.fielddata.FieldData.docsWithValue(bytes, maxDoc); return org.elasticsearch.index.fielddata.FieldData.docsWithValue(bytes, context.reader().maxDoc());
} }
} }
public static abstract class WithOrdinals extends Bytes implements TopReaderContextAware { public static abstract class WithOrdinals extends Bytes {
public Bits docsWithValue(int maxDoc) { public Bits docsWithValue(LeafReaderContext context) {
final RandomAccessOrds ordinals = ordinalsValues(); final RandomAccessOrds ordinals = ordinalsValues(context);
if (DocValues.unwrapSingleton(ordinals) != null) { if (DocValues.unwrapSingleton(ordinals) != null) {
return DocValues.docsWithValue(DocValues.unwrapSingleton(ordinals), maxDoc); return DocValues.docsWithValue(DocValues.unwrapSingleton(ordinals), context.reader().maxDoc());
} else { } else {
return DocValues.docsWithValue(ordinals, maxDoc); return DocValues.docsWithValue(ordinals, context.reader().maxDoc());
} }
} }
public abstract RandomAccessOrds ordinalsValues(); public abstract RandomAccessOrds ordinalsValues(LeafReaderContext context);
public abstract void setNextReader(IndexReaderContext reader); public abstract RandomAccessOrds globalOrdinalsValues(LeafReaderContext context);
public abstract RandomAccessOrds globalOrdinalsValues();
public abstract long globalMaxOrd(IndexSearcher indexSearcher); public abstract long globalMaxOrd(IndexSearcher indexSearcher);
public static class FieldData extends WithOrdinals implements ReaderContextAware { public static class FieldData extends WithOrdinals {
protected final IndexOrdinalsFieldData indexFieldData; protected final IndexOrdinalsFieldData indexFieldData;
protected final MetaData metaData;
private boolean needsGlobalOrdinals;
protected AtomicOrdinalsFieldData atomicFieldData; public FieldData(IndexOrdinalsFieldData indexFieldData) {
private SortedBinaryDocValues bytesValues;
private RandomAccessOrds ordinalsValues;
protected IndexOrdinalsFieldData globalFieldData;
protected AtomicOrdinalsFieldData globalAtomicFieldData;
private RandomAccessOrds globalBytesValues;
private long maxOrd = -1;
public FieldData(IndexOrdinalsFieldData indexFieldData, MetaData metaData) {
this.indexFieldData = indexFieldData; this.indexFieldData = indexFieldData;
this.metaData = metaData;
} }
@Override @Override
public MetaData metaData() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) {
return metaData; final AtomicOrdinalsFieldData atomicFieldData = indexFieldData.load(context);
return atomicFieldData.getBytesValues();
} }
@Override @Override
public void setNeedsGlobalOrdinals(boolean needsGlobalOrdinals) { public RandomAccessOrds ordinalsValues(LeafReaderContext context) {
this.needsGlobalOrdinals = needsGlobalOrdinals; final AtomicOrdinalsFieldData atomicFieldData = indexFieldData.load(context);
return atomicFieldData.getOrdinalsValues();
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public RandomAccessOrds globalOrdinalsValues(LeafReaderContext context) {
atomicFieldData = indexFieldData.load(reader); final IndexOrdinalsFieldData global = indexFieldData.loadGlobal(context.parent.reader());
if (bytesValues != null) { final AtomicOrdinalsFieldData atomicFieldData = global.load(context);
bytesValues = atomicFieldData.getBytesValues(); return atomicFieldData.getOrdinalsValues();
}
if (ordinalsValues != null) {
ordinalsValues = atomicFieldData.getOrdinalsValues();
}
if (globalFieldData != null) {
globalAtomicFieldData = globalFieldData.load(reader);
if (globalBytesValues != null) {
globalBytesValues = globalAtomicFieldData.getOrdinalsValues();
}
}
}
@Override
public SortedBinaryDocValues bytesValues() {
if (bytesValues == null) {
bytesValues = atomicFieldData.getBytesValues();
}
return bytesValues;
}
@Override
public RandomAccessOrds ordinalsValues() {
if (ordinalsValues == null) {
ordinalsValues = atomicFieldData.getOrdinalsValues();
}
return ordinalsValues;
}
@Override
public void setNextReader(IndexReaderContext reader) {
if (needsGlobalOrdinals) {
globalFieldData = indexFieldData.loadGlobal(reader.reader());
}
}
@Override
public RandomAccessOrds globalOrdinalsValues() {
if (globalBytesValues == null) {
globalBytesValues = globalAtomicFieldData.getOrdinalsValues();
}
return globalBytesValues;
} }
@Override @Override
public long globalMaxOrd(IndexSearcher indexSearcher) { public long globalMaxOrd(IndexSearcher indexSearcher) {
if (maxOrd != -1) {
return maxOrd;
}
IndexReader indexReader = indexSearcher.getIndexReader(); IndexReader indexReader = indexSearcher.getIndexReader();
if (indexReader.leaves().isEmpty()) { if (indexReader.leaves().isEmpty()) {
return maxOrd = 0; return 0;
} else { } else {
LeafReaderContext atomicReaderContext = indexReader.leaves().get(0); LeafReaderContext atomicReaderContext = indexReader.leaves().get(0);
IndexOrdinalsFieldData globalFieldData = indexFieldData.loadGlobal(indexReader); IndexOrdinalsFieldData globalFieldData = indexFieldData.loadGlobal(indexReader);
AtomicOrdinalsFieldData afd = globalFieldData.load(atomicReaderContext); AtomicOrdinalsFieldData afd = globalFieldData.load(atomicReaderContext);
RandomAccessOrds values = afd.getOrdinalsValues(); RandomAccessOrds values = afd.getOrdinalsValues();
return maxOrd = values.getValueCount(); return values.getValueCount();
} }
} }
} }
} }
public static class ParentChild extends Bytes implements ReaderContextAware, TopReaderContextAware { public static class ParentChild extends Bytes {
protected final ParentChildIndexFieldData indexFieldData; protected final ParentChildIndexFieldData indexFieldData;
protected final MetaData metaData;
protected AtomicParentChildFieldData atomicFieldData; public ParentChild(ParentChildIndexFieldData indexFieldData) {
protected IndexParentChildFieldData globalFieldData;
private long maxOrd = -1;
public ParentChild(ParentChildIndexFieldData indexFieldData, MetaData metaData) {
this.indexFieldData = indexFieldData; this.indexFieldData = indexFieldData;
this.metaData = metaData;
}
@Override
public void setNextReader(LeafReaderContext reader) {
atomicFieldData = globalFieldData.load(reader);
}
@Override
public void setNextReader(IndexReaderContext reader) {
globalFieldData = indexFieldData.loadGlobal(reader.reader());
}
public SortedDocValues globalOrdinalsValues(String type) {
return atomicFieldData.getOrdinalsValues(type);
} }
public long globalMaxOrd(IndexSearcher indexSearcher, String type) { public long globalMaxOrd(IndexSearcher indexSearcher, String type) {
if (maxOrd != -1) {
return maxOrd;
}
IndexReader indexReader = indexSearcher.getIndexReader(); IndexReader indexReader = indexSearcher.getIndexReader();
if (indexReader.leaves().isEmpty()) { if (indexReader.leaves().isEmpty()) {
return maxOrd = 0; return 0;
} else { } else {
LeafReaderContext atomicReaderContext = indexReader.leaves().get(0); LeafReaderContext atomicReaderContext = indexReader.leaves().get(0);
IndexParentChildFieldData globalFieldData = indexFieldData.loadGlobal(indexReader); IndexParentChildFieldData globalFieldData = indexFieldData.loadGlobal(indexReader);
AtomicParentChildFieldData afd = globalFieldData.load(atomicReaderContext); AtomicParentChildFieldData afd = globalFieldData.load(atomicReaderContext);
SortedDocValues values = afd.getOrdinalsValues(type); SortedDocValues values = afd.getOrdinalsValues(type);
return maxOrd = values.getValueCount(); return values.getValueCount();
} }
} }
@Override public SortedDocValues globalOrdinalsValues(String type, LeafReaderContext context) {
public SortedBinaryDocValues bytesValues() { final IndexParentChildFieldData global = indexFieldData.loadGlobal(context.parent.reader());
return atomicFieldData.getBytesValues(); final AtomicParentChildFieldData atomicFieldData = global.load(context);
return atomicFieldData.getOrdinalsValues(type);
} }
@Override @Override
public MetaData metaData() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) {
return metaData; final AtomicParentChildFieldData atomicFieldData = indexFieldData.load(context);
return atomicFieldData.getBytesValues();
} }
} }
public static class FieldData extends Bytes implements ReaderContextAware { public static class FieldData extends Bytes {
protected final IndexFieldData<?> indexFieldData; protected final IndexFieldData<?> indexFieldData;
protected final MetaData metaData;
protected AtomicFieldData atomicFieldData;
private SortedBinaryDocValues bytesValues;
public FieldData(IndexFieldData<?> indexFieldData, MetaData metaData) { public FieldData(IndexFieldData<?> indexFieldData) {
this.indexFieldData = indexFieldData; this.indexFieldData = indexFieldData;
this.metaData = metaData;
} }
@Override @Override
public MetaData metaData() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) {
return metaData; return indexFieldData.load(context).getBytesValues();
}
@Override
public void setNextReader(LeafReaderContext reader) {
atomicFieldData = indexFieldData.load(reader);
if (bytesValues != null) {
bytesValues = atomicFieldData.getBytesValues();
}
}
@Override
public SortedBinaryDocValues bytesValues() {
if (bytesValues == null) {
bytesValues = atomicFieldData.getBytesValues();
}
return bytesValues;
} }
} }
@ -391,13 +193,15 @@ public abstract class ValuesSource {
} }
@Override @Override
public MetaData metaData() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException {
return MetaData.UNKNOWN; values.script().setNextReader(context);
return values;
} }
@Override @Override
public SortedBinaryDocValues bytesValues() { public boolean needsScores() {
return values; // TODO: add a way to know whether scripts are using scores
return true;
} }
} }
@ -409,40 +213,39 @@ public abstract class ValuesSource {
/** Whether the underlying data is floating-point or not. */ /** Whether the underlying data is floating-point or not. */
public abstract boolean isFloatingPoint(); public abstract boolean isFloatingPoint();
/** Get the current {@link LongValues}. */ /** Get the current {@link SortedNumericDocValues}. */
public abstract SortedNumericDocValues longValues(); public abstract SortedNumericDocValues longValues(LeafReaderContext context) throws IOException;
/** Get the current {@link DoubleValues}. */ /** Get the current {@link SortedNumericDoubleValues}. */
public abstract SortedNumericDoubleValues doubleValues(); public abstract SortedNumericDoubleValues doubleValues(LeafReaderContext context) throws IOException;
public Bits docsWithValue(int maxDoc) { @Override
public Bits docsWithValue(LeafReaderContext context) throws IOException {
if (isFloatingPoint()) { if (isFloatingPoint()) {
final SortedNumericDoubleValues values = doubleValues(); final SortedNumericDoubleValues values = doubleValues(context);
if (org.elasticsearch.index.fielddata.FieldData.unwrapSingleton(values) != null) { if (org.elasticsearch.index.fielddata.FieldData.unwrapSingleton(values) != null) {
return org.elasticsearch.index.fielddata.FieldData.unwrapSingletonBits(values); return org.elasticsearch.index.fielddata.FieldData.unwrapSingletonBits(values);
} else { } else {
return org.elasticsearch.index.fielddata.FieldData.docsWithValue(values, maxDoc); return org.elasticsearch.index.fielddata.FieldData.docsWithValue(values, context.reader().maxDoc());
} }
} else { } else {
final SortedNumericDocValues values = longValues(); final SortedNumericDocValues values = longValues(context);
if (DocValues.unwrapSingleton(values) != null) { if (DocValues.unwrapSingleton(values) != null) {
return DocValues.unwrapSingletonBits(values); return DocValues.unwrapSingletonBits(values);
} else { } else {
return DocValues.docsWithValue(values, maxDoc); return DocValues.docsWithValue(values, context.reader().maxDoc());
} }
} }
} }
public static class WithScript extends Numeric { public static class WithScript extends Numeric {
private final SortedNumericDocValues longValues; private final Numeric delegate;
private final SortedNumericDoubleValues doubleValues; private final SearchScript script;
private final SortedBinaryDocValues bytesValues;
public WithScript(Numeric delegate, SearchScript script) { public WithScript(Numeric delegate, SearchScript script) {
this.longValues = new LongValues(delegate, script); this.delegate = delegate;
this.doubleValues = new DoubleValues(delegate, script); this.script = script;
this.bytesValues = new ValuesSource.WithScript.BytesValues(delegate, script);
} }
@Override @Override
@ -451,92 +254,92 @@ public abstract class ValuesSource {
} }
@Override @Override
public SortedBinaryDocValues bytesValues() { public boolean needsScores() {
return bytesValues; // TODO: add a way to know whether scripts are using scores
return true;
} }
@Override @Override
public SortedNumericDocValues longValues() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException {
return longValues; script.setNextReader(context);
return new ValuesSource.WithScript.BytesValues(delegate.bytesValues(context), script);
} }
@Override @Override
public SortedNumericDoubleValues doubleValues() { public SortedNumericDocValues longValues(LeafReaderContext context) throws IOException {
return doubleValues; script.setNextReader(context);
return new LongValues(delegate.longValues(context), script);
} }
@Override @Override
public MetaData metaData() { public SortedNumericDoubleValues doubleValues(LeafReaderContext context) throws IOException {
return MetaData.UNKNOWN; script.setNextReader(context);
return new DoubleValues(delegate.doubleValues(context), script);
} }
static class LongValues extends SortingNumericDocValues { static class LongValues extends SortingNumericDocValues implements ScorerAware {
private final Numeric source; private final SortedNumericDocValues longValues;
private final SearchScript script; private final SearchScript script;
public LongValues(Numeric source, SearchScript script) { public LongValues(SortedNumericDocValues values, SearchScript script) {
this.source = source; this.longValues = values;
this.script = script; this.script = script;
} }
@Override @Override
public void setDocument(int docId) { public void setDocument(int doc) {
script.setNextDocId(docId); longValues.setDocument(doc);
source.longValues().setDocument(docId); resize(longValues.count());
resize(source.longValues().count()); script.setNextDocId(doc);
for (int i = 0; i < count(); ++i) { for (int i = 0; i < count(); ++i) {
script.setNextVar("_value", source.longValues().valueAt(i)); script.setNextVar("_value", longValues.valueAt(i));
values[i] = script.runAsLong(); values[i] = script.runAsLong();
} }
sort(); sort();
} }
@Override
public void setScorer(Scorer scorer) {
script.setScorer(scorer);
}
} }
static class DoubleValues extends SortingNumericDoubleValues { static class DoubleValues extends SortingNumericDoubleValues implements ScorerAware {
private final Numeric source; private final SortedNumericDoubleValues doubleValues;
private final SearchScript script; private final SearchScript script;
public DoubleValues(Numeric source, SearchScript script) { public DoubleValues(SortedNumericDoubleValues values, SearchScript script) {
this.source = source; this.doubleValues = values;
this.script = script; this.script = script;
} }
@Override @Override
public void setDocument(int docId) { public void setDocument(int doc) {
script.setNextDocId(docId); doubleValues.setDocument(doc);
source.doubleValues().setDocument(docId); resize(doubleValues.count());
count = source.doubleValues().count(); script.setNextDocId(doc);
grow(); for (int i = 0; i < count(); ++i) {
for (int i = 0; i < count; ++i) { script.setNextVar("_value", doubleValues.valueAt(i));
script.setNextVar("_value", source.doubleValues().valueAt(i));
values[i] = script.runAsDouble(); values[i] = script.runAsDouble();
} }
sort(); sort();
} }
@Override
public void setScorer(Scorer scorer) {
script.setScorer(scorer);
}
} }
} }
public static class FieldData extends Numeric implements ReaderContextAware { public static class FieldData extends Numeric {
protected boolean needsHashes;
protected final IndexNumericFieldData indexFieldData; protected final IndexNumericFieldData indexFieldData;
protected final MetaData metaData;
protected AtomicNumericFieldData atomicFieldData;
private SortedBinaryDocValues bytesValues;
private SortedNumericDocValues longValues;
private SortedNumericDoubleValues doubleValues;
public FieldData(IndexNumericFieldData indexFieldData, MetaData metaData) { public FieldData(IndexNumericFieldData indexFieldData) {
this.indexFieldData = indexFieldData; this.indexFieldData = indexFieldData;
this.metaData = metaData;
needsHashes = false;
}
@Override
public MetaData metaData() {
return metaData;
} }
@Override @Override
@ -545,61 +348,28 @@ public abstract class ValuesSource {
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public SortedBinaryDocValues bytesValues(LeafReaderContext context) {
atomicFieldData = indexFieldData.load(reader); return indexFieldData.load(context).getBytesValues();
if (bytesValues != null) {
bytesValues = atomicFieldData.getBytesValues();
}
if (longValues != null) {
longValues = atomicFieldData.getLongValues();
}
if (doubleValues != null) {
doubleValues = atomicFieldData.getDoubleValues();
}
} }
@Override @Override
public SortedBinaryDocValues bytesValues() { public SortedNumericDocValues longValues(LeafReaderContext context) {
if (bytesValues == null) { return indexFieldData.load(context).getLongValues();
bytesValues = atomicFieldData.getBytesValues();
}
return bytesValues;
} }
@Override @Override
public SortedNumericDocValues longValues() { public SortedNumericDoubleValues doubleValues(LeafReaderContext context) {
if (longValues == null) { return indexFieldData.load(context).getDoubleValues();
longValues = atomicFieldData.getLongValues();
}
return longValues;
}
@Override
public SortedNumericDoubleValues doubleValues() {
if (doubleValues == null) {
doubleValues = atomicFieldData.getDoubleValues();
}
return doubleValues;
} }
} }
public static class Script extends Numeric { public static class Script extends Numeric {
private final SearchScript script;
private final ValueType scriptValueType; private final ValueType scriptValueType;
private final ScriptDoubleValues doubleValues;
private final ScriptLongValues longValues;
private final ScriptBytesValues bytesValues;
public Script(SearchScript script, ValueType scriptValueType) { public Script(SearchScript script, ValueType scriptValueType) {
this.script = script;
this.scriptValueType = scriptValueType; this.scriptValueType = scriptValueType;
longValues = new ScriptLongValues(script);
doubleValues = new ScriptDoubleValues(script);
bytesValues = new ScriptBytesValues(script);
}
@Override
public MetaData metaData() {
return MetaData.UNKNOWN;
} }
@Override @Override
@ -608,20 +378,28 @@ public abstract class ValuesSource {
} }
@Override @Override
public SortedNumericDocValues longValues() { public SortedNumericDocValues longValues(LeafReaderContext context) throws IOException {
return longValues; script.setNextReader(context);
return new ScriptLongValues(script);
} }
@Override @Override
public SortedNumericDoubleValues doubleValues() { public SortedNumericDoubleValues doubleValues(LeafReaderContext context) throws IOException {
return doubleValues; script.setNextReader(context);
return new ScriptDoubleValues(script);
} }
@Override @Override
public SortedBinaryDocValues bytesValues() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException {
return bytesValues; script.setNextReader(context);
return new ScriptBytesValues(script);
} }
@Override
public boolean needsScores() {
// TODO: add a way to know whether scripts are using scores
return true;
}
} }
} }
@ -629,98 +407,81 @@ public abstract class ValuesSource {
// No need to implement ReaderContextAware here, the delegate already takes care of updating data structures // No need to implement ReaderContextAware here, the delegate already takes care of updating data structures
public static class WithScript extends Bytes { public static class WithScript extends Bytes {
private final SortedBinaryDocValues bytesValues; private final ValuesSource delegate;
private final SearchScript script;
public WithScript(ValuesSource delegate, SearchScript script) { public WithScript(ValuesSource delegate, SearchScript script) {
this.bytesValues = new BytesValues(delegate, script); this.delegate = delegate;
this.script = script;
} }
@Override @Override
public MetaData metaData() { public boolean needsScores() {
return MetaData.UNKNOWN; // TODO: add a way to know whether scripts are using scores
return true;
} }
@Override @Override
public SortedBinaryDocValues bytesValues() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException {
return bytesValues; script.setNextReader(context);
return new BytesValues(delegate.bytesValues(context), script);
} }
static class BytesValues extends SortingBinaryDocValues { static class BytesValues extends SortingBinaryDocValues implements ScorerAware {
private final ValuesSource source; private final SortedBinaryDocValues bytesValues;
private final SearchScript script; private final SearchScript script;
public BytesValues(ValuesSource source, SearchScript script) { public BytesValues(SortedBinaryDocValues bytesValues, SearchScript script) {
this.source = source; this.bytesValues = bytesValues;
this.script = script; this.script = script;
} }
@Override @Override
public void setDocument(int docId) { public void setDocument(int docId) {
source.bytesValues().setDocument(docId); bytesValues.setDocument(docId);
count = source.bytesValues().count(); count = bytesValues.count();
grow(); grow();
for (int i = 0; i < count; ++i) { for (int i = 0; i < count; ++i) {
final BytesRef value = source.bytesValues().valueAt(i); final BytesRef value = bytesValues.valueAt(i);
script.setNextVar("_value", value.utf8ToString()); script.setNextVar("_value", value.utf8ToString());
values[i].copyChars(script.run().toString()); values[i].copyChars(script.run().toString());
} }
sort(); sort();
} }
@Override
public void setScorer(Scorer scorer) {
script.setScorer(scorer);
}
} }
} }
public static class GeoPoint extends ValuesSource implements ReaderContextAware { public static class GeoPoint extends ValuesSource {
protected final IndexGeoPointFieldData indexFieldData; protected final IndexGeoPointFieldData indexFieldData;
private final MetaData metaData;
protected AtomicGeoPointFieldData atomicFieldData;
private SortedBinaryDocValues bytesValues;
private MultiGeoPointValues geoPointValues;
public GeoPoint(IndexGeoPointFieldData indexFieldData, MetaData metaData) { public GeoPoint(IndexGeoPointFieldData indexFieldData) {
this.indexFieldData = indexFieldData; this.indexFieldData = indexFieldData;
this.metaData = metaData;
} }
public Bits docsWithValue(int maxDoc) { @Override
final MultiGeoPointValues geoPoints = geoPointValues(); public Bits docsWithValue(LeafReaderContext context) {
final MultiGeoPointValues geoPoints = geoPointValues(context);
if (org.elasticsearch.index.fielddata.FieldData.unwrapSingleton(geoPoints) != null) { if (org.elasticsearch.index.fielddata.FieldData.unwrapSingleton(geoPoints) != null) {
return org.elasticsearch.index.fielddata.FieldData.unwrapSingletonBits(geoPoints); return org.elasticsearch.index.fielddata.FieldData.unwrapSingletonBits(geoPoints);
} else { } else {
return org.elasticsearch.index.fielddata.FieldData.docsWithValue(geoPoints, maxDoc); return org.elasticsearch.index.fielddata.FieldData.docsWithValue(geoPoints, context.reader().maxDoc());
} }
} }
@Override @Override
public MetaData metaData() { public SortedBinaryDocValues bytesValues(LeafReaderContext context) {
return metaData; return indexFieldData.load(context).getBytesValues();
} }
@Override public org.elasticsearch.index.fielddata.MultiGeoPointValues geoPointValues(LeafReaderContext context) {
public void setNextReader(LeafReaderContext reader) { return indexFieldData.load(context).getGeoPointValues();
atomicFieldData = indexFieldData.load(reader);
if (bytesValues != null) {
bytesValues = atomicFieldData.getBytesValues();
}
if (geoPointValues != null) {
geoPointValues = atomicFieldData.getGeoPointValues();
}
}
@Override
public SortedBinaryDocValues bytesValues() {
if (bytesValues == null) {
bytesValues = atomicFieldData.getBytesValues();
}
return bytesValues;
}
public org.elasticsearch.index.fielddata.MultiGeoPointValues geoPointValues() {
if (geoPointValues == null) {
geoPointValues = atomicFieldData.getGeoPointValues();
}
return geoPointValues;
} }
} }

View File

@ -53,7 +53,7 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
if (config.unmapped()) { if (config.unmapped()) {
return createUnmapped(context, parent, metaData); return createUnmapped(context, parent, metaData);
} }
VS vs = context.valuesSource(config, parent == null ? 0 : 1 + parent.depth()); VS vs = context.valuesSource(config);
return doCreateInternal(vs, context, parent, collectsFromSingleBucket, metaData); return doCreateInternal(vs, context, parent, collectsFromSingleBucket, metaData);
} }
@ -64,15 +64,6 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
} }
} }
@Override
public boolean needsScores() {
// TODO: we have no way to know whether scripts use the score so
// for now we assume that they do but in the future it would be
// nice to be able to know if they need scores so that the query
// would only produce scores if required.
return config.script != null || super.needsScores();
}
protected abstract Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException; protected abstract Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException;
protected abstract Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException; protected abstract Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException;

View File

@ -18,6 +18,8 @@
*/ */
package org.elasticsearch.search.aggregations.support.values; package org.elasticsearch.search.aggregations.support.values;
import org.apache.lucene.search.Scorer;
import org.elasticsearch.common.lucene.ScorerAware;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortingBinaryDocValues; import org.elasticsearch.index.fielddata.SortingBinaryDocValues;
import org.elasticsearch.script.SearchScript; import org.elasticsearch.script.SearchScript;
@ -29,7 +31,7 @@ import java.util.Collection;
/** /**
* {@link SortedBinaryDocValues} implementation that reads values from a script. * {@link SortedBinaryDocValues} implementation that reads values from a script.
*/ */
public class ScriptBytesValues extends SortingBinaryDocValues implements ScriptValues { public class ScriptBytesValues extends SortingBinaryDocValues implements ScriptValues, ScorerAware {
private final SearchScript script; private final SearchScript script;
@ -78,4 +80,9 @@ public class ScriptBytesValues extends SortingBinaryDocValues implements ScriptV
} }
sort(); sort();
} }
@Override
public void setScorer(Scorer scorer) {
script.setScorer(scorer);
}
} }

View File

@ -18,6 +18,8 @@
*/ */
package org.elasticsearch.search.aggregations.support.values; package org.elasticsearch.search.aggregations.support.values;
import org.apache.lucene.search.Scorer;
import org.elasticsearch.common.lucene.ScorerAware;
import org.elasticsearch.index.fielddata.SortingNumericDoubleValues; import org.elasticsearch.index.fielddata.SortingNumericDoubleValues;
import org.elasticsearch.script.SearchScript; import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationExecutionException;
@ -30,7 +32,7 @@ import java.util.Iterator;
/** /**
* {@link DoubleValues} implementation which is based on a script * {@link DoubleValues} implementation which is based on a script
*/ */
public class ScriptDoubleValues extends SortingNumericDoubleValues implements ScriptValues { public class ScriptDoubleValues extends SortingNumericDoubleValues implements ScriptValues, ScorerAware {
final SearchScript script; final SearchScript script;
@ -50,30 +52,28 @@ public class ScriptDoubleValues extends SortingNumericDoubleValues implements Sc
final Object value = script.run(); final Object value = script.run();
if (value == null) { if (value == null) {
count = 0; resize(0);
} }
else if (value instanceof Number) { else if (value instanceof Number) {
count = 1; resize(1);
values[0] = ((Number) value).doubleValue(); values[0] = ((Number) value).doubleValue();
} }
else if (value.getClass().isArray()) { else if (value.getClass().isArray()) {
count = Array.getLength(value); resize(Array.getLength(value));
grow(); for (int i = 0; i < count(); ++i) {
for (int i = 0; i < count; ++i) {
values[i] = ((Number) Array.get(value, i)).doubleValue(); values[i] = ((Number) Array.get(value, i)).doubleValue();
} }
} }
else if (value instanceof Collection) { else if (value instanceof Collection) {
count = ((Collection<?>) value).size(); resize(((Collection<?>) value).size());
grow();
int i = 0; int i = 0;
for (Iterator<?> it = ((Collection<?>) value).iterator(); it.hasNext(); ++i) { for (Iterator<?> it = ((Collection<?>) value).iterator(); it.hasNext(); ++i) {
values[i] = ((Number) it.next()).doubleValue(); values[i] = ((Number) it.next()).doubleValue();
} }
assert i == count; assert i == count();
} }
else { else {
@ -82,4 +82,9 @@ public class ScriptDoubleValues extends SortingNumericDoubleValues implements Sc
sort(); sort();
} }
@Override
public void setScorer(Scorer scorer) {
script.setScorer(scorer);
}
} }

View File

@ -18,7 +18,9 @@
*/ */
package org.elasticsearch.search.aggregations.support.values; package org.elasticsearch.search.aggregations.support.values;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.util.LongValues; import org.apache.lucene.util.LongValues;
import org.elasticsearch.common.lucene.ScorerAware;
import org.elasticsearch.index.fielddata.SortingNumericDocValues; import org.elasticsearch.index.fielddata.SortingNumericDocValues;
import org.elasticsearch.script.SearchScript; import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationExecutionException;
@ -31,7 +33,7 @@ import java.util.Iterator;
/** /**
* {@link LongValues} implementation which is based on a script * {@link LongValues} implementation which is based on a script
*/ */
public class ScriptLongValues extends SortingNumericDocValues implements ScriptValues { public class ScriptLongValues extends SortingNumericDocValues implements ScriptValues, ScorerAware {
final SearchScript script; final SearchScript script;
@ -81,4 +83,9 @@ public class ScriptLongValues extends SortingNumericDocValues implements ScriptV
sort(); sort();
} }
@Override
public void setScorer(Scorer scorer) {
script.setScorer(scorer);
}
} }

View File

@ -21,6 +21,7 @@ package org.elasticsearch.benchmark.search.aggregations;
import com.carrotsearch.hppc.ObjectOpenHashSet; import com.carrotsearch.hppc.ObjectOpenHashSet;
import com.carrotsearch.randomizedtesting.generators.RandomStrings; import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse; import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;

View File

@ -21,6 +21,7 @@ package org.elasticsearch.benchmark.search.aggregations;
import com.carrotsearch.hppc.ObjectOpenHashSet; import com.carrotsearch.hppc.ObjectOpenHashSet;
import com.carrotsearch.randomizedtesting.generators.RandomStrings; import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse; import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;

View File

@ -27,7 +27,9 @@ import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
public class DisabledFieldDataFormatTests extends ElasticsearchSingleNodeTest { public class DisabledFieldDataFormatTests extends ElasticsearchSingleNodeTest {

View File

@ -22,6 +22,8 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import java.io.IOException; import java.io.IOException;
@ -57,8 +59,12 @@ public class AggregationCollectorTests extends ElasticsearchSingleNodeTest {
AggregatorParsers parser = getInstanceFromNode(AggregatorParsers.class); AggregatorParsers parser = getInstanceFromNode(AggregatorParsers.class);
XContentParser aggParser = JsonXContent.jsonXContent.createParser(agg); XContentParser aggParser = JsonXContent.jsonXContent.createParser(agg);
aggParser.nextToken(); aggParser.nextToken();
final AggregatorFactories factories = parser.parseAggregators(aggParser, createSearchContext(index)); SearchContext searchContext = createSearchContext(index);
return factories.needsScores(); final AggregatorFactories factories = parser.parseAggregators(aggParser, searchContext);
AggregationContext aggregationContext = new AggregationContext(searchContext);
final Aggregator[] aggregators = factories.createTopLevelAggregators(aggregationContext);
assertEquals(1, aggregators.length);
return aggregators[0].needsScores();
} }
} }

View File

@ -269,7 +269,7 @@ public class LongTermsTests extends AbstractTermsTests {
assertThat(bucket.getDocCount(), equalTo(1l)); assertThat(bucket.getDocCount(), equalTo(1l));
} }
} }
@Test @Test
public void singleValueFieldWithFiltering() throws Exception { public void singleValueFieldWithFiltering() throws Exception {
long includes[] = { 1, 2, 3, 98 }; long includes[] = { 1, 2, 3, 98 };
@ -300,7 +300,7 @@ public class LongTermsTests extends AbstractTermsTests {
assertThat(bucket.getDocCount(), equalTo(1l)); assertThat(bucket.getDocCount(), equalTo(1l));
} }
} }
@Test @Test
public void singleValueField_WithMaxSize() throws Exception { public void singleValueField_WithMaxSize() throws Exception {
SearchResponse response = client().prepareSearch("idx").setTypes("high_card_type") SearchResponse response = client().prepareSearch("idx").setTypes("high_card_type")
@ -699,7 +699,7 @@ public class LongTermsTests extends AbstractTermsTests {
try { try {
SearchResponse response = client().prepareSearch("idx").setTypes("type") client().prepareSearch("idx").setTypes("type")
.addAggregation(terms("terms") .addAggregation(terms("terms")
.collectMode(randomFrom(SubAggCollectionMode.values())) .collectMode(randomFrom(SubAggCollectionMode.values()))
.script("doc['" + MULTI_VALUED_FIELD_NAME + "']") .script("doc['" + MULTI_VALUED_FIELD_NAME + "']")

View File

@ -33,7 +33,10 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test; import org.junit.Test;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.search.aggregations.AggregationBuilders.*; import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.core.IsNull.notNullValue; import static org.hamcrest.core.IsNull.notNullValue;

View File

@ -38,6 +38,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.TermsBuilder;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Set; import java.util.Set;
@ -194,13 +195,7 @@ public class SignificantTermsTests extends ElasticsearchIntegrationTest {
for (Bucket topTerm : topTerms) { for (Bucket topTerm : topTerms) {
terms.add(topTerm.getKeyAsString()); terms.add(topTerm.getKeyAsString());
} }
assertThat(terms, hasSize(6)); assertEquals(new HashSet<String>(Arrays.asList("jam", "council", "style", "paul", "of", "the")), terms);
assertThat(terms.contains("jam"), is(true));
assertThat(terms.contains("council"), is(true));
assertThat(terms.contains("style"), is(true));
assertThat(terms.contains("paul"), is(true));
assertThat(terms.contains("of"), is(true));
assertThat(terms.contains("the"), is(true));
response = client().prepareSearch("test") response = client().prepareSearch("test")
.setQuery(new TermQueryBuilder("_all", "weller")) .setQuery(new TermQueryBuilder("_all", "weller"))

View File

@ -558,7 +558,7 @@ public class TopHitsTests extends ElasticsearchIntegrationTest {
// under an aggregation with collect_mode set to breadth_first as this would // under an aggregation with collect_mode set to breadth_first as this would
// require the buffering of scores alongside each document ID and that is a // require the buffering of scores alongside each document ID and that is a
// a RAM cost we are not willing to pay // a RAM cost we are not willing to pay
assertThat(e.getMessage(), containsString("ElasticsearchParseException")); assertThat(e.getMessage(), containsString("ElasticsearchIllegalStateException"));
} }
} }

View File

@ -38,6 +38,7 @@ import org.elasticsearch.index.search.nested.NonNestedDocsFilter;
import org.elasticsearch.search.aggregations.AggregationPhase; import org.elasticsearch.search.aggregations.AggregationPhase;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.BucketCollector;
import org.elasticsearch.search.aggregations.SearchContextAggregations; import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
@ -123,7 +124,7 @@ public class NestedAggregatorTest extends ElasticsearchSingleNodeLuceneTestCase
AggregatorFactories factories = builder.build(); AggregatorFactories factories = builder.build();
searchContext.aggregations(new SearchContextAggregations(factories)); searchContext.aggregations(new SearchContextAggregations(factories));
Aggregator[] aggs = factories.createTopLevelAggregators(context); Aggregator[] aggs = factories.createTopLevelAggregators(context);
AggregationPhase.AggregationsCollector collector = new AggregationPhase.AggregationsCollector(Arrays.asList(aggs), context); BucketCollector collector = BucketCollector.wrap(Arrays.asList(aggs));
// A regular search always exclude nested docs, so we use NonNestedDocsFilter.INSTANCE here (otherwise MatchAllDocsQuery would be sufficient) // A regular search always exclude nested docs, so we use NonNestedDocsFilter.INSTANCE here (otherwise MatchAllDocsQuery would be sufficient)
// We exclude root doc with uid type#2, this will trigger the bug if we don't reset the root doc when we process a new segment, because // We exclude root doc with uid type#2, this will trigger the bug if we don't reset the root doc when we process a new segment, because
// root doc type#3 and root doc type#1 have the same segment docid // root doc type#3 and root doc type#1 have the same segment docid