Aggregations: Adds ability to sort on multiple criteria
The terms aggregation can now support sorting on multiple criteria by replacing the sort object with an array or sort object whose order signifies the priority of the sort. The existing syntax for sorting on a single criteria also still works. Contributes to #6917
This commit is contained in:
parent
11fe940ea9
commit
bfedd11ffa
|
@ -342,6 +342,38 @@ PATH := <AGG_NAME>[<AGG_SEPARATOR><AGG_NAME>]*[<METRIC_SEPARATOR
|
|||
|
||||
The above will sort the countries buckets based on the average height among the female population.
|
||||
|
||||
coming[1.4.0]
|
||||
|
||||
Multiple criteria can be used to order the buckets by providing an array of order criteria such as the following:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"aggs" : {
|
||||
"countries" : {
|
||||
"terms" : {
|
||||
"field" : "address.country",
|
||||
"order" : [ { "females>height_stats.avg" : "desc" }, { "_count" : "desc" } ]
|
||||
},
|
||||
"aggs" : {
|
||||
"females" : {
|
||||
"filter" : { "term" : { "gender" : { "female" }}},
|
||||
"aggs" : {
|
||||
"height_stats" : { "stats" : { "field" : "height" }}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
|
||||
The above will sort the countries buckets based on the average height among the female population and then by
|
||||
their `doc_count` in descending order.
|
||||
|
||||
NOTE: In the event that two buckets share the same values for all order criteria the bucket's term value is used as a
|
||||
tie-breaker in ascending alphabetical order to prevent non-deterministic ordering of buckets.
|
||||
|
||||
==== Minimum document count
|
||||
|
||||
It is possible to only return terms that match more than a configured number of hits using the `min_doc_count` option:
|
||||
|
|
|
@ -33,7 +33,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
|
|||
|
||||
public AbstractStringTermsAggregator(String name, AggregatorFactories factories,
|
||||
long estimatedBucketsCount, AggregationContext context, Aggregator parent,
|
||||
InternalOrder order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
Terms.Order order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds, order, subAggCollectMode);
|
||||
this.showTermDocCountError = showTermDocCountError;
|
||||
}
|
||||
|
|
|
@ -98,7 +98,7 @@ public class DoubleTerms extends InternalTerms {
|
|||
|
||||
DoubleTerms() {} // for serialization
|
||||
|
||||
public DoubleTerms(String name, InternalOrder order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
public DoubleTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError);
|
||||
this.formatter = formatter;
|
||||
}
|
||||
|
|
|
@ -37,7 +37,7 @@ import java.util.Arrays;
|
|||
public class DoubleTermsAggregator extends LongTermsAggregator {
|
||||
|
||||
public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount,
|
||||
InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
|
||||
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
|
||||
super(name, factories, valuesSource, format, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError);
|
||||
}
|
||||
|
||||
|
|
|
@ -68,7 +68,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
protected Collector collector;
|
||||
|
||||
public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, BucketCountThresholds bucketCountThresholds,
|
||||
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds,
|
||||
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
|
||||
super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError);
|
||||
this.valuesSource = valuesSource;
|
||||
|
@ -249,7 +249,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
private final LongHash bucketOrds;
|
||||
|
||||
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext,
|
||||
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext,
|
||||
Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
|
||||
// Set maxOrd to estimatedBucketCount! To be conservative with memory.
|
||||
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError);
|
||||
|
@ -318,7 +318,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
private RandomAccessOrds segmentOrds;
|
||||
|
||||
public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
|
||||
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
|
||||
super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError);
|
||||
assert factories == null || factories.count() == 0;
|
||||
this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true);
|
||||
|
|
|
@ -27,61 +27,59 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.OrderPath;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.*;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
class InternalOrder extends Terms.Order {
|
||||
|
||||
private static final byte COUNT_DESC_ID = 1;
|
||||
private static final byte COUNT_ASC_ID = 2;
|
||||
private static final byte TERM_DESC_ID = 3;
|
||||
private static final byte TERM_ASC_ID = 4;
|
||||
|
||||
/**
|
||||
* Order by the (higher) count of each term.
|
||||
*/
|
||||
public static final InternalOrder COUNT_DESC = new InternalOrder((byte) 1, "_count", false, new Comparator<Terms.Bucket>() {
|
||||
public static final InternalOrder COUNT_DESC = new InternalOrder(COUNT_DESC_ID, "_count", false, new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
int cmp = - Long.compare(o1.getDocCount(), o2.getDocCount());
|
||||
if (cmp == 0) {
|
||||
cmp = o1.compareTerm(o2);
|
||||
}
|
||||
return cmp;
|
||||
return Long.compare(o2.getDocCount(), o1.getDocCount());
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the (lower) count of each term.
|
||||
*/
|
||||
public static final InternalOrder COUNT_ASC = new InternalOrder((byte) 2, "_count", true, new Comparator<Terms.Bucket>() {
|
||||
public static final InternalOrder COUNT_ASC = new InternalOrder(COUNT_ASC_ID, "_count", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
int cmp = Long.compare(o1.getDocCount(), o2.getDocCount());
|
||||
if (cmp == 0) {
|
||||
cmp = o1.compareTerm(o2);
|
||||
}
|
||||
return cmp;
|
||||
return Long.compare(o1.getDocCount(), o2.getDocCount());
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final InternalOrder TERM_DESC = new InternalOrder((byte) 3, "_term", false, new Comparator<Terms.Bucket>() {
|
||||
public static final InternalOrder TERM_DESC = new InternalOrder(TERM_DESC_ID, "_term", false, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return - o1.compareTerm(o2);
|
||||
return o2.compareTerm(o1);
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final InternalOrder TERM_ASC = new InternalOrder((byte) 4, "_term", true, new Comparator<Terms.Bucket>() {
|
||||
public static final InternalOrder TERM_ASC = new InternalOrder(TERM_ASC_ID, "_term", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
|
@ -89,6 +87,18 @@ class InternalOrder extends Terms.Order {
|
|||
}
|
||||
});
|
||||
|
||||
public static boolean isCountDesc(Terms.Order order) {
|
||||
if (order == COUNT_DESC) {
|
||||
return true;
|
||||
}else if (order instanceof CompoundOrder) {
|
||||
// check if its a compound order with count desc and the tie breaker (term asc)
|
||||
CompoundOrder compoundOrder = (CompoundOrder) order;
|
||||
if (compoundOrder.compoundOrder.size() == 2 && compoundOrder.compoundOrder.get(0) == COUNT_DESC && compoundOrder.compoundOrder.get(1) == TERM_ASC) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
final byte id;
|
||||
final String key;
|
||||
|
@ -116,8 +126,13 @@ 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)) {
|
||||
public static Terms.Order validate(Terms.Order order, Aggregator termsAggregator) {
|
||||
if (order instanceof CompoundOrder) {
|
||||
for (Terms.Order innerOrder : ((CompoundOrder)order).compoundOrder) {
|
||||
validate(innerOrder, termsAggregator);
|
||||
}
|
||||
return order;
|
||||
} else if (!(order instanceof Aggregation)) {
|
||||
return order;
|
||||
}
|
||||
OrderPath path = ((Aggregation) order).path();
|
||||
|
@ -199,12 +214,63 @@ class InternalOrder extends Terms.Order {
|
|||
}
|
||||
}
|
||||
|
||||
static class CompoundOrder extends Terms.Order{
|
||||
|
||||
static final byte ID = -1;
|
||||
|
||||
private final List<Terms.Order> compoundOrder;
|
||||
|
||||
public CompoundOrder(List<Terms.Order> compoundOrder) {
|
||||
this.compoundOrder = new LinkedList<>(compoundOrder);
|
||||
}
|
||||
|
||||
@Override
|
||||
byte id() {
|
||||
return ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startArray();
|
||||
for (Terms.Order order : compoundOrder) {
|
||||
order.toXContent(builder, params);
|
||||
}
|
||||
return builder.endArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Bucket> comparator(Aggregator aggregator) {
|
||||
return new CompoundOrderComparator(compoundOrder, aggregator);
|
||||
}
|
||||
|
||||
public static class CompoundOrderComparator implements Comparator<Terms.Bucket> {
|
||||
|
||||
private List<Terms.Order> compoundOrder;
|
||||
private Aggregator aggregator;
|
||||
|
||||
public CompoundOrderComparator(List<Terms.Order> compoundOrder, Aggregator aggregator) {
|
||||
this.compoundOrder = compoundOrder;
|
||||
this.aggregator = aggregator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(Bucket o1, Bucket o2) {
|
||||
int result = 0;
|
||||
for (Iterator<Terms.Order> itr = compoundOrder.iterator(); itr.hasNext() && result == 0;) {
|
||||
result = itr.next().comparator(aggregator).compare(o1, o2);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Streams {
|
||||
|
||||
public static void writeOrder(InternalOrder order, StreamOutput out) throws IOException {
|
||||
public static void writeOrder(Terms.Order order, StreamOutput out) throws IOException {
|
||||
out.writeByte(order.id());
|
||||
if (order instanceof Aggregation) {
|
||||
out.writeBoolean(((MultiBucketsAggregation.Bucket.SubAggregationComparator) order.comparator).asc());
|
||||
Aggregation aggregationOrder = (Aggregation) order;
|
||||
out.writeBoolean(((MultiBucketsAggregation.Bucket.SubAggregationComparator) aggregationOrder.comparator).asc());
|
||||
OrderPath path = ((Aggregation) order).path();
|
||||
if (out.getVersion().onOrAfter(Version.V_1_1_0)) {
|
||||
out.writeString(path.toString());
|
||||
|
@ -218,17 +284,23 @@ class InternalOrder extends Terms.Order {
|
|||
out.writeString(token.key);
|
||||
}
|
||||
}
|
||||
} else if (order instanceof CompoundOrder) {
|
||||
CompoundOrder compoundOrder = (CompoundOrder) order;
|
||||
out.writeVInt(compoundOrder.compoundOrder.size());
|
||||
for (Terms.Order innerOrder : compoundOrder.compoundOrder) {
|
||||
Streams.writeOrder(innerOrder, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static InternalOrder readOrder(StreamInput in) throws IOException {
|
||||
public static Terms.Order readOrder(StreamInput in) throws IOException {
|
||||
byte id = in.readByte();
|
||||
switch (id) {
|
||||
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:
|
||||
case COUNT_DESC_ID: return InternalOrder.COUNT_DESC;
|
||||
case COUNT_ASC_ID: return InternalOrder.COUNT_ASC;
|
||||
case TERM_DESC_ID: return InternalOrder.TERM_DESC;
|
||||
case TERM_ASC_ID: return InternalOrder.TERM_ASC;
|
||||
case Aggregation.ID:
|
||||
boolean asc = in.readBoolean();
|
||||
String key = in.readString();
|
||||
if (in.getVersion().onOrAfter(Version.V_1_1_0)) {
|
||||
|
@ -239,6 +311,13 @@ class InternalOrder extends Terms.Order {
|
|||
return new InternalOrder.Aggregation(key + "." + in.readString(), asc);
|
||||
}
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
case CompoundOrder.ID:
|
||||
int size = in.readVInt();
|
||||
List<Terms.Order> compoundOrder = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
compoundOrder.add(Streams.readOrder(in));
|
||||
}
|
||||
return new CompoundOrder(compoundOrder);
|
||||
default:
|
||||
throw new RuntimeException("unknown terms order");
|
||||
}
|
||||
|
|
|
@ -96,7 +96,7 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
|
|||
}
|
||||
}
|
||||
|
||||
protected InternalOrder order;
|
||||
protected Terms.Order order;
|
||||
protected int requiredSize;
|
||||
protected int shardSize;
|
||||
protected long minDocCount;
|
||||
|
@ -107,7 +107,7 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
|
|||
|
||||
protected InternalTerms() {} // for serialization
|
||||
|
||||
protected InternalTerms(String name, InternalOrder order, int requiredSize, int shardSize, long minDocCount, List<Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
protected InternalTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List<Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
super(name);
|
||||
this.order = order;
|
||||
this.requiredSize = requiredSize;
|
||||
|
@ -150,7 +150,7 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
|
|||
final long thisAggDocCountError;
|
||||
if (terms.buckets.size() < this.shardSize || this.order == InternalOrder.TERM_ASC || this.order == InternalOrder.TERM_DESC) {
|
||||
thisAggDocCountError = 0;
|
||||
} else if (this.order == InternalOrder.COUNT_DESC) {
|
||||
} else if (InternalOrder.isCountDesc(this.order)) {
|
||||
thisAggDocCountError = terms.buckets.get(terms.buckets.size() - 1).docCount;
|
||||
} else {
|
||||
thisAggDocCountError = -1;
|
||||
|
|
|
@ -99,7 +99,7 @@ public class LongTerms extends InternalTerms {
|
|||
|
||||
LongTerms() {} // for serialization
|
||||
|
||||
public LongTerms(String name, InternalOrder order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
public LongTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError);
|
||||
this.formatter = formatter;
|
||||
}
|
||||
|
|
|
@ -48,7 +48,7 @@ public class LongTermsAggregator extends TermsAggregator {
|
|||
private SortedNumericDocValues values;
|
||||
|
||||
public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount,
|
||||
InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode);
|
||||
this.valuesSource = valuesSource;
|
||||
this.showTermDocCountError = showTermDocCountError;
|
||||
|
|
|
@ -98,7 +98,7 @@ public class StringTerms extends InternalTerms {
|
|||
|
||||
StringTerms() {} // for serialization
|
||||
|
||||
public StringTerms(String name, InternalOrder order, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
public StringTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError) {
|
||||
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError);
|
||||
}
|
||||
|
||||
|
|
|
@ -48,7 +48,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
|
|||
private final BytesRefBuilder previous;
|
||||
|
||||
public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
|
||||
InternalOrder order, BucketCountThresholds bucketCountThresholds,
|
||||
Terms.Order order, BucketCountThresholds bucketCountThresholds,
|
||||
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
|
||||
|
||||
super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError);
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.elasticsearch.common.xcontent.ToXContent;
|
|||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -125,6 +126,24 @@ public interface Terms extends MultiBucketsAggregation {
|
|||
return new InternalOrder.Aggregation(aggregationName + "." + metricName, asc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based multiple criteria
|
||||
*
|
||||
* @param orders a list of {@link Order} objects to sort on, in order of priority
|
||||
*/
|
||||
public static Order compound(List<Order> orders) {
|
||||
return new InternalOrder.CompoundOrder(orders);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based multiple criteria
|
||||
*
|
||||
* @param orders a list of {@link Order} parameters to sort on, in order of priority
|
||||
*/
|
||||
public static Order compound(Order... orders) {
|
||||
return compound(Arrays.asList(orders));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A comparator for the bucket based on the given terms aggregator. The comparator is used in two phases:
|
||||
*
|
||||
|
@ -137,5 +156,7 @@ public interface Terms extends MultiBucketsAggregation {
|
|||
*/
|
||||
protected abstract Comparator<Bucket> comparator(Aggregator aggregator);
|
||||
|
||||
abstract byte id();
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -127,11 +127,11 @@ public abstract class TermsAggregator extends BucketsAggregator {
|
|||
}
|
||||
|
||||
protected final BucketCountThresholds bucketCountThresholds;
|
||||
protected InternalOrder order;
|
||||
protected Terms.Order order;
|
||||
protected Aggregator aggUsedForSorting;
|
||||
protected SubAggCollectionMode subAggCollectMode;
|
||||
|
||||
public TermsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, InternalOrder order, SubAggCollectionMode subAggCollectMode) {
|
||||
public TermsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode subAggCollectMode) {
|
||||
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent);
|
||||
this.bucketCountThresholds = bucketCountThresholds;
|
||||
this.order = InternalOrder.validate(order, this);
|
||||
|
|
|
@ -40,7 +40,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
|
|||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
|
|||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
|
||||
}
|
||||
|
@ -70,7 +70,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
|
|||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
|
|||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
|
||||
if (includeExclude != null || factories.count() > 0) {
|
||||
return GLOBAL_ORDINALS.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
|
||||
|
@ -114,7 +114,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
|
|||
}
|
||||
|
||||
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
|
||||
long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds,
|
||||
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds,
|
||||
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError);
|
||||
|
||||
abstract boolean needsGlobalOrdinals();
|
||||
|
@ -125,14 +125,14 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
|
|||
}
|
||||
}
|
||||
|
||||
private final InternalOrder order;
|
||||
private final Terms.Order order;
|
||||
private final IncludeExclude includeExclude;
|
||||
private final String executionHint;
|
||||
private SubAggCollectionMode subAggCollectMode;
|
||||
private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
|
||||
private boolean showTermDocCountError;
|
||||
|
||||
public TermsAggregatorFactory(String name, ValuesSourceConfig config, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint,SubAggCollectionMode executionMode, boolean showTermDocCountError) {
|
||||
public TermsAggregatorFactory(String name, ValuesSourceConfig config, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint,SubAggCollectionMode executionMode, boolean showTermDocCountError) {
|
||||
super(name, StringTerms.TYPE.name(), config);
|
||||
this.order = order;
|
||||
this.includeExclude = includeExclude;
|
||||
|
|
|
@ -25,32 +25,67 @@ import org.elasticsearch.search.SearchParseException;
|
|||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
|
||||
public class TermsParametersParser extends AbstractTermsParametersParser {
|
||||
|
||||
private static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds(1, 0, 10, -1);
|
||||
|
||||
public String getOrderKey() {
|
||||
return orderKey;
|
||||
}
|
||||
|
||||
public boolean isOrderAsc() {
|
||||
return orderAsc;
|
||||
public List<OrderElement> getOrderElements() {
|
||||
return orderElements;
|
||||
}
|
||||
|
||||
public boolean showTermDocCountError() {
|
||||
return showTermDocCountError;
|
||||
}
|
||||
|
||||
String orderKey = "_count";
|
||||
boolean orderAsc = false;
|
||||
List<OrderElement> orderElements;
|
||||
private boolean showTermDocCountError = false;
|
||||
|
||||
public TermsParametersParser() {
|
||||
orderElements = new ArrayList<>(1);
|
||||
orderElements.add(new OrderElement("_count", false));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void parseSpecial(String aggregationName, XContentParser parser, SearchContext context, XContentParser.Token token, String currentFieldName) throws IOException {
|
||||
if (token == XContentParser.Token.START_OBJECT) {
|
||||
if ("order".equals(currentFieldName)) {
|
||||
this.orderElements = Collections.singletonList(parseOrderParam(aggregationName, parser, context));
|
||||
} else {
|
||||
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
|
||||
}
|
||||
} else if (token == XContentParser.Token.START_ARRAY) {
|
||||
if ("order".equals(currentFieldName)) {
|
||||
orderElements = new ArrayList<>();
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
|
||||
if (token == XContentParser.Token.START_OBJECT) {
|
||||
OrderElement orderParam = parseOrderParam(aggregationName, parser, context);
|
||||
orderElements.add(orderParam);
|
||||
} else {
|
||||
throw new SearchParseException(context, "Order elements must be of type object in [" + aggregationName + "].");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
|
||||
}
|
||||
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
|
||||
if (SHOW_TERM_DOC_COUNT_ERROR.match(currentFieldName)) {
|
||||
showTermDocCountError = parser.booleanValue();
|
||||
}
|
||||
} else {
|
||||
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
|
||||
}
|
||||
}
|
||||
|
||||
private OrderElement parseOrderParam(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
|
||||
XContentParser.Token token;
|
||||
OrderElement orderParam = null;
|
||||
String orderKey = null;
|
||||
boolean orderAsc = false;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
orderKey = parser.currentName();
|
||||
|
@ -67,16 +102,32 @@ public class TermsParametersParser extends AbstractTermsParametersParser {
|
|||
throw new SearchParseException(context, "Unexpected token " + token + " for [order] in [" + aggregationName + "].");
|
||||
}
|
||||
}
|
||||
if (orderKey == null) {
|
||||
throw new SearchParseException(context, "Must specify at least one field for [order] in [" + aggregationName + "].");
|
||||
} else {
|
||||
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
|
||||
orderParam = new OrderElement(orderKey, orderAsc);
|
||||
}
|
||||
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
|
||||
if (SHOW_TERM_DOC_COUNT_ERROR.match(currentFieldName)) {
|
||||
showTermDocCountError = parser.booleanValue();
|
||||
return orderParam;
|
||||
}
|
||||
} else {
|
||||
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
|
||||
|
||||
static class OrderElement {
|
||||
private final String key;
|
||||
private final boolean asc;
|
||||
|
||||
public OrderElement(String key, boolean asc) {
|
||||
this.key = key;
|
||||
this.asc = asc;
|
||||
}
|
||||
|
||||
public String key() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public boolean asc() {
|
||||
return asc;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,11 +22,15 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsParametersParser.OrderElement;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -45,7 +49,24 @@ public class TermsParser implements Aggregator.Parser {
|
|||
IncludeExclude.Parser incExcParser = new IncludeExclude.Parser(aggregationName, StringTerms.TYPE, context);
|
||||
aggParser.parse(aggregationName, parser, context, vsParser, incExcParser);
|
||||
|
||||
InternalOrder order = resolveOrder(aggParser.getOrderKey(), aggParser.isOrderAsc());
|
||||
List<OrderElement> orderElements = aggParser.getOrderElements();
|
||||
List<Terms.Order> orders = new ArrayList<>(orderElements.size());
|
||||
for (OrderElement orderElement : orderElements) {
|
||||
orders.add(resolveOrder(orderElement.key(), orderElement.asc()));
|
||||
}
|
||||
Terms.Order order;
|
||||
if (orders.size() == 1 && (orders.get(0) == InternalOrder.TERM_ASC || orders.get(0) == InternalOrder.TERM_DESC))
|
||||
{
|
||||
// If order is only terms order then we don't need the tie-breaker
|
||||
order = orders.get(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
// add term order ascending as a tie-breaker to avoid non-deterministic ordering
|
||||
// if all user provided comparators return 0.
|
||||
orders.add(Order.term(true));
|
||||
order = Order.compound(orders);
|
||||
}
|
||||
TermsAggregator.BucketCountThresholds bucketCountThresholds = aggParser.getBucketCountThresholds();
|
||||
if (!(order == InternalOrder.TERM_ASC || order == InternalOrder.TERM_DESC)
|
||||
&& bucketCountThresholds.getShardSize() == aggParser.getDefaultBucketCountThresholds().getShardSize()) {
|
||||
|
@ -57,14 +78,14 @@ public class TermsParser implements Aggregator.Parser {
|
|||
return new TermsAggregatorFactory(aggregationName, vsParser.config(), order, bucketCountThresholds, aggParser.getIncludeExclude(), aggParser.getExecutionHint(), aggParser.getCollectionMode(), aggParser.showTermDocCountError());
|
||||
}
|
||||
|
||||
static InternalOrder resolveOrder(String key, boolean asc) {
|
||||
static Terms.Order resolveOrder(String key, boolean asc) {
|
||||
if ("_term".equals(key)) {
|
||||
return asc ? InternalOrder.TERM_ASC : InternalOrder.TERM_DESC;
|
||||
return Order.term(asc);
|
||||
}
|
||||
if ("_count".equals(key)) {
|
||||
return asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC;
|
||||
return Order.count(asc);
|
||||
}
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
return Order.aggregation(key, asc);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ public class UnmappedTerms extends InternalTerms {
|
|||
|
||||
UnmappedTerms() {} // for serialization
|
||||
|
||||
public UnmappedTerms(String name, InternalOrder order, int requiredSize, int shardSize, long minDocCount) {
|
||||
public UnmappedTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount) {
|
||||
super(name, order, requiredSize, shardSize, minDocCount, BUCKETS, false, 0);
|
||||
}
|
||||
|
||||
|
|
|
@ -36,9 +36,8 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
|||
import org.hamcrest.Matchers;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery;
|
||||
|
@ -59,6 +58,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
|
|||
private static final int NUM_DOCS = 5; // TODO: randomize the size?
|
||||
private static final String SINGLE_VALUED_FIELD_NAME = "d_value";
|
||||
private static final String MULTI_VALUED_FIELD_NAME = "d_values";
|
||||
private static HashMap<Double, Map<String, Object>> expectedMultiSortBuckets;
|
||||
|
||||
public void setupSuiteScopeCluster() throws Exception {
|
||||
createIndex("idx");
|
||||
|
@ -88,10 +88,123 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
|
|||
.field(SINGLE_VALUED_FIELD_NAME, i*2)
|
||||
.endObject()));
|
||||
}
|
||||
|
||||
getMultiSortDocs(builders);
|
||||
|
||||
indexRandom(true, builders);
|
||||
ensureSearchable();
|
||||
}
|
||||
|
||||
private void getMultiSortDocs(List<IndexRequestBuilder> builders) throws IOException {
|
||||
expectedMultiSortBuckets = new HashMap<>();
|
||||
Map<String, Object> bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 1d);
|
||||
bucketProps.put("_count", 3l);
|
||||
bucketProps.put("avg_l", 1d);
|
||||
bucketProps.put("sum_d", 6d);
|
||||
expectedMultiSortBuckets.put((Double) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 2d);
|
||||
bucketProps.put("_count", 3l);
|
||||
bucketProps.put("avg_l", 2d);
|
||||
bucketProps.put("sum_d", 6d);
|
||||
expectedMultiSortBuckets.put((Double) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 3d);
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 3d);
|
||||
bucketProps.put("sum_d", 3d);
|
||||
expectedMultiSortBuckets.put((Double) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 4d);
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 3d);
|
||||
bucketProps.put("sum_d", 4d);
|
||||
expectedMultiSortBuckets.put((Double) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 5d);
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 3d);
|
||||
expectedMultiSortBuckets.put((Double) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 6d);
|
||||
bucketProps.put("_count", 1l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 1d);
|
||||
expectedMultiSortBuckets.put((Double) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 7d);
|
||||
bucketProps.put("_count", 1l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 1d);
|
||||
expectedMultiSortBuckets.put((Double) bucketProps.get("_term"), bucketProps);
|
||||
|
||||
assertAcked(prepareCreate("sort_idx").addMapping("multi_sort_type", SINGLE_VALUED_FIELD_NAME, "type=double"));
|
||||
for (int i = 1; i <= 3; i++) {
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 1)
|
||||
.field("l", 1)
|
||||
.field("d", i)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 2)
|
||||
.field("l", 2)
|
||||
.field("d", i)
|
||||
.endObject()));
|
||||
}
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 3)
|
||||
.field("l", 3)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 3)
|
||||
.field("l", 3)
|
||||
.field("d", 2)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 4)
|
||||
.field("l", 3)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 4)
|
||||
.field("l", 3)
|
||||
.field("d", 3)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 5)
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 5)
|
||||
.field("l", 5)
|
||||
.field("d", 2)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 6)
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 7)
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
}
|
||||
|
||||
private String key(Terms.Bucket bucket) {
|
||||
return randomBoolean() ? bucket.getKey() : bucket.getKeyAsText().string();
|
||||
}
|
||||
|
@ -1046,4 +1159,78 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAndTermsDesc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 1, 2, 4, 3, 7, 6, 5 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAndTermsAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationDescAndTermsAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 5, 6, 7, 3, 4, 2, 1 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", false), Terms.Order.term(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 6, 7, 3, 4, 5, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(true), Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 6, 7, 3, 5, 4, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("sum_d", true), Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByThreeCriteria() throws Exception {
|
||||
double[] expectedKeys = new double[] { 2, 1, 4, 5, 3, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(false), Terms.Order.aggregation("sum_d", false), Terms.Order.aggregation("avg_l", false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
|
||||
double[] expectedKeys = new double[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
private void assertMultiSortResponse(double[] expectedKeys, Terms.Order... order) {
|
||||
SearchResponse response = client().prepareSearch("sort_idx").setTypes("multi_sort_type")
|
||||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.compound(order))
|
||||
.subAggregation(avg("avg_l").field("l"))
|
||||
.subAggregation(sum("sum_d").field("d"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.getBuckets().size(), equalTo(expectedKeys.length));
|
||||
|
||||
int i = 0;
|
||||
for (Terms.Bucket bucket : terms.getBuckets()) {
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(key(bucket), equalTo(String.valueOf(expectedKeys[i])));
|
||||
assertThat(bucket.getDocCount(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("_count")));
|
||||
Avg avg = bucket.getAggregations().get("avg_l");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("avg_l")));
|
||||
Sum sum = bucket.getAggregations().get("sum_d");
|
||||
assertThat(sum, notNullValue());
|
||||
assertThat(sum.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("sum_d")));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -35,9 +35,8 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
|||
import org.hamcrest.Matchers;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
|
@ -57,6 +56,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
|
|||
private static final int NUM_DOCS = 5; // TODO randomize the size?
|
||||
private static final String SINGLE_VALUED_FIELD_NAME = "l_value";
|
||||
private static final String MULTI_VALUED_FIELD_NAME = "l_values";
|
||||
private static HashMap<Long, Map<String, Object>> expectedMultiSortBuckets;
|
||||
|
||||
@Override
|
||||
public void setupSuiteScopeCluster() throws Exception {
|
||||
|
@ -91,10 +91,123 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
|
|||
.field(SINGLE_VALUED_FIELD_NAME, i * 2)
|
||||
.endObject()));
|
||||
}
|
||||
|
||||
getMultiSortDocs(builders);
|
||||
|
||||
indexRandom(true, builders.toArray(new IndexRequestBuilder[builders.size()]));
|
||||
ensureSearchable();
|
||||
}
|
||||
|
||||
private void getMultiSortDocs(List<IndexRequestBuilder> builders) throws IOException {
|
||||
expectedMultiSortBuckets = new HashMap<>();
|
||||
Map<String, Object> bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 1l);
|
||||
bucketProps.put("_count", 3l);
|
||||
bucketProps.put("avg_l", 1d);
|
||||
bucketProps.put("sum_d", 6d);
|
||||
expectedMultiSortBuckets.put((Long) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 2l);
|
||||
bucketProps.put("_count", 3l);
|
||||
bucketProps.put("avg_l", 2d);
|
||||
bucketProps.put("sum_d", 6d);
|
||||
expectedMultiSortBuckets.put((Long) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 3l);
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 3d);
|
||||
bucketProps.put("sum_d", 3d);
|
||||
expectedMultiSortBuckets.put((Long) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 4l);
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 3d);
|
||||
bucketProps.put("sum_d", 4d);
|
||||
expectedMultiSortBuckets.put((Long) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 5l);
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 3d);
|
||||
expectedMultiSortBuckets.put((Long) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 6l);
|
||||
bucketProps.put("_count", 1l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 1d);
|
||||
expectedMultiSortBuckets.put((Long) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", 7l);
|
||||
bucketProps.put("_count", 1l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 1d);
|
||||
expectedMultiSortBuckets.put((Long) bucketProps.get("_term"), bucketProps);
|
||||
|
||||
createIndex("sort_idx");
|
||||
for (int i = 1; i <= 3; i++) {
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 1)
|
||||
.field("l", 1)
|
||||
.field("d", i)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 2)
|
||||
.field("l", 2)
|
||||
.field("d", i)
|
||||
.endObject()));
|
||||
}
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 3)
|
||||
.field("l", 3)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 3)
|
||||
.field("l", 3)
|
||||
.field("d", 2)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 4)
|
||||
.field("l", 3)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 4)
|
||||
.field("l", 3)
|
||||
.field("d", 3)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 5)
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 5)
|
||||
.field("l", 5)
|
||||
.field("d", 2)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 6)
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, 7)
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
}
|
||||
|
||||
private String key(Terms.Bucket bucket) {
|
||||
return randomBoolean() ? bucket.getKey() : key(bucket);
|
||||
}
|
||||
|
@ -1020,4 +1133,78 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAndTermsDesc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 4, 3, 7, 6, 5 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAndTermsAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationDescAndTermsAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 5, 6, 7, 3, 4, 2, 1 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", false), Terms.Order.term(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 6, 7, 3, 4, 5, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(true), Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 6, 7, 3, 5, 4, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("sum_d", true), Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByThreeCriteria() throws Exception {
|
||||
long[] expectedKeys = new long[] { 2, 1, 4, 5, 3, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(false), Terms.Order.aggregation("sum_d", false), Terms.Order.aggregation("avg_l", false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
private void assertMultiSortResponse(long[] expectedKeys, Terms.Order... order) {
|
||||
SearchResponse response = client().prepareSearch("sort_idx").setTypes("multi_sort_type")
|
||||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.compound(order))
|
||||
.subAggregation(avg("avg_l").field("l"))
|
||||
.subAggregation(sum("sum_d").field("d"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.getBuckets().size(), equalTo(expectedKeys.length));
|
||||
|
||||
int i = 0;
|
||||
for (Terms.Bucket bucket : terms.getBuckets()) {
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(key(bucket), equalTo(String.valueOf(expectedKeys[i])));
|
||||
assertThat(bucket.getDocCount(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("_count")));
|
||||
Avg avg = bucket.getAggregations().get("avg_l");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("avg_l")));
|
||||
Sum sum = bucket.getAggregations().get("sum_d");
|
||||
assertThat(sum, notNullValue());
|
||||
assertThat(sum.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("sum_d")));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -33,14 +33,14 @@ 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.sum.Sum;
|
||||
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
|
@ -60,6 +60,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
private static final String SINGLE_VALUED_FIELD_NAME = "s_value";
|
||||
private static final String MULTI_VALUED_FIELD_NAME = "s_values";
|
||||
private static Map<String, Map<String, Object>> expectedMultiSortBuckets;
|
||||
|
||||
public static String randomExecutionHint() {
|
||||
return randomBoolean() ? null : randomFrom(ExecutionMode.values()).toString();
|
||||
|
@ -79,6 +80,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
.endObject()));
|
||||
}
|
||||
|
||||
getMultiSortDocs(builders);
|
||||
|
||||
for (int i = 0; i < 100; i++) {
|
||||
builders.add(client().prepareIndex("idx", "high_card_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
|
@ -99,6 +102,116 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
ensureSearchable();
|
||||
}
|
||||
|
||||
private void getMultiSortDocs(List<IndexRequestBuilder> builders) throws IOException {
|
||||
expectedMultiSortBuckets = new HashMap<>();
|
||||
Map<String, Object> bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", "val1");
|
||||
bucketProps.put("_count", 3l);
|
||||
bucketProps.put("avg_l", 1d);
|
||||
bucketProps.put("sum_d", 6d);
|
||||
expectedMultiSortBuckets.put((String) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", "val2");
|
||||
bucketProps.put("_count", 3l);
|
||||
bucketProps.put("avg_l", 2d);
|
||||
bucketProps.put("sum_d", 6d);
|
||||
expectedMultiSortBuckets.put((String) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", "val3");
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 3d);
|
||||
bucketProps.put("sum_d", 3d);
|
||||
expectedMultiSortBuckets.put((String) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", "val4");
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 3d);
|
||||
bucketProps.put("sum_d", 4d);
|
||||
expectedMultiSortBuckets.put((String) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", "val5");
|
||||
bucketProps.put("_count", 2l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 3d);
|
||||
expectedMultiSortBuckets.put((String) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", "val6");
|
||||
bucketProps.put("_count", 1l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 1d);
|
||||
expectedMultiSortBuckets.put((String) bucketProps.get("_term"), bucketProps);
|
||||
bucketProps = new HashMap<>();
|
||||
bucketProps.put("_term", "val7");
|
||||
bucketProps.put("_count", 1l);
|
||||
bucketProps.put("avg_l", 5d);
|
||||
bucketProps.put("sum_d", 1d);
|
||||
expectedMultiSortBuckets.put((String) bucketProps.get("_term"), bucketProps);
|
||||
|
||||
createIndex("sort_idx");
|
||||
for (int i = 1; i <= 3; i++) {
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val1")
|
||||
.field("l", 1)
|
||||
.field("d", i)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val2")
|
||||
.field("l", 2)
|
||||
.field("d", i)
|
||||
.endObject()));
|
||||
}
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val3")
|
||||
.field("l", 3)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val3")
|
||||
.field("l", 3)
|
||||
.field("d", 2)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val4")
|
||||
.field("l", 3)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val4")
|
||||
.field("l", 3)
|
||||
.field("d", 3)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val5")
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val5")
|
||||
.field("l", 5)
|
||||
.field("d", 2)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val6")
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "multi_sort_type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, "val7")
|
||||
.field("l", 5)
|
||||
.field("d", 1)
|
||||
.endObject()));
|
||||
}
|
||||
|
||||
private String key(Terms.Bucket bucket) {
|
||||
return randomBoolean() ? bucket.getKey() : bucket.getKeyAsText().string();
|
||||
}
|
||||
|
@ -1368,6 +1481,82 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAndTermsDesc() throws Exception {
|
||||
String[] expectedKeys = new String[] { "val1", "val2", "val4", "val3", "val7", "val6", "val5" };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAndTermsAsc() throws Exception {
|
||||
String[] expectedKeys = new String[] { "val1", "val2", "val3", "val4", "val5", "val6", "val7" };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationDescAndTermsAsc() throws Exception {
|
||||
String[] expectedKeys = new String[] { "val5", "val6", "val7", "val3", "val4", "val2", "val1" };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", false), Terms.Order.term(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
|
||||
String[] expectedKeys = new String[] { "val6", "val7", "val3", "val4", "val5", "val1", "val2" };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(true), Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
|
||||
String[] expectedKeys = new String[] { "val6", "val7", "val3", "val5", "val4", "val1", "val2" };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("sum_d", true), Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedByThreeCriteria() throws Exception {
|
||||
String[] expectedKeys = new String[] { "val2", "val1", "val4", "val5", "val3", "val6", "val7" };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(false), Terms.Order.aggregation("sum_d", false), Terms.Order.aggregation("avg_l", false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void singleValuedField_OrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
|
||||
String[] expectedKeys = new String[] { "val1", "val2", "val3", "val4", "val5", "val6", "val7" };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
private void assertMultiSortResponse(String[] expectedKeys, Terms.Order... order) {
|
||||
SearchResponse response = client().prepareSearch("sort_idx").setTypes("multi_sort_type")
|
||||
.addAggregation(terms("terms")
|
||||
.executionHint(randomExecutionHint())
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.compound(order))
|
||||
.subAggregation(avg("avg_l").field("l"))
|
||||
.subAggregation(sum("sum_d").field("d"))
|
||||
).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.getBuckets().size(), equalTo(expectedKeys.length));
|
||||
|
||||
int i = 0;
|
||||
for (Terms.Bucket bucket : terms.getBuckets()) {
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(key(bucket), equalTo(expectedKeys[i]));
|
||||
assertThat(bucket.getDocCount(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("_count")));
|
||||
Avg avg = bucket.getAggregations().get("avg_l");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("avg_l")));
|
||||
Sum sum = bucket.getAggregations().get("sum_d");
|
||||
assertThat(sum, notNullValue());
|
||||
assertThat(sum.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("sum_d")));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void indexMetaField() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx", "empty_bucket_idx").setTypes("type")
|
||||
|
|
Loading…
Reference in New Issue