- fixed terms aggs ordering by sub metric agg - introduced MetricsAggregator to enable access to metrics values without creating buckets
fixes: #4643
This commit is contained in:
parent
1584c73e4a
commit
01b18ad219
|
@ -106,6 +106,10 @@ public abstract class Aggregator implements Releasable {
|
|||
return parent;
|
||||
}
|
||||
|
||||
public Aggregator[] subAggregators() {
|
||||
return subAggregators;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The current aggregation context.
|
||||
*/
|
||||
|
|
|
@ -35,13 +35,13 @@ public interface Bucket {
|
|||
|
||||
Aggregations getAggregations();
|
||||
|
||||
static class Comparator<B extends Bucket> implements java.util.Comparator<B> {
|
||||
static class SubAggregationComparator<B extends Bucket> implements java.util.Comparator<B> {
|
||||
|
||||
private final String aggName;
|
||||
private final String valueName;
|
||||
private final boolean asc;
|
||||
|
||||
public Comparator(String expression, boolean asc) {
|
||||
public SubAggregationComparator(String expression, boolean asc) {
|
||||
this.asc = asc;
|
||||
int i = expression.indexOf('.');
|
||||
if (i < 0) {
|
||||
|
@ -53,7 +53,7 @@ public interface Bucket {
|
|||
}
|
||||
}
|
||||
|
||||
public Comparator(String aggName, String valueName, boolean asc) {
|
||||
public SubAggregationComparator(String aggName, String valueName, boolean asc) {
|
||||
this.aggName = aggName;
|
||||
this.valueName = valueName;
|
||||
this.asc = asc;
|
||||
|
|
|
@ -29,7 +29,9 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -38,10 +40,19 @@ public abstract class BucketsAggregator extends Aggregator {
|
|||
|
||||
private LongArray docCounts;
|
||||
|
||||
private final Aggregator[] collectableSugAggregators;
|
||||
|
||||
public BucketsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories,
|
||||
long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
|
||||
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent);
|
||||
docCounts = BigArrays.newLongArray(estimatedBucketsCount, context.pageCacheRecycler(), true);
|
||||
List<Aggregator> collectables = new ArrayList<Aggregator>(subAggregators.length);
|
||||
for (int i = 0; i < subAggregators.length; i++) {
|
||||
if (subAggregators[i].shouldCollect()) {
|
||||
collectables.add((subAggregators[i]));
|
||||
}
|
||||
}
|
||||
collectableSugAggregators = collectables.toArray(new Aggregator[collectables.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -50,8 +61,8 @@ public abstract class BucketsAggregator extends Aggregator {
|
|||
protected final void collectBucket(int doc, long bucketOrd) throws IOException {
|
||||
docCounts = BigArrays.grow(docCounts, bucketOrd + 1);
|
||||
docCounts.increment(bucketOrd, 1);
|
||||
for (int i = 0; i < subAggregators.length; i++) {
|
||||
subAggregators[i].collect(doc, bucketOrd);
|
||||
for (int i = 0; i < collectableSugAggregators.length; i++) {
|
||||
collectableSugAggregators[i].collect(doc, bucketOrd);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -59,8 +70,8 @@ public abstract class BucketsAggregator extends Aggregator {
|
|||
* 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 {
|
||||
for (int i = 0; i < subAggregators.length; i++) {
|
||||
subAggregators[i].collect(doc, bucketOrd);
|
||||
for (int i = 0; i < collectableSugAggregators.length; i++) {
|
||||
collectableSugAggregators[i].collect(doc, bucketOrd);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.Bucket;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
@ -70,11 +71,11 @@ class InternalOrder extends HistogramBase.Order {
|
|||
static final byte ID = 0;
|
||||
|
||||
Aggregation(String key, boolean asc) {
|
||||
super(ID, key, asc, new HistogramBase.Bucket.Comparator<HistogramBase.Bucket>(key, asc));
|
||||
super(ID, key, asc, new Bucket.SubAggregationComparator<HistogramBase.Bucket>(key, asc));
|
||||
}
|
||||
|
||||
Aggregation(String aggName, String valueName, boolean asc) {
|
||||
super(ID, key(aggName, valueName), asc, new HistogramBase.Bucket.Comparator<HistogramBase.Bucket>(aggName, valueName, asc));
|
||||
super(ID, key(aggName, valueName), asc, new Bucket.SubAggregationComparator<HistogramBase.Bucket>(aggName, valueName, asc));
|
||||
}
|
||||
|
||||
private static String key(String aggName, String valueName) {
|
||||
|
|
|
@ -147,7 +147,7 @@ public class DoubleTerms extends InternalTerms {
|
|||
|
||||
// TODO: would it be better to sort the backing array buffer of hppc map directly instead of using a PQ?
|
||||
final int size = Math.min(requiredSize, buckets.v().size());
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator());
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator(null));
|
||||
boolean[] states = buckets.v().allocated;
|
||||
Object[] internalBuckets = buckets.v().values;
|
||||
for (int i = 0; i < states.length; i++) {
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.elasticsearch.common.lease.Releasables;
|
|||
import org.elasticsearch.index.fielddata.DoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.LongHash;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
|
||||
|
@ -49,7 +48,7 @@ public class DoubleTermsAggregator extends BucketsAggregator {
|
|||
InternalOrder order, int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
this.order = order;
|
||||
this.order = InternalOrder.validate(order, this);
|
||||
this.requiredSize = requiredSize;
|
||||
this.shardSize = shardSize;
|
||||
bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.pageCacheRecycler());
|
||||
|
@ -77,24 +76,13 @@ public class DoubleTermsAggregator extends BucketsAggregator {
|
|||
}
|
||||
}
|
||||
|
||||
// private impl that stores a bucket ord. This allows for computing the aggregations lazily.
|
||||
static class OrdinalBucket extends DoubleTerms.Bucket {
|
||||
|
||||
long bucketOrd;
|
||||
|
||||
public OrdinalBucket() {
|
||||
super(0, 0, (InternalAggregations) null);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DoubleTerms buildAggregation(long owningBucketOrdinal) {
|
||||
assert owningBucketOrdinal == 0;
|
||||
final int size = (int) Math.min(bucketOrds.size(), shardSize);
|
||||
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator());
|
||||
OrdinalBucket spare = null;
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator(this));
|
||||
DoubleTerms.Bucket spare = null;
|
||||
for (long i = 0; i < bucketOrds.capacity(); ++i) {
|
||||
final long ord = bucketOrds.id(i);
|
||||
if (ord < 0) {
|
||||
|
@ -103,17 +91,17 @@ public class DoubleTermsAggregator extends BucketsAggregator {
|
|||
}
|
||||
|
||||
if (spare == null) {
|
||||
spare = new OrdinalBucket();
|
||||
spare = new DoubleTerms.Bucket(0, 0, null);
|
||||
}
|
||||
spare.term = Double.longBitsToDouble(bucketOrds.key(i));
|
||||
spare.docCount = bucketDocCount(ord);
|
||||
spare.bucketOrd = ord;
|
||||
spare = (OrdinalBucket) ordered.insertWithOverflow(spare);
|
||||
spare = (DoubleTerms.Bucket) ordered.insertWithOverflow(spare);
|
||||
}
|
||||
|
||||
final InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
|
||||
for (int i = ordered.size() - 1; i >= 0; --i) {
|
||||
final OrdinalBucket bucket = (OrdinalBucket) ordered.pop();
|
||||
final DoubleTerms.Bucket bucket = (DoubleTerms.Bucket) ordered.pop();
|
||||
bucket.aggregations = bucketAggregations(bucket.bucketOrd);
|
||||
list[i] = bucket;
|
||||
}
|
||||
|
|
|
@ -22,6 +22,10 @@ package org.elasticsearch.search.aggregations.bucket.terms;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.Bucket;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
@ -31,10 +35,61 @@ import java.util.Comparator;
|
|||
*/
|
||||
class InternalOrder extends Terms.Order {
|
||||
|
||||
/**
|
||||
* Order by the (higher) count of each term.
|
||||
*/
|
||||
public static final InternalOrder COUNT_DESC = new InternalOrder((byte) 1, "_count", false, new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
long i = o2.getDocCount() - o1.getDocCount();
|
||||
if (i == 0) {
|
||||
i = o2.compareTo(o1);
|
||||
if (i == 0) {
|
||||
i = System.identityHashCode(o2) - System.identityHashCode(o1);
|
||||
}
|
||||
}
|
||||
return i > 0 ? 1 : -1;
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the (lower) count of each term.
|
||||
*/
|
||||
public static final InternalOrder COUNT_ASC = new InternalOrder((byte) 2, "_count", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return -COUNT_DESC.comparator(null).compare(o1, o2);
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final InternalOrder TERM_DESC = new InternalOrder((byte) 3, "_term", false, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return o2.compareTo(o1);
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final InternalOrder TERM_ASC = new InternalOrder((byte) 4, "_term", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return -TERM_DESC.comparator(null).compare(o1, o2);
|
||||
}
|
||||
});
|
||||
|
||||
|
||||
final byte id;
|
||||
final String key;
|
||||
final boolean asc;
|
||||
final Comparator<Terms.Bucket> comparator;
|
||||
protected final Comparator<Terms.Bucket> comparator;
|
||||
|
||||
InternalOrder(byte id, String key, boolean asc, Comparator<Terms.Bucket> comparator) {
|
||||
this.id = id;
|
||||
|
@ -56,7 +111,7 @@ class InternalOrder extends Terms.Order {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Terms.Bucket> comparator() {
|
||||
protected Comparator<Terms.Bucket> comparator(Aggregator aggregator) {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
|
@ -65,22 +120,129 @@ class InternalOrder extends Terms.Order {
|
|||
return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
|
||||
}
|
||||
|
||||
public static InternalOrder validate(InternalOrder order, Aggregator termsAggregator) {
|
||||
if (!(order instanceof Aggregation)) {
|
||||
return order;
|
||||
}
|
||||
String aggName = ((Aggregation) order).aggName();
|
||||
Aggregator[] subAggregators = termsAggregator.subAggregators();
|
||||
for (int i = 0; i < subAggregators.length; i++) {
|
||||
Aggregator aggregator = subAggregators[i];
|
||||
if (aggregator.name().equals(aggName)) {
|
||||
|
||||
// we can only apply order on metrics sub-aggregators
|
||||
if (!(aggregator instanceof MetricsAggregator)) {
|
||||
throw new AggregationExecutionException("terms aggregation [" + termsAggregator.name() + "] is configured to order by sub-aggregation ["
|
||||
+ aggName + "] which is is not a metrics aggregation. Terms aggregation order can only refer to metrics aggregations");
|
||||
}
|
||||
|
||||
if (aggregator instanceof MetricsAggregator.MultiValue) {
|
||||
String valueName = ((Aggregation) order).metricName();
|
||||
if (valueName == null) {
|
||||
throw new AggregationExecutionException("terms aggregation [" + termsAggregator.name() + "] is configured with a sub-aggregation order ["
|
||||
+ aggName + "] which is a multi-valued aggregation, yet no metric name was specified");
|
||||
}
|
||||
if (!((MetricsAggregator.MultiValue) aggregator).hasMetric(valueName)) {
|
||||
throw new AggregationExecutionException("terms aggregation [" + termsAggregator.name() + "] is configured with a sub-aggregation order ["
|
||||
+ aggName + "] and value [" + valueName + "] yet the referred sub aggregator holds no metric that goes by this name");
|
||||
}
|
||||
return order;
|
||||
}
|
||||
|
||||
// aggregator must be of a single value type
|
||||
// todo we can also choose to be really strict and verify that the user didn't specify a value name and if so fail?
|
||||
return order;
|
||||
}
|
||||
}
|
||||
|
||||
throw new AggregationExecutionException("terms aggregation [" + termsAggregator.name() + "] is configured with a sub-aggregation order ["
|
||||
+ aggName + "] but no sub aggregation with this name is configured");
|
||||
}
|
||||
|
||||
static class Aggregation extends InternalOrder {
|
||||
|
||||
static final byte ID = 0;
|
||||
|
||||
Aggregation(String key, boolean asc) {
|
||||
super(ID, key, asc, new Terms.Bucket.Comparator<Terms.Bucket>(key, asc));
|
||||
super(ID, key, asc, new Bucket.SubAggregationComparator<Terms.Bucket>(key, asc));
|
||||
}
|
||||
|
||||
Aggregation(String aggName, String valueName, boolean asc) {
|
||||
super(ID, key(aggName, valueName), asc, new Terms.Bucket.Comparator<Terms.Bucket>(aggName, valueName, asc));
|
||||
Aggregation(String aggName, String metricName, boolean asc) {
|
||||
super(ID, key(aggName, metricName), asc, new Bucket.SubAggregationComparator<Terms.Bucket>(aggName, metricName, asc));
|
||||
}
|
||||
|
||||
String aggName() {
|
||||
int index = key.indexOf('.');
|
||||
return index < 0 ? key : key.substring(0, index);
|
||||
}
|
||||
|
||||
String metricName() {
|
||||
int index = key.indexOf('.');
|
||||
return index < 0 ? null : key.substring(index + 1, key.length());
|
||||
}
|
||||
|
||||
private static String key(String aggName, String valueName) {
|
||||
return (valueName == null) ? aggName : aggName + "." + valueName;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Terms.Bucket> comparator(Aggregator termsAggregator) {
|
||||
if (termsAggregator == null) {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
// Internal Optimization:
|
||||
//
|
||||
// in this phase, if the order is based on sub-aggregations, we need to use a different comparator
|
||||
// to avoid constructing buckets for ordering purposes (we can potentially have a lot of buckets and building
|
||||
// them will cause loads of redundant object constructions). The "special" comparators here will fetch the
|
||||
// sub aggregation values directly from the sub aggregators bypassing bucket creation. Note that the comparator
|
||||
// attached to the order will still be used in the reduce phase of the Aggregation.
|
||||
|
||||
final Aggregator aggregator = subAggregator(aggName(), termsAggregator);
|
||||
assert aggregator != null && aggregator instanceof MetricsAggregator : "this should be picked up before the aggregation is executed";
|
||||
if (aggregator instanceof MetricsAggregator.MultiValue) {
|
||||
final String valueName = metricName();
|
||||
assert valueName != null : "this should be picked up before the aggregation is executed";
|
||||
return new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
double v1 = ((MetricsAggregator.MultiValue) aggregator).metric(valueName, ((InternalTerms.Bucket) o1).bucketOrd);
|
||||
double v2 = ((MetricsAggregator.MultiValue) aggregator).metric(valueName, ((InternalTerms.Bucket) o2).bucketOrd);
|
||||
// some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
|
||||
// the bottom
|
||||
if (v1 == Double.NaN) {
|
||||
return asc ? 1 : -1;
|
||||
}
|
||||
return Double.compare(v1, v2);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
double v1 = ((MetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) o1).bucketOrd);
|
||||
double v2 = ((MetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) o2).bucketOrd);
|
||||
// some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
|
||||
// the bottom
|
||||
if (v1 == Double.NaN) {
|
||||
return asc ? 1 : -1;
|
||||
}
|
||||
return Double.compare(v1, v2);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Aggregator subAggregator(String aggName, Aggregator termsAggregator) {
|
||||
Aggregator[] subAggregators = termsAggregator.subAggregators();
|
||||
for (int i = 0; i < subAggregators.length; i++) {
|
||||
if (subAggregators[i].name().equals(aggName)) {
|
||||
return subAggregators[i];
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public static class Streams {
|
||||
|
@ -88,12 +250,12 @@ class InternalOrder extends Terms.Order {
|
|||
public static void writeOrder(InternalOrder order, StreamOutput out) throws IOException {
|
||||
out.writeByte(order.id());
|
||||
if (order instanceof Aggregation) {
|
||||
out.writeBoolean(((Terms.Bucket.Comparator) order.comparator).asc());
|
||||
out.writeString(((Terms.Bucket.Comparator) order.comparator).aggName());
|
||||
boolean hasValueName = ((Terms.Bucket.Comparator) order.comparator).aggName() != null;
|
||||
out.writeBoolean(((Bucket.SubAggregationComparator) order.comparator).asc());
|
||||
out.writeString(((Bucket.SubAggregationComparator) order.comparator).aggName());
|
||||
boolean hasValueName = ((Bucket.SubAggregationComparator) order.comparator).valueName() != null;
|
||||
out.writeBoolean(hasValueName);
|
||||
if (hasValueName) {
|
||||
out.writeString(((Terms.Bucket.Comparator) order.comparator).valueName());
|
||||
out.writeString(((Bucket.SubAggregationComparator) order.comparator).valueName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -101,16 +263,19 @@ class InternalOrder extends Terms.Order {
|
|||
public static InternalOrder readOrder(StreamInput in) throws IOException {
|
||||
byte id = in.readByte();
|
||||
switch (id) {
|
||||
case 1: return (InternalOrder) Terms.Order.COUNT_DESC;
|
||||
case 2: return (InternalOrder) Terms.Order.COUNT_ASC;
|
||||
case 3: return (InternalOrder) Terms.Order.TERM_DESC;
|
||||
case 4: return (InternalOrder) Terms.Order.TERM_ASC;
|
||||
case 1: return InternalOrder.COUNT_DESC;
|
||||
case 2: return InternalOrder.COUNT_ASC;
|
||||
case 3: return InternalOrder.TERM_DESC;
|
||||
case 4: return InternalOrder.TERM_ASC;
|
||||
case 0:
|
||||
boolean asc = in.readBoolean();
|
||||
String key = in.readString();
|
||||
if (in.readBoolean()) {
|
||||
return new InternalOrder.Aggregation(key, in.readString(), asc);
|
||||
}
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
default:
|
||||
throw new RuntimeException("unknown histogram order");
|
||||
throw new RuntimeException("unknown terms order");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,6 +38,8 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
|
|||
|
||||
public static abstract class Bucket implements Terms.Bucket {
|
||||
|
||||
long bucketOrd;
|
||||
|
||||
protected long docCount;
|
||||
protected InternalAggregations aggregations;
|
||||
|
||||
|
@ -166,7 +168,7 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
|
|||
}
|
||||
|
||||
final int size = Math.min(requiredSize, buckets.size());
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator());
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator(null));
|
||||
for (Map.Entry<Text, List<Bucket>> entry : buckets.entrySet()) {
|
||||
List<Bucket> sameTermBuckets = entry.getValue();
|
||||
ordered.insertWithOverflow(sameTermBuckets.get(0).reduce(sameTermBuckets, reduceContext.cacheRecycler()));
|
||||
|
|
|
@ -145,7 +145,7 @@ public class LongTerms extends InternalTerms {
|
|||
|
||||
// TODO: would it be better to sort the backing array buffer of the hppc map directly instead of using a PQ?
|
||||
final int size = Math.min(requiredSize, buckets.v().size());
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator());
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator(null));
|
||||
Object[] internalBuckets = buckets.v().values;
|
||||
boolean[] states = buckets.v().allocated;
|
||||
for (int i = 0; i < states.length; i++) {
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.elasticsearch.common.lease.Releasables;
|
|||
import org.elasticsearch.index.fielddata.LongValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.LongHash;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
|
||||
|
@ -49,7 +48,7 @@ public class LongTermsAggregator extends BucketsAggregator {
|
|||
InternalOrder order, int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
this.order = order;
|
||||
this.order = InternalOrder.validate(order, this);
|
||||
this.requiredSize = requiredSize;
|
||||
this.shardSize = shardSize;
|
||||
bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.pageCacheRecycler());
|
||||
|
@ -76,24 +75,13 @@ public class LongTermsAggregator extends BucketsAggregator {
|
|||
}
|
||||
}
|
||||
|
||||
// private impl that stores a bucket ord. This allows for computing the aggregations lazily.
|
||||
static class OrdinalBucket extends LongTerms.Bucket {
|
||||
|
||||
long bucketOrd;
|
||||
|
||||
public OrdinalBucket() {
|
||||
super(0, 0, (InternalAggregations) null);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public LongTerms buildAggregation(long owningBucketOrdinal) {
|
||||
assert owningBucketOrdinal == 0;
|
||||
final int size = (int) Math.min(bucketOrds.size(), shardSize);
|
||||
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator());
|
||||
OrdinalBucket spare = null;
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator(this));
|
||||
LongTerms.Bucket spare = null;
|
||||
for (long i = 0; i < bucketOrds.capacity(); ++i) {
|
||||
final long ord = bucketOrds.id(i);
|
||||
if (ord < 0) {
|
||||
|
@ -102,17 +90,17 @@ public class LongTermsAggregator extends BucketsAggregator {
|
|||
}
|
||||
|
||||
if (spare == null) {
|
||||
spare = new OrdinalBucket();
|
||||
spare = new LongTerms.Bucket(0, 0, null);
|
||||
}
|
||||
spare.term = bucketOrds.key(i);
|
||||
spare.docCount = bucketDocCount(ord);
|
||||
spare.bucketOrd = ord;
|
||||
spare = (OrdinalBucket) ordered.insertWithOverflow(spare);
|
||||
spare = (LongTerms.Bucket) ordered.insertWithOverflow(spare);
|
||||
}
|
||||
|
||||
final InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
|
||||
for (int i = ordered.size() - 1; i >= 0; --i) {
|
||||
final OrdinalBucket bucket = (OrdinalBucket) ordered.pop();
|
||||
final LongTerms.Bucket bucket = (LongTerms.Bucket) ordered.pop();
|
||||
bucket.aggregations = bucketAggregations(bucket.bucketOrd);
|
||||
list[i] = bucket;
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.bucket.terms;
|
||||
|
||||
import com.sun.corba.se.impl.naming.cosnaming.InterOperableNamingImpl;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
|
@ -27,12 +28,14 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.index.fielddata.BytesValues;
|
||||
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
|
||||
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.BytesRefHash;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.bytes.BytesValuesSource;
|
||||
|
@ -59,7 +62,7 @@ public class StringTermsAggregator extends BucketsAggregator {
|
|||
|
||||
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
this.order = order;
|
||||
this.order = InternalOrder.validate(order, this);
|
||||
this.requiredSize = requiredSize;
|
||||
this.shardSize = shardSize;
|
||||
this.includeExclude = includeExclude;
|
||||
|
@ -92,37 +95,26 @@ public class StringTermsAggregator extends BucketsAggregator {
|
|||
}
|
||||
}
|
||||
|
||||
// private impl that stores a bucket ord. This allows for computing the aggregations lazily.
|
||||
static class OrdinalBucket extends StringTerms.Bucket {
|
||||
|
||||
int bucketOrd;
|
||||
|
||||
public OrdinalBucket() {
|
||||
super(new BytesRef(), 0, null);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringTerms buildAggregation(long owningBucketOrdinal) {
|
||||
assert owningBucketOrdinal == 0;
|
||||
final int size = (int) Math.min(bucketOrds.size(), shardSize);
|
||||
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator());
|
||||
OrdinalBucket spare = null;
|
||||
for (int i = 0; i < bucketOrds.size(); ++i) {
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size, order.comparator(this));
|
||||
StringTerms.Bucket spare = null;
|
||||
for (int i = 0; i < bucketOrds.size(); i++) {
|
||||
if (spare == null) {
|
||||
spare = new OrdinalBucket();
|
||||
spare = new StringTerms.Bucket(new BytesRef(), 0, null);
|
||||
}
|
||||
bucketOrds.get(i, spare.termBytes);
|
||||
spare.docCount = bucketDocCount(i);
|
||||
spare.bucketOrd = i;
|
||||
spare = (OrdinalBucket) ordered.insertWithOverflow(spare);
|
||||
spare = (StringTerms.Bucket) ordered.insertWithOverflow(spare);
|
||||
}
|
||||
|
||||
final InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
|
||||
for (int i = ordered.size() - 1; i >= 0; --i) {
|
||||
final OrdinalBucket bucket = (OrdinalBucket) ordered.pop();
|
||||
final StringTerms.Bucket bucket = (StringTerms.Bucket) ordered.pop();
|
||||
bucket.aggregations = bucketAggregations(bucket.bucketOrd);
|
||||
list[i] = bucket;
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.search.aggregations.bucket.terms;
|
|||
import org.elasticsearch.common.text.Text;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.search.aggregations.Aggregation;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.support.ScriptValueType;
|
||||
|
||||
import java.util.Collection;
|
||||
|
@ -76,54 +77,18 @@ public interface Terms extends Aggregation, Iterable<Terms.Bucket> {
|
|||
static abstract class Order implements ToXContent {
|
||||
|
||||
/**
|
||||
* Order by the (higher) count of each term.
|
||||
* @return a bucket ordering strategy that sorts buckets by their document counts (ascending or descending)
|
||||
*/
|
||||
public static final Order COUNT_DESC = new InternalOrder((byte) 1, "_count", false, new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
long i = o2.getDocCount() - o1.getDocCount();
|
||||
if (i == 0) {
|
||||
i = o2.compareTo(o1);
|
||||
if (i == 0) {
|
||||
i = System.identityHashCode(o2) - System.identityHashCode(o1);
|
||||
}
|
||||
}
|
||||
return i > 0 ? 1 : -1;
|
||||
}
|
||||
});
|
||||
public static Order count(boolean asc) {
|
||||
return asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC;
|
||||
}
|
||||
|
||||
/**
|
||||
* Order by the (lower) count of each term.
|
||||
* @return a bucket ordering strategy that sorts buckets by their terms (ascending or descending)
|
||||
*/
|
||||
public static final Order COUNT_ASC = new InternalOrder((byte) 2, "_count", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return -COUNT_DESC.comparator().compare(o1, o2);
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final Order TERM_DESC = new InternalOrder((byte) 3, "_term", false, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return o2.compareTo(o1);
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final Order TERM_ASC = new InternalOrder((byte) 4, "_term", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return -TERM_DESC.comparator().compare(o1, o2);
|
||||
}
|
||||
});
|
||||
public static Order term(boolean asc) {
|
||||
return asc ? InternalOrder.TERM_ASC : InternalOrder.TERM_DESC;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based on a single-valued calc get
|
||||
|
@ -131,7 +96,7 @@ public interface Terms extends Aggregation, Iterable<Terms.Bucket> {
|
|||
* @param aggregationName the name of the get
|
||||
* @param asc The direction of the order (ascending or descending)
|
||||
*/
|
||||
public static InternalOrder aggregation(String aggregationName, boolean asc) {
|
||||
public static Order aggregation(String aggregationName, boolean asc) {
|
||||
return new InternalOrder.Aggregation(aggregationName, null, asc);
|
||||
}
|
||||
|
||||
|
@ -139,15 +104,24 @@ public interface Terms extends Aggregation, Iterable<Terms.Bucket> {
|
|||
* Creates a bucket ordering strategy which sorts buckets based on a multi-valued calc get
|
||||
*
|
||||
* @param aggregationName the name of the get
|
||||
* @param valueName The name of the value of the multi-value get by which the sorting will be applied
|
||||
* @param metricName The name of the value of the multi-value get by which the sorting will be applied
|
||||
* @param asc The direction of the order (ascending or descending)
|
||||
*/
|
||||
public static InternalOrder aggregation(String aggregationName, String valueName, boolean asc) {
|
||||
return new InternalOrder.Aggregation(aggregationName, valueName, asc);
|
||||
public static Order aggregation(String aggregationName, String metricName, boolean asc) {
|
||||
return new InternalOrder.Aggregation(aggregationName, metricName, asc);
|
||||
}
|
||||
|
||||
|
||||
protected abstract Comparator<Bucket> comparator();
|
||||
/**
|
||||
* @return A comparator for the bucket based on the given terms aggregator. The comparator is used in two phases:
|
||||
*
|
||||
* - aggregation phase, where each shard builds a list of term buckets to be sent to the coordinating node.
|
||||
* In this phase, the passed in aggregator will be the terms aggregator that aggregates the buckets on the
|
||||
* shard level.
|
||||
*
|
||||
* - reduce phase, where the coordinating node gathers all the buckets from all the shards and reduces them
|
||||
* to a final bucket list. In this case, the passed in aggregator will be {@code null}
|
||||
*/
|
||||
protected abstract Comparator<Bucket> comparator(Aggregator aggregator);
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -261,16 +261,16 @@ public class TermsParser implements Aggregator.Parser {
|
|||
|
||||
static InternalOrder resolveOrder(String key, boolean asc) {
|
||||
if ("_term".equals(key)) {
|
||||
return (InternalOrder) (asc ? InternalOrder.TERM_ASC : InternalOrder.TERM_DESC);
|
||||
return asc ? InternalOrder.TERM_ASC : InternalOrder.TERM_DESC;
|
||||
}
|
||||
if ("_count".equals(key)) {
|
||||
return (InternalOrder) (asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC);
|
||||
return asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC;
|
||||
}
|
||||
int i = key.indexOf('.');
|
||||
if (i < 0) {
|
||||
return Terms.Order.aggregation(key, asc);
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
}
|
||||
return Terms.Order.aggregation(key.substring(0, i), key.substring(i+1), asc);
|
||||
return new InternalOrder.Aggregation(key.substring(0, i), key.substring(i+1), asc);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.metrics;
|
||||
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public abstract class MetricsAggregator extends Aggregator {
|
||||
|
||||
private MetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
|
||||
}
|
||||
|
||||
public static abstract class SingleValue extends MetricsAggregator {
|
||||
|
||||
protected SingleValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
}
|
||||
|
||||
public abstract double metric(long owningBucketOrd);
|
||||
}
|
||||
|
||||
public static abstract class MultiValue extends MetricsAggregator {
|
||||
|
||||
protected MultiValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
}
|
||||
|
||||
public abstract boolean hasMetric(String name);
|
||||
|
||||
public abstract double metric(String name, long owningBucketOrd);
|
||||
}
|
||||
}
|
|
@ -25,8 +25,8 @@ import org.elasticsearch.common.util.DoubleArray;
|
|||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.index.fielddata.DoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -37,16 +37,15 @@ import java.io.IOException;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class AvgAggregator extends Aggregator {
|
||||
public class AvgAggregator extends MetricsAggregator.SingleValue {
|
||||
|
||||
private final NumericValuesSource valuesSource;
|
||||
|
||||
private LongArray counts;
|
||||
private DoubleArray sums;
|
||||
|
||||
|
||||
public AvgAggregator(String name, long estimatedBucketsCount, NumericValuesSource valuesSource, AggregationContext context, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
if (valuesSource != null) {
|
||||
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
|
||||
|
@ -81,6 +80,11 @@ public class AvgAggregator extends Aggregator {
|
|||
sums.increment(owningBucketOrdinal, sum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double metric(long owningBucketOrd) {
|
||||
return sums.get(owningBucketOrd) / counts.get(owningBucketOrd);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||
if (valuesSource == null || owningBucketOrdinal >= counts.size()) {
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.avg;
|
||||
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
|
||||
/**
|
||||
*
|
||||
|
|
|
@ -24,8 +24,8 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.index.fielddata.DoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -36,14 +36,14 @@ import java.io.IOException;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class MaxAggregator extends Aggregator {
|
||||
public class MaxAggregator extends MetricsAggregator.SingleValue {
|
||||
|
||||
private final NumericValuesSource valuesSource;
|
||||
|
||||
private DoubleArray maxes;
|
||||
|
||||
public MaxAggregator(String name, long estimatedBucketsCount, NumericValuesSource valuesSource, AggregationContext context, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
if (valuesSource != null) {
|
||||
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
|
||||
|
@ -80,6 +80,11 @@ public class MaxAggregator extends Aggregator {
|
|||
maxes.set(owningBucketOrdinal, max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double metric(long owningBucketOrd) {
|
||||
return maxes.get(owningBucketOrd);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||
if (valuesSource == null) {
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.max;
|
||||
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
|
||||
/**
|
||||
*
|
||||
|
|
|
@ -24,8 +24,8 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.index.fielddata.DoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -36,14 +36,14 @@ import java.io.IOException;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class MinAggregator extends Aggregator {
|
||||
public class MinAggregator extends MetricsAggregator.SingleValue {
|
||||
|
||||
private final NumericValuesSource valuesSource;
|
||||
|
||||
private DoubleArray mins;
|
||||
|
||||
public MinAggregator(String name, long estimatedBucketsCount, NumericValuesSource valuesSource, AggregationContext context, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
if (valuesSource != null) {
|
||||
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
|
||||
|
@ -75,6 +75,11 @@ public class MinAggregator extends Aggregator {
|
|||
mins.set(owningBucketOrdinal, Math.min(values.nextValue(), mins.get(owningBucketOrdinal)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public double metric(long owningBucketOrd) {
|
||||
return mins.get(owningBucketOrd);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||
if (valuesSource == null) {
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.min;
|
||||
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
|
||||
/**
|
||||
*
|
||||
|
|
|
@ -51,6 +51,15 @@ public class InternalStats extends MetricsAggregation.MultiValue implements Stat
|
|||
AggregationStreams.registerStream(STREAM, TYPE.stream());
|
||||
}
|
||||
|
||||
enum Metrics {
|
||||
|
||||
count, sum, min, max, avg;
|
||||
|
||||
public static Metrics resolve(String name) {
|
||||
return Metrics.valueOf(name);
|
||||
}
|
||||
}
|
||||
|
||||
protected long count;
|
||||
protected double min;
|
||||
protected double max;
|
||||
|
@ -98,18 +107,15 @@ public class InternalStats extends MetricsAggregation.MultiValue implements Stat
|
|||
|
||||
@Override
|
||||
public double value(String name) {
|
||||
if ("min".equals(name)) {
|
||||
return min;
|
||||
} else if ("max".equals(name)) {
|
||||
return max;
|
||||
} else if ("avg".equals(name)) {
|
||||
return getAvg();
|
||||
} else if ("count".equals(name)) {
|
||||
return count;
|
||||
} else if ("sum".equals(name)) {
|
||||
return sum;
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown value [" + name + "] in common stats aggregation");
|
||||
Metrics metrics = Metrics.valueOf(name);
|
||||
switch (metrics) {
|
||||
case min: return this.min;
|
||||
case max: return this.max;
|
||||
case avg: return this.getAvg();
|
||||
case count: return this.count;
|
||||
case sum: return this.sum;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown value [" + name + "] in common stats aggregation");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,14 +19,15 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.stats;
|
||||
|
||||
import org.elasticsearch.ElasticsearchIllegalArgumentException;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.index.fielddata.DoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -37,7 +38,7 @@ import java.io.IOException;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class StatsAggegator extends Aggregator {
|
||||
public class StatsAggegator extends MetricsAggregator.MultiValue {
|
||||
|
||||
private final NumericValuesSource valuesSource;
|
||||
|
||||
|
@ -47,7 +48,7 @@ public class StatsAggegator extends Aggregator {
|
|||
private DoubleArray maxes;
|
||||
|
||||
public StatsAggegator(String name, long estimatedBucketsCount, NumericValuesSource valuesSource, AggregationContext context, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
if (valuesSource != null) {
|
||||
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
|
||||
|
@ -101,6 +102,29 @@ public class StatsAggegator extends Aggregator {
|
|||
maxes.set(owningBucketOrdinal, max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMetric(String name) {
|
||||
try {
|
||||
InternalStats.Metrics.resolve(name);
|
||||
return true;
|
||||
} catch (IllegalArgumentException iae) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public double metric(String name, long owningBucketOrd) {
|
||||
switch(InternalStats.Metrics.resolve(name)) {
|
||||
case count: return valuesSource == null ? 0 : counts.get(owningBucketOrd);
|
||||
case sum: return valuesSource == null ? 0 : sums.get(owningBucketOrd);
|
||||
case min: return valuesSource == null ? Double.POSITIVE_INFINITY : mins.get(owningBucketOrd);
|
||||
case max: return valuesSource == null ? Double.NEGATIVE_INFINITY : maxes.get(owningBucketOrd);
|
||||
case avg: return valuesSource == null ? Double.NaN : sums.get(owningBucketOrd) / counts.get(owningBucketOrd);
|
||||
default:
|
||||
throw new ElasticsearchIllegalArgumentException("Unknown value [" + name + "] in common stats aggregation");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||
if (valuesSource == null) {
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.stats;
|
||||
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
|
||||
/**
|
||||
*
|
||||
|
|
|
@ -19,14 +19,15 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.stats.extended;
|
||||
|
||||
import org.elasticsearch.ElasticsearchIllegalArgumentException;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.index.fielddata.DoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -37,7 +38,7 @@ import java.io.IOException;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class ExtendedStatsAggregator extends Aggregator {
|
||||
public class ExtendedStatsAggregator extends MetricsAggregator.MultiValue {
|
||||
|
||||
private final NumericValuesSource valuesSource;
|
||||
|
||||
|
@ -48,7 +49,7 @@ public class ExtendedStatsAggregator extends Aggregator {
|
|||
private DoubleArray sumOfSqrs;
|
||||
|
||||
public ExtendedStatsAggregator(String name, long estimatedBucketsCount, NumericValuesSource valuesSource, AggregationContext context, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
if (valuesSource != null) {
|
||||
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
|
||||
|
@ -107,6 +108,38 @@ public class ExtendedStatsAggregator extends Aggregator {
|
|||
maxes.set(owningBucketOrdinal, max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMetric(String name) {
|
||||
try {
|
||||
InternalExtendedStats.Metrics.resolve(name);
|
||||
return true;
|
||||
} catch (IllegalArgumentException iae) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public double metric(String name, long owningBucketOrd) {
|
||||
switch(InternalExtendedStats.Metrics.resolve(name)) {
|
||||
case count: return valuesSource == null ? 0 : counts.get(owningBucketOrd);
|
||||
case sum: return valuesSource == null ? 0 : sums.get(owningBucketOrd);
|
||||
case min: return valuesSource == null ? Double.POSITIVE_INFINITY : mins.get(owningBucketOrd);
|
||||
case max: return valuesSource == null ? Double.NEGATIVE_INFINITY : maxes.get(owningBucketOrd);
|
||||
case avg: return valuesSource == null ? Double.NaN : sums.get(owningBucketOrd) / counts.get(owningBucketOrd);
|
||||
case sum_of_squares: return valuesSource == null ? 0 : sumOfSqrs.get(owningBucketOrd);
|
||||
case variance: return valuesSource == null ? Double.NaN : variance(owningBucketOrd);
|
||||
case std_deviation: return valuesSource == null ? Double.NaN : Math.sqrt(variance(owningBucketOrd));
|
||||
default:
|
||||
throw new ElasticsearchIllegalArgumentException("Unknown value [" + name + "] in common stats aggregation");
|
||||
}
|
||||
}
|
||||
|
||||
private double variance(long owningBucketOrd) {
|
||||
double sum = sums.get(owningBucketOrd);
|
||||
long count = counts.get(owningBucketOrd);
|
||||
return (sumOfSqrs.get(owningBucketOrd) - ((sum * sum) / count)) / count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||
if (valuesSource == null) {
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.stats.extended;
|
||||
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
|
||||
/**
|
||||
*
|
||||
|
|
|
@ -49,6 +49,15 @@ public class InternalExtendedStats extends InternalStats implements ExtendedStat
|
|||
AggregationStreams.registerStream(STREAM, TYPE.stream());
|
||||
}
|
||||
|
||||
enum Metrics {
|
||||
|
||||
count, sum, min, max, avg, sum_of_squares, variance, std_deviation;
|
||||
|
||||
public static Metrics resolve(String name) {
|
||||
return Metrics.valueOf(name);
|
||||
}
|
||||
}
|
||||
|
||||
private double sumOfSqrs;
|
||||
|
||||
InternalExtendedStats() {} // for serialization
|
||||
|
|
|
@ -24,8 +24,8 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.index.fielddata.DoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -36,14 +36,14 @@ import java.io.IOException;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class SumAggregator extends Aggregator {
|
||||
public class SumAggregator extends MetricsAggregator.SingleValue {
|
||||
|
||||
private final NumericValuesSource valuesSource;
|
||||
|
||||
private DoubleArray sums;
|
||||
|
||||
public SumAggregator(String name, long estimatedBucketsCount, NumericValuesSource valuesSource, AggregationContext context, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
|
||||
super(name, estimatedBucketsCount, context, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
if (valuesSource != null) {
|
||||
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
|
||||
|
@ -75,6 +75,11 @@ public class SumAggregator extends Aggregator {
|
|||
sums.increment(owningBucketOrdinal, sum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double metric(long owningBucketOrd) {
|
||||
return sums.get(owningBucketOrd);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||
if (valuesSource == null) {
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.sum;
|
||||
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
|
||||
/**
|
||||
*
|
||||
|
|
|
@ -24,8 +24,8 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.index.fielddata.BytesValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -39,7 +39,7 @@ import java.io.IOException;
|
|||
* This aggregator works in a multi-bucket mode, that is, when serves as a sub-aggregator, a single aggregator instance aggregates the
|
||||
* counts for all buckets owned by the parent aggregator)
|
||||
*/
|
||||
public class ValueCountAggregator extends Aggregator {
|
||||
public class ValueCountAggregator extends MetricsAggregator.SingleValue {
|
||||
|
||||
private final BytesValuesSource valuesSource;
|
||||
|
||||
|
@ -47,7 +47,7 @@ public class ValueCountAggregator extends Aggregator {
|
|||
LongArray counts;
|
||||
|
||||
public ValueCountAggregator(String name, long expectedBucketsCount, BytesValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent) {
|
||||
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 0, aggregationContext, parent);
|
||||
super(name, 0, aggregationContext, parent);
|
||||
this.valuesSource = valuesSource;
|
||||
if (valuesSource != null) {
|
||||
// expectedBucketsCount == 0 means it's a top level bucket
|
||||
|
@ -71,6 +71,11 @@ public class ValueCountAggregator extends Aggregator {
|
|||
counts.increment(owningBucketOrdinal, values.setDocument(doc));
|
||||
}
|
||||
|
||||
@Override
|
||||
public double metric(long owningBucketOrd) {
|
||||
return counts.get(owningBucketOrd);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||
if (valuesSource == null) {
|
||||
|
|
|
@ -19,12 +19,17 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.query.FilterBuilders;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.Avg;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.hamcrest.Matchers;
|
||||
|
@ -115,7 +120,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
|
|||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.size(20)
|
||||
.order(Terms.Order.TERM_ASC)) // we need to sort by terms cause we're checking the first 20 values
|
||||
.order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -140,7 +145,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
|
|||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.TERM_ASC))
|
||||
.order(Terms.Order.term(true)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -166,7 +171,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
|
|||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.TERM_DESC))
|
||||
.order(Terms.Order.term(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -610,7 +615,235 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
|
|||
assertThat(terms, Matchers.notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().isEmpty(), is(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
Avg avg = bucket.getAggregations().get("avg_i");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo((double) i));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMissingSubAggregation() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", true))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByNonMetricsSubAggregation() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("filter", true))
|
||||
.subAggregation(filter("filter").filter(FilterBuilders.termFilter("foo", "bar")))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation which is not of a metrics type");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValuedSubAggregation_WithUknownMetric() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.foo", true))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
"with an unknown specified metric to order by");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValuedSubAggregation_WithoutMetric() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats", true))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
"where the metric name is not specified");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationDesc() throws Exception {
|
||||
boolean asc = false;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 4; i >= 0; i--) {
|
||||
|
||||
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Avg avg = bucket.getAggregations().get("avg_i");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueSubAggregationAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Stats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueSubAggregationDesc() throws Exception {
|
||||
boolean asc = false;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 4; i >= 0; i--) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Stats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueExtendedStatsAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.variance", asc))
|
||||
.subAggregation(extendedStats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
ExtendedStats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -555,7 +555,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
|
|||
public void multiValuedField_WithValueScript_WithInheritedSubAggregator() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo").field("values").script("_value + 1").interval(interval)
|
||||
.subAggregation(terms("values").order(Terms.Order.TERM_ASC)))
|
||||
.subAggregation(terms("values").order(Terms.Order.term(true))))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
|
|
@ -19,12 +19,17 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.query.FilterBuilders;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.Avg;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.hamcrest.Matchers;
|
||||
|
@ -113,7 +118,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
|
|||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.size(20)
|
||||
.order(Terms.Order.TERM_ASC)) // we need to sort by terms cause we're checking the first 20 values
|
||||
.order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -138,7 +143,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
|
|||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.TERM_ASC))
|
||||
.order(Terms.Order.term(true)))
|
||||
.execute().actionGet();
|
||||
assertSearchResponse(response);
|
||||
|
||||
|
@ -162,7 +167,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
|
|||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.TERM_DESC))
|
||||
.order(Terms.Order.term(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -604,6 +609,233 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
|
|||
assertThat(terms, Matchers.notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().isEmpty(), is(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
Avg avg = bucket.getAggregations().get("avg_i");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo((double) i));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMissingSubAggregation() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", true))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByNonMetricsSubAggregation() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("filter", true))
|
||||
.subAggregation(filter("filter").filter(FilterBuilders.termFilter("foo", "bar")))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation which is not of a metrics type");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValuedSubAggregation_WithUknownMetric() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.foo", true))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
"with an unknown specified metric to order by");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValuedSubAggregation_WithoutMetric() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats", true))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
"where the metric name is not specified");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationDesc() throws Exception {
|
||||
boolean asc = false;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 4; i >= 0; i--) {
|
||||
|
||||
Terms.Bucket bucket = terms.getByTerm("" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Avg avg = bucket.getAggregations().get("avg_i");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueSubAggregationAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Stats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueSubAggregationDesc() throws Exception {
|
||||
boolean asc = false;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 4; i >= 0; i--) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Stats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueExtendedStatsAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.variance", asc))
|
||||
.subAggregation(extendedStats("stats").field("value"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
ExtendedStats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -73,7 +73,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -100,7 +100,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -127,7 +127,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting("1")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -154,7 +154,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -181,7 +181,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -208,7 +208,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting("1")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -235,7 +235,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -262,7 +262,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
@ -289,7 +289,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting("1")
|
||||
.setQuery(matchAllQuery())
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.COUNT_DESC))
|
||||
.addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
Terms terms = response.getAggregations().get("keys");
|
||||
|
|
|
@ -20,14 +20,19 @@
|
|||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.query.FilterBuilders;
|
||||
import org.elasticsearch.search.aggregations.bucket.filter.Filter;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.Avg;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.hamcrest.Matchers;
|
||||
|
@ -74,6 +79,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
lowCardBuilders[i] = client().prepareIndex("idx", "type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field("value", "val" + i)
|
||||
.field("i", i)
|
||||
.startArray("values").value("val" + i).value("val" + (i + 1)).endArray()
|
||||
.endObject());
|
||||
}
|
||||
|
@ -266,7 +272,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.size(20)
|
||||
.order(Terms.Order.TERM_ASC)) // we need to sort by terms cause we're checking the first 20 values
|
||||
.order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -290,7 +296,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.TERM_ASC))
|
||||
.order(Terms.Order.term(true)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -315,7 +321,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.TERM_DESC))
|
||||
.order(Terms.Order.term(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -769,4 +775,240 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
assertThat(terms.buckets().isEmpty(), is(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field("i"))
|
||||
).execute().actionGet();
|
||||
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("val" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("val" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
Avg avg = bucket.getAggregations().get("avg_i");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo((double) i));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMissingSubAggregation() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", true))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByNonMetricsSubAggregation() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("filter", true))
|
||||
.subAggregation(filter("filter").filter(FilterBuilders.termFilter("foo", "bar")))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation which is not of a metrics type");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValuedSubAggregation_WithUknownMetric() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.foo", true))
|
||||
.subAggregation(stats("stats").field("i"))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
"with an unknown specified metric to order by");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValuedSubAggregation_WithoutMetric() throws Exception {
|
||||
|
||||
try {
|
||||
|
||||
client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats", true))
|
||||
.subAggregation(stats("stats").field("i"))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
"where the metric name is not specified");
|
||||
|
||||
} catch (ElasticsearchException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationDesc() throws Exception {
|
||||
boolean asc = false;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field("i"))
|
||||
).execute().actionGet();
|
||||
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 4; i >= 0; i--) {
|
||||
|
||||
Terms.Bucket bucket = terms.getByTerm("val" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("val" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Avg avg = bucket.getAggregations().get("avg_i");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueSubAggregationAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field("i"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("val" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("val" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Stats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueSubAggregationDesc() throws Exception {
|
||||
boolean asc = false;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field("i"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 4; i >= 0; i--) {
|
||||
Terms.Bucket bucket = terms.getByTerm("val" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("val" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
Stats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByMultiValueExtendedStatsAsc() throws Exception {
|
||||
boolean asc = true;
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field("value")
|
||||
.order(Terms.Order.aggregation("stats.variance", asc))
|
||||
.subAggregation(extendedStats("stats").field("i"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.buckets().size(), equalTo(5));
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Terms.Bucket bucket = terms.getByTerm("val" + i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey().string(), equalTo("val" + i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1l));
|
||||
|
||||
ExtendedStats stats = bucket.getAggregations().get("stats");
|
||||
assertThat(stats, notNullValue());
|
||||
assertThat(stats.getMax(), equalTo((double) i));
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue