mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-24 17:09:48 +00:00
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:
parent
d1deb6bd1e
commit
de41981373
@ -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) {
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
|
||||||
}
|
|
@ -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);
|
||||||
|
@ -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];
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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 {
|
||||||
|
@ -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();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -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 + "]");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
@ -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];
|
||||||
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
@ -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;
|
|
||||||
}
|
}
|
||||||
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
@ -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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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");
|
|
||||||
}
|
|
||||||
}
|
|
@ -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 {
|
|
||||||
}
|
|
||||||
}
|
|
@ -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.
|
||||||
|
@ -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);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -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;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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.
|
||||||
|
@ -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;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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) {
|
||||||
|
@ -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);
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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());
|
||||||
|
@ -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());
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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());
|
||||||
|
@ -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);
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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() +
|
||||||
|
@ -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);
|
||||||
|
@ -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)) {
|
||||||
|
@ -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);
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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) {
|
||||||
|
@ -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() {
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
@ -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 {
|
||||||
|
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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 + "']")
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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"))
|
||||||
|
@ -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"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
Loading…
x
Reference in New Issue
Block a user