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:
Colin Goodheart-Smithe 2014-09-03 09:26:54 +01:00
parent 11fe940ea9
commit bfedd11ffa
20 changed files with 909 additions and 142 deletions

View File

@ -54,19 +54,19 @@ size buckets was not returned). If set to `0`, the `size` will be set to `Intege
==== Document counts are approximate ==== Document counts are approximate
As described above, the document counts (and the results of any sub aggregations) in the terms aggregation are not always As described above, the document counts (and the results of any sub aggregations) in the terms aggregation are not always
accurate. This is because each shard provides its own view of what the ordered list of terms should be and these are accurate. This is because each shard provides its own view of what the ordered list of terms should be and these are
combined to give a final view. Consider the following scenario: combined to give a final view. Consider the following scenario:
A request is made to obtain the top 5 terms in the field product, ordered by descending document count from an index with A request is made to obtain the top 5 terms in the field product, ordered by descending document count from an index with
3 shards. In this case each shard is asked to give its top 5 terms. 3 shards. In this case each shard is asked to give its top 5 terms.
[source,js] [source,js]
-------------------------------------------------- --------------------------------------------------
{ {
"aggs" : { "aggs" : {
"products" : { "products" : {
"terms" : { "terms" : {
"field" : "product", "field" : "product",
"size" : 5 "size" : 5
} }
@ -75,23 +75,23 @@ A request is made to obtain the top 5 terms in the field product, ordered by des
} }
-------------------------------------------------- --------------------------------------------------
The terms for each of the three shards are shown below with their The terms for each of the three shards are shown below with their
respective document counts in brackets: respective document counts in brackets:
[width="100%",cols="^2,^2,^2,^2",options="header"] [width="100%",cols="^2,^2,^2,^2",options="header"]
|========================================================= |=========================================================
| | Shard A | Shard B | Shard C | | Shard A | Shard B | Shard C
| 1 | Product A (25) | Product A (30) | Product A (45) | 1 | Product A (25) | Product A (30) | Product A (45)
| 2 | Product B (18) | Product B (25) | Product C (44) | 2 | Product B (18) | Product B (25) | Product C (44)
| 3 | Product C (6) | Product F (17) | Product Z (36) | 3 | Product C (6) | Product F (17) | Product Z (36)
| 4 | Product D (3) | Product Z (16) | Product G (30) | 4 | Product D (3) | Product Z (16) | Product G (30)
| 5 | Product E (2) | Product G (15) | Product E (29) | 5 | Product E (2) | Product G (15) | Product E (29)
| 6 | Product F (2) | Product H (14) | Product H (28) | 6 | Product F (2) | Product H (14) | Product H (28)
| 7 | Product G (2) | Product I (10) | Product Q (2) | 7 | Product G (2) | Product I (10) | Product Q (2)
| 8 | Product H (2) | Product Q (6) | Product D (1) | 8 | Product H (2) | Product Q (6) | Product D (1)
| 9 | Product I (1) | Product J (8) | | 9 | Product I (1) | Product J (8) |
| 10 | Product J (1) | Product C (4) | | 10 | Product J (1) | Product C (4) |
|========================================================= |=========================================================
@ -102,41 +102,41 @@ The shards will return their top 5 terms so the results from the shards will be:
|========================================================= |=========================================================
| | Shard A | Shard B | Shard C | | Shard A | Shard B | Shard C
| 1 | Product A (25) | Product A (30) | Product A (45) | 1 | Product A (25) | Product A (30) | Product A (45)
| 2 | Product B (18) | Product B (25) | Product C (44) | 2 | Product B (18) | Product B (25) | Product C (44)
| 3 | Product C (6) | Product F (17) | Product Z (36) | 3 | Product C (6) | Product F (17) | Product Z (36)
| 4 | Product D (3) | Product Z (16) | Product G (30) | 4 | Product D (3) | Product Z (16) | Product G (30)
| 5 | Product E (2) | Product G (15) | Product E (29) | 5 | Product E (2) | Product G (15) | Product E (29)
|========================================================= |=========================================================
Taking the top 5 results from each of the shards (as requested) and combining them to make a final top 5 list produces Taking the top 5 results from each of the shards (as requested) and combining them to make a final top 5 list produces
the following: the following:
[width="40%",cols="^2,^2"] [width="40%",cols="^2,^2"]
|========================================================= |=========================================================
| 1 | Product A (100) | 1 | Product A (100)
| 2 | Product Z (52) | 2 | Product Z (52)
| 3 | Product C (50) | 3 | Product C (50)
| 4 | Product G (45) | 4 | Product G (45)
| 5 | Product B (43) | 5 | Product B (43)
|========================================================= |=========================================================
Because Product A was returned from all shards we know that its document count value is accurate. Product C was only Because Product A was returned from all shards we know that its document count value is accurate. Product C was only
returned by shards A and C so its document count is shown as 50 but this is not an accurate count. Product C exists on returned by shards A and C so its document count is shown as 50 but this is not an accurate count. Product C exists on
shard B, but its count of 4 was not high enough to put Product C into the top 5 list for that shard. Product Z was also shard B, but its count of 4 was not high enough to put Product C into the top 5 list for that shard. Product Z was also
returned only by 2 shards but the third shard does not contain the term. There is no way of knowing, at the point of returned only by 2 shards but the third shard does not contain the term. There is no way of knowing, at the point of
combining the results to produce the final list of terms, that there is an error in the document count for Product C and combining the results to produce the final list of terms, that there is an error in the document count for Product C and
not for Product Z. Product H has a document count of 44 across all 3 shards but was not included in the final list of not for Product Z. Product H has a document count of 44 across all 3 shards but was not included in the final list of
terms because it did not make it into the top five terms on any of the shards. terms because it did not make it into the top five terms on any of the shards.
==== Shard Size ==== Shard Size
The higher the requested `size` is, the more accurate the results will be, but also, the more expensive it will be to The higher the requested `size` is, the more accurate the results will be, but also, the more expensive it will be to
compute the final results (both due to bigger priority queues that are managed on a shard level and due to bigger data compute the final results (both due to bigger priority queues that are managed on a shard level and due to bigger data
transfers between the nodes and the client). transfers between the nodes and the client).
The `shard_size` parameter can be used to minimize the extra work that comes with bigger requested `size`. When defined, The `shard_size` parameter can be used to minimize the extra work that comes with bigger requested `size`. When defined,
it will determine how many terms the coordinating node will request from each shard. Once all the shards responded, the it will determine how many terms the coordinating node will request from each shard. Once all the shards responded, the
@ -153,12 +153,12 @@ on high-cardinality fields as this will kill both your CPU since terms need to b
==== Calculating Document Count Error ==== Calculating Document Count Error
coming[1.4.0] coming[1.4.0]
There are two error values which can be shown on the terms aggregation. The first gives a value for the aggregation as There are two error values which can be shown on the terms aggregation. The first gives a value for the aggregation as
a whole which represents the maximum potential document count for a term which did not make it into the final list of a whole which represents the maximum potential document count for a term which did not make it into the final list of
terms. This is calculated as the sum of the document count from the last term returned from each shard .For the example terms. This is calculated as the sum of the document count from the last term returned from each shard .For the example
given above the value would be 46 (2 + 15 + 29). This means that in the worst case scenario a term which was not returned given above the value would be 46 (2 + 15 + 29). This means that in the worst case scenario a term which was not returned
could have the 4th highest document count. could have the 4th highest document count.
[source,js] [source,js]
@ -185,13 +185,13 @@ could have the 4th highest document count.
} }
-------------------------------------------------- --------------------------------------------------
The second error value can be enabled by setting the `show_term_doc_count_error` parameter to true. This shows an error value The second error value can be enabled by setting the `show_term_doc_count_error` parameter to true. This shows an error value
for each term returned by the aggregation which represents the 'worst case' error in the document count and can be useful when for each term returned by the aggregation which represents the 'worst case' error in the document count and can be useful when
deciding on a value for the `shard_size` parameter. This is calculated by summing the document counts for the last term returned deciding on a value for the `shard_size` parameter. This is calculated by summing the document counts for the last term returned
by all shards which did not return the term. In the example above the error in the document count for Product C would be 15 as by all shards which did not return the term. In the example above the error in the document count for Product C would be 15 as
Shard B was the only shard not to return the term and the document count of the last termit did return was 15. The actual document Shard B was the only shard not to return the term and the document count of the last termit did return was 15. The actual document
count of Product C was 54 so the document count was only actually off by 4 even though the worst case was that it would be off by count of Product C was 54 so the document count was only actually off by 4 even though the worst case was that it would be off by
15. Product A, however has an error of 0 for its document count, since every shard returned it we can be confident that the count 15. Product A, however has an error of 0 for its document count, since every shard returned it we can be confident that the count
returned is accurate. returned is accurate.
[source,js] [source,js]
@ -220,10 +220,10 @@ returned is accurate.
} }
-------------------------------------------------- --------------------------------------------------
These errors can only be calculated in this way when the terms are ordered by descending document count. When the aggregation is These errors can only be calculated in this way when the terms are ordered by descending document count. When the aggregation is
ordered by the terms values themselves (either ascending or descending) there is no error in the document count since if a shard ordered by the terms values themselves (either ascending or descending) there is no error in the document count since if a shard
does not return a particular term which appears in the results from another shard, it must not have that term in its index. When the does not return a particular term which appears in the results from another shard, it must not have that term in its index. When the
aggregation is either sorted by a sub aggregation or in order of ascending document count, the error in the document counts cannot be aggregation is either sorted by a sub aggregation or in order of ascending document count, the error in the document counts cannot be
determined and is given a value of -1 to indicate this. determined and is given a value of -1 to indicate this.
==== Order ==== Order
@ -342,7 +342,39 @@ 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. The above will sort the countries buckets based on the average height among the female population.
==== Minimum document count 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: It is possible to only return terms that match more than a configured number of hits using the `min_doc_count` option:
@ -511,7 +543,7 @@ added[1.3.0] Deferring calculation of child aggregations
For fields with many unique terms and a small number of required results it can be more efficient to delay the calculation For fields with many unique terms and a small number of required results it can be more efficient to delay the calculation
of child aggregations until the top parent-level aggs have been pruned. Ordinarily, all branches of the aggregation tree of child aggregations until the top parent-level aggs have been pruned. Ordinarily, all branches of the aggregation tree
are expanded in one depth-first pass and only then any pruning occurs. In some rare scenarios this can be very wasteful and can hit memory constraints. are expanded in one depth-first pass and only then any pruning occurs. In some rare scenarios this can be very wasteful and can hit memory constraints.
An example problem scenario is querying a movie database for the 10 most popular actors and their 5 most common co-stars: An example problem scenario is querying a movie database for the 10 most popular actors and their 5 most common co-stars:
[source,js] [source,js]
-------------------------------------------------- --------------------------------------------------
@ -535,11 +567,11 @@ An example problem scenario is querying a movie database for the 10 most popular
} }
-------------------------------------------------- --------------------------------------------------
Even though the number of movies may be comparatively small and we want only 50 result buckets there is a combinatorial explosion of buckets Even though the number of movies may be comparatively small and we want only 50 result buckets there is a combinatorial explosion of buckets
during calculation - a single movie will produce n² buckets where n is the number of actors. The sane option would be to first determine during calculation - a single movie will produce n² buckets where n is the number of actors. The sane option would be to first determine
the 10 most popular actors and only then examine the top co-stars for these 10 actors. This alternative strategy is what we call the `breadth_first` collection the 10 most popular actors and only then examine the top co-stars for these 10 actors. This alternative strategy is what we call the `breadth_first` collection
mode as opposed to the default `depth_first` mode: mode as opposed to the default `depth_first` mode:
[source,js] [source,js]
-------------------------------------------------- --------------------------------------------------
{ {
@ -565,16 +597,16 @@ mode as opposed to the default `depth_first` mode:
When using `breadth_first` mode the set of documents that fall into the uppermost buckets are When using `breadth_first` mode the set of documents that fall into the uppermost buckets are
cached for subsequent replay so there is a memory overhead in doing this which is linear with the number of matching documents. cached for subsequent replay so there is a memory overhead in doing this which is linear with the number of matching documents.
In most requests the volume of buckets generated is smaller than the number of documents that fall into them so the default `depth_first` In most requests the volume of buckets generated is smaller than the number of documents that fall into them so the default `depth_first`
collection mode is normally the best bet but occasionally the `breadth_first` strategy can be significantly more efficient. Currently collection mode is normally the best bet but occasionally the `breadth_first` strategy can be significantly more efficient. Currently
elasticsearch will always use the `depth_first` collect_mode unless explicitly instructed to use `breadth_first` as in the above example. elasticsearch will always use the `depth_first` collect_mode unless explicitly instructed to use `breadth_first` as in the above example.
Note that the `order` parameter can still be used to refer to data from a child aggregation when using the `breadth_first` setting - the parent Note that the `order` parameter can still be used to refer to data from a child aggregation when using the `breadth_first` setting - the parent
aggregation understands that this child aggregation will need to be called first before any of the other child aggregations. aggregation understands that this child aggregation will need to be called first before any of the other child aggregations.
WARNING: It is not possible to nest aggregations such as `top_hits` which require access to match score information under an aggregation that uses WARNING: It is not possible to nest aggregations such as `top_hits` which require access to match score information under an aggregation that uses
the `breadth_first` collection mode. This is because this would require a RAM buffer to hold the float score value for every document and the `breadth_first` collection mode. This is because this would require a RAM buffer to hold the float score value for every document and
this would typically be too costly in terms of RAM. this would typically be too costly in terms of RAM.
[[search-aggregations-bucket-terms-aggregation-execution-hint]] [[search-aggregations-bucket-terms-aggregation-execution-hint]]
==== Execution hint ==== Execution hint

View File

@ -33,7 +33,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
public AbstractStringTermsAggregator(String name, AggregatorFactories factories, public AbstractStringTermsAggregator(String name, AggregatorFactories factories,
long estimatedBucketsCount, AggregationContext context, Aggregator parent, 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); super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds, order, subAggCollectMode);
this.showTermDocCountError = showTermDocCountError; this.showTermDocCountError = showTermDocCountError;
} }

View File

@ -98,7 +98,7 @@ public class DoubleTerms extends InternalTerms {
DoubleTerms() {} // for serialization 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); super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError);
this.formatter = formatter; this.formatter = formatter;
} }

View File

@ -37,7 +37,7 @@ import java.util.Arrays;
public class DoubleTermsAggregator extends LongTermsAggregator { public class DoubleTermsAggregator extends LongTermsAggregator {
public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount, 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); super(name, factories, valuesSource, format, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError);
} }

View File

@ -68,7 +68,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
protected Collector collector; protected Collector collector;
public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, 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) { IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError); super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
@ -249,7 +249,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
private final LongHash bucketOrds; private final LongHash bucketOrds;
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, 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) { Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
// Set maxOrd to estimatedBucketCount! To be conservative with memory. // Set maxOrd to estimatedBucketCount! To be conservative with memory.
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError); 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; private RandomAccessOrds segmentOrds;
public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, 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); super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError);
assert factories == null || factories.count() == 0; assert factories == null || factories.count() == 0;
this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true); this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true);

View File

@ -27,61 +27,59 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; 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.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.OrderPath; import org.elasticsearch.search.aggregations.support.OrderPath;
import java.io.IOException; import java.io.IOException;
import java.util.Comparator; import java.util.*;
/** /**
* *
*/ */
class InternalOrder extends Terms.Order { 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. * 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 @Override
public int compare(Terms.Bucket o1, Terms.Bucket o2) { public int compare(Terms.Bucket o1, Terms.Bucket o2) {
int cmp = - Long.compare(o1.getDocCount(), o2.getDocCount()); return Long.compare(o2.getDocCount(), o1.getDocCount());
if (cmp == 0) {
cmp = o1.compareTerm(o2);
}
return cmp;
} }
}); });
/** /**
* Order by the (lower) count of each term. * 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 @Override
public int compare(Terms.Bucket o1, Terms.Bucket o2) { public int compare(Terms.Bucket o1, Terms.Bucket o2) {
int cmp = Long.compare(o1.getDocCount(), o2.getDocCount()); return Long.compare(o1.getDocCount(), o2.getDocCount());
if (cmp == 0) {
cmp = o1.compareTerm(o2);
}
return cmp;
} }
}); });
/** /**
* Order by the terms. * 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 @Override
public int compare(Terms.Bucket o1, Terms.Bucket o2) { public int compare(Terms.Bucket o1, Terms.Bucket o2) {
return - o1.compareTerm(o2); return o2.compareTerm(o1);
} }
}); });
/** /**
* Order by the terms. * 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 @Override
public int compare(Terms.Bucket o1, Terms.Bucket o2) { 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 byte id;
final String key; final String key;
@ -116,8 +126,13 @@ class InternalOrder extends Terms.Order {
return builder.startObject().field(key, asc ? "asc" : "desc").endObject(); return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
} }
public static InternalOrder validate(InternalOrder order, Aggregator termsAggregator) { public static Terms.Order validate(Terms.Order order, Aggregator termsAggregator) {
if (!(order instanceof Aggregation)) { if (order instanceof CompoundOrder) {
for (Terms.Order innerOrder : ((CompoundOrder)order).compoundOrder) {
validate(innerOrder, termsAggregator);
}
return order;
} else if (!(order instanceof Aggregation)) {
return order; return order;
} }
OrderPath path = ((Aggregation) order).path(); 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 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()); out.writeByte(order.id());
if (order instanceof Aggregation) { 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(); OrderPath path = ((Aggregation) order).path();
if (out.getVersion().onOrAfter(Version.V_1_1_0)) { if (out.getVersion().onOrAfter(Version.V_1_1_0)) {
out.writeString(path.toString()); out.writeString(path.toString());
@ -218,17 +284,23 @@ class InternalOrder extends Terms.Order {
out.writeString(token.key); 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(); byte id = in.readByte();
switch (id) { switch (id) {
case 1: return InternalOrder.COUNT_DESC; case COUNT_DESC_ID: return InternalOrder.COUNT_DESC;
case 2: return InternalOrder.COUNT_ASC; case COUNT_ASC_ID: return InternalOrder.COUNT_ASC;
case 3: return InternalOrder.TERM_DESC; case TERM_DESC_ID: return InternalOrder.TERM_DESC;
case 4: return InternalOrder.TERM_ASC; case TERM_ASC_ID: return InternalOrder.TERM_ASC;
case 0: case Aggregation.ID:
boolean asc = in.readBoolean(); boolean asc = in.readBoolean();
String key = in.readString(); String key = in.readString();
if (in.getVersion().onOrAfter(Version.V_1_1_0)) { 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 + "." + in.readString(), asc);
} }
return new InternalOrder.Aggregation(key, 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: default:
throw new RuntimeException("unknown terms order"); throw new RuntimeException("unknown terms order");
} }

View File

@ -96,7 +96,7 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
} }
} }
protected InternalOrder order; protected Terms.Order order;
protected int requiredSize; protected int requiredSize;
protected int shardSize; protected int shardSize;
protected long minDocCount; protected long minDocCount;
@ -107,7 +107,7 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
protected InternalTerms() {} // for serialization 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); super(name);
this.order = order; this.order = order;
this.requiredSize = requiredSize; this.requiredSize = requiredSize;
@ -150,7 +150,7 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
final long thisAggDocCountError; final long thisAggDocCountError;
if (terms.buckets.size() < this.shardSize || this.order == InternalOrder.TERM_ASC || this.order == InternalOrder.TERM_DESC) { if (terms.buckets.size() < this.shardSize || this.order == InternalOrder.TERM_ASC || this.order == InternalOrder.TERM_DESC) {
thisAggDocCountError = 0; thisAggDocCountError = 0;
} else if (this.order == InternalOrder.COUNT_DESC) { } else if (InternalOrder.isCountDesc(this.order)) {
thisAggDocCountError = terms.buckets.get(terms.buckets.size() - 1).docCount; thisAggDocCountError = terms.buckets.get(terms.buckets.size() - 1).docCount;
} else { } else {
thisAggDocCountError = -1; thisAggDocCountError = -1;

View File

@ -99,7 +99,7 @@ public class LongTerms extends InternalTerms {
LongTerms() {} // for serialization 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); super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError);
this.formatter = formatter; this.formatter = formatter;
} }

View File

@ -48,7 +48,7 @@ public class LongTermsAggregator extends TermsAggregator {
private SortedNumericDocValues values; private SortedNumericDocValues values;
public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount, 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); super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.showTermDocCountError = showTermDocCountError; this.showTermDocCountError = showTermDocCountError;

View File

@ -98,7 +98,7 @@ public class StringTerms extends InternalTerms {
StringTerms() {} // for serialization 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); super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError);
} }

View File

@ -48,7 +48,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
private final BytesRefBuilder previous; private final BytesRefBuilder previous;
public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, 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) { IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError); super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError);

View File

@ -22,6 +22,7 @@ import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import java.util.Arrays;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
@ -125,6 +126,24 @@ public interface Terms extends MultiBucketsAggregation {
return new InternalOrder.Aggregation(aggregationName + "." + metricName, asc); 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: * @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); protected abstract Comparator<Bucket> comparator(Aggregator aggregator);
abstract byte id();
} }
} }

View File

@ -127,11 +127,11 @@ public abstract class TermsAggregator extends BucketsAggregator {
} }
protected final BucketCountThresholds bucketCountThresholds; protected final BucketCountThresholds bucketCountThresholds;
protected InternalOrder order; protected Terms.Order order;
protected Aggregator aggUsedForSorting; protected Aggregator aggUsedForSorting;
protected SubAggCollectionMode subAggCollectMode; 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); super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent);
this.bucketCountThresholds = bucketCountThresholds; this.bucketCountThresholds = bucketCountThresholds;
this.order = InternalOrder.validate(order, this); this.order = InternalOrder.validate(order, this);

View File

@ -40,7 +40,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, 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) { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, 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 @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, 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) { 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); 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 @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, 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) { 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); 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 @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, 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) { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
if (includeExclude != null || factories.count() > 0) { if (includeExclude != null || factories.count() > 0) {
return GLOBAL_ORDINALS.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError); 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, 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); IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError);
abstract boolean needsGlobalOrdinals(); 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 IncludeExclude includeExclude;
private final String executionHint; private final String executionHint;
private SubAggCollectionMode subAggCollectMode; private SubAggCollectionMode subAggCollectMode;
private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
private boolean showTermDocCountError; 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); super(name, StringTerms.TYPE.name(), config);
this.order = order; this.order = order;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;

View File

@ -25,46 +25,48 @@ import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
public class TermsParametersParser extends AbstractTermsParametersParser { public class TermsParametersParser extends AbstractTermsParametersParser {
private static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds(1, 0, 10, -1); private static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds(1, 0, 10, -1);
public String getOrderKey() { public List<OrderElement> getOrderElements() {
return orderKey; return orderElements;
}
public boolean isOrderAsc() {
return orderAsc;
} }
public boolean showTermDocCountError() { public boolean showTermDocCountError() {
return showTermDocCountError; return showTermDocCountError;
} }
String orderKey = "_count"; List<OrderElement> orderElements;
boolean orderAsc = false;
private boolean showTermDocCountError = false; private boolean showTermDocCountError = false;
public TermsParametersParser() {
orderElements = new ArrayList<>(1);
orderElements.add(new OrderElement("_count", false));
}
@Override @Override
public void parseSpecial(String aggregationName, XContentParser parser, SearchContext context, XContentParser.Token token, String currentFieldName) throws IOException { public void parseSpecial(String aggregationName, XContentParser parser, SearchContext context, XContentParser.Token token, String currentFieldName) throws IOException {
if (token == XContentParser.Token.START_OBJECT) { if (token == XContentParser.Token.START_OBJECT) {
if ("order".equals(currentFieldName)) { if ("order".equals(currentFieldName)) {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { this.orderElements = Collections.singletonList(parseOrderParam(aggregationName, parser, context));
if (token == XContentParser.Token.FIELD_NAME) { } else {
orderKey = parser.currentName(); throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
} else if (token == XContentParser.Token.VALUE_STRING) { }
String dir = parser.text(); } else if (token == XContentParser.Token.START_ARRAY) {
if ("asc".equalsIgnoreCase(dir)) { if ("order".equals(currentFieldName)) {
orderAsc = true; orderElements = new ArrayList<>();
} else if ("desc".equalsIgnoreCase(dir)) { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
orderAsc = false; if (token == XContentParser.Token.START_OBJECT) {
} else { OrderElement orderParam = parseOrderParam(aggregationName, parser, context);
throw new SearchParseException(context, "Unknown terms order direction [" + dir + "] in terms aggregation [" + aggregationName + "]"); orderElements.add(orderParam);
}
} else { } else {
throw new SearchParseException(context, "Unexpected token " + token + " for [order] in [" + aggregationName + "]."); throw new SearchParseException(context, "Order elements must be of type object in [" + aggregationName + "].");
} }
} }
} else { } else {
@ -79,6 +81,55 @@ public class TermsParametersParser extends AbstractTermsParametersParser {
} }
} }
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();
} else if (token == XContentParser.Token.VALUE_STRING) {
String dir = parser.text();
if ("asc".equalsIgnoreCase(dir)) {
orderAsc = true;
} else if ("desc".equalsIgnoreCase(dir)) {
orderAsc = false;
} else {
throw new SearchParseException(context, "Unknown terms order direction [" + dir + "] in terms aggregation [" + aggregationName + "]");
}
} else {
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 {
orderParam = new OrderElement(orderKey, orderAsc);
}
return orderParam;
}
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 @Override
public TermsAggregator.BucketCountThresholds getDefaultBucketCountThresholds() { public TermsAggregator.BucketCountThresholds getDefaultBucketCountThresholds() {
return new TermsAggregator.BucketCountThresholds(DEFAULT_BUCKET_COUNT_THRESHOLDS); return new TermsAggregator.BucketCountThresholds(DEFAULT_BUCKET_COUNT_THRESHOLDS);

View File

@ -22,11 +22,15 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.BucketUtils; 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.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
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); IncludeExclude.Parser incExcParser = new IncludeExclude.Parser(aggregationName, StringTerms.TYPE, context);
aggParser.parse(aggregationName, parser, context, vsParser, incExcParser); 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(); TermsAggregator.BucketCountThresholds bucketCountThresholds = aggParser.getBucketCountThresholds();
if (!(order == InternalOrder.TERM_ASC || order == InternalOrder.TERM_DESC) if (!(order == InternalOrder.TERM_ASC || order == InternalOrder.TERM_DESC)
&& bucketCountThresholds.getShardSize() == aggParser.getDefaultBucketCountThresholds().getShardSize()) { && 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()); 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)) { if ("_term".equals(key)) {
return asc ? InternalOrder.TERM_ASC : InternalOrder.TERM_DESC; return Order.term(asc);
} }
if ("_count".equals(key)) { 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);
} }
} }

View File

@ -54,7 +54,7 @@ public class UnmappedTerms extends InternalTerms {
UnmappedTerms() {} // for serialization 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); super(name, order, requiredSize, shardSize, minDocCount, BUCKETS, false, 0);
} }

View File

@ -36,9 +36,8 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.io.IOException;
import java.util.Iterator; import java.util.*;
import java.util.List;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; 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 int NUM_DOCS = 5; // TODO: randomize the size?
private static final String SINGLE_VALUED_FIELD_NAME = "d_value"; private static final String SINGLE_VALUED_FIELD_NAME = "d_value";
private static final String MULTI_VALUED_FIELD_NAME = "d_values"; private static final String MULTI_VALUED_FIELD_NAME = "d_values";
private static HashMap<Double, Map<String, Object>> expectedMultiSortBuckets;
public void setupSuiteScopeCluster() throws Exception { public void setupSuiteScopeCluster() throws Exception {
createIndex("idx"); createIndex("idx");
@ -88,10 +88,123 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
.field(SINGLE_VALUED_FIELD_NAME, i*2) .field(SINGLE_VALUED_FIELD_NAME, i*2)
.endObject())); .endObject()));
} }
getMultiSortDocs(builders);
indexRandom(true, builders); indexRandom(true, builders);
ensureSearchable(); 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) { private String key(Terms.Bucket bucket) {
return randomBoolean() ? bucket.getKey() : bucket.getKeyAsText().string(); 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++;
}
}
} }

View File

@ -35,9 +35,8 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.io.IOException;
import java.util.Iterator; import java.util.*;
import java.util.List;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; 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 int NUM_DOCS = 5; // TODO randomize the size?
private static final String SINGLE_VALUED_FIELD_NAME = "l_value"; private static final String SINGLE_VALUED_FIELD_NAME = "l_value";
private static final String MULTI_VALUED_FIELD_NAME = "l_values"; private static final String MULTI_VALUED_FIELD_NAME = "l_values";
private static HashMap<Long, Map<String, Object>> expectedMultiSortBuckets;
@Override @Override
public void setupSuiteScopeCluster() throws Exception { public void setupSuiteScopeCluster() throws Exception {
@ -91,10 +91,123 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
.field(SINGLE_VALUED_FIELD_NAME, i * 2) .field(SINGLE_VALUED_FIELD_NAME, i * 2)
.endObject())); .endObject()));
} }
getMultiSortDocs(builders);
indexRandom(true, builders.toArray(new IndexRequestBuilder[builders.size()])); indexRandom(true, builders.toArray(new IndexRequestBuilder[builders.size()]));
ensureSearchable(); 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) { private String key(Terms.Bucket bucket) {
return randomBoolean() ? bucket.getKey() : key(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++;
}
}
} }

View File

@ -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.avg.Avg;
import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.elasticsearch.search.aggregations.metrics.stats.Stats;
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats; 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.search.aggregations.metrics.valuecount.ValueCount;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.io.IOException;
import java.util.Iterator; import java.util.*;
import java.util.List;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; 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 SINGLE_VALUED_FIELD_NAME = "s_value";
private static final String MULTI_VALUED_FIELD_NAME = "s_values"; private static final String MULTI_VALUED_FIELD_NAME = "s_values";
private static Map<String, Map<String, Object>> expectedMultiSortBuckets;
public static String randomExecutionHint() { public static String randomExecutionHint() {
return randomBoolean() ? null : randomFrom(ExecutionMode.values()).toString(); return randomBoolean() ? null : randomFrom(ExecutionMode.values()).toString();
@ -79,6 +80,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
.endObject())); .endObject()));
} }
getMultiSortDocs(builders);
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
builders.add(client().prepareIndex("idx", "high_card_type").setSource(jsonBuilder() builders.add(client().prepareIndex("idx", "high_card_type").setSource(jsonBuilder()
.startObject() .startObject()
@ -99,6 +102,116 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
ensureSearchable(); 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) { private String key(Terms.Bucket bucket) {
return randomBoolean() ? bucket.getKey() : bucket.getKeyAsText().string(); 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 @Test
public void indexMetaField() throws Exception { public void indexMetaField() throws Exception {
SearchResponse response = client().prepareSearch("idx", "empty_bucket_idx").setTypes("type") SearchResponse response = client().prepareSearch("idx", "empty_bucket_idx").setTypes("type")