Browse Source

Compound order for histogram aggregations. (#22343)

This commit adds support for histogram and date_histogram agg compound order by refactoring and reusing terms agg order code. The major change is that the Terms.Order and Histogram.Order classes have been replaced/refactored into a new class BucketOrder. This is a breaking change for the Java Transport API. For backward compatibility with previous ES versions the (date)histogram compound order will use the first order. Also the _term and _time aggregation order keys have been deprecated; replaced by _key.

Relates to #20003: now that all these aggregations use the same order code, it should be easier to move validation to parse time (as a follow up PR).

Relates to #14771: histogram and date_histogram aggregation order will now be validated at reduce time.

Closes #23613: if a single BucketOrder that is not a tie-breaker is added with the Java Transport API, it will be converted into a CompoundOrder with a tie-breaker.
qwerty4030 8 years ago
parent
commit
e7d352b489
86 changed files with 2260 additions and 1431 deletions
  1. 127 0
      core/src/main/java/org/elasticsearch/search/aggregations/BucketOrder.java
  2. 595 0
      core/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java
  3. 41 0
      core/src/main/java/org/elasticsearch/search/aggregations/KeyComparable.java
  4. 0 27
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/MultiBucketsAggregation.java
  5. 30 48
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java
  6. 6 4
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java
  7. 3 2
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java
  8. 0 81
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/Histogram.java
  9. 30 48
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java
  10. 6 4
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java
  11. 4 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java
  12. 17 9
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java
  13. 17 9
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java
  14. 0 135
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalOrder.java
  15. 2 1
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java
  16. 4 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java
  17. 2 1
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java
  18. 12 11
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java
  19. 3 2
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedTerms.java
  20. 0 385
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalOrder.java
  21. 10 7
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java
  22. 4 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTerms.java
  23. 4 2
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java
  24. 4 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTerms.java
  25. 4 2
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java
  26. 0 86
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/Terms.java
  27. 20 56
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java
  28. 64 5
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java
  29. 11 9
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java
  30. 2 1
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/UnmappedTerms.java
  31. 15 14
      core/src/main/java/org/elasticsearch/search/aggregations/support/AggregationPath.java
  32. 13 0
      core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java
  33. 9 0
      core/src/test/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilderTests.java
  34. 158 0
      core/src/test/java/org/elasticsearch/search/aggregations/InternalOrderTests.java
  35. 194 10
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java
  36. 28 13
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java
  37. 2 1
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/DiversifiedSamplerIT.java
  38. 59 30
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsIT.java
  39. 192 20
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramIT.java
  40. 36 21
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java
  41. 53 26
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsIT.java
  42. 41 40
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/MinDocCountIT.java
  43. 3 2
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/NaNSortingIT.java
  44. 3 2
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedIT.java
  45. 2 1
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/SamplerIT.java
  46. 16 15
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardSizeTermsIT.java
  47. 74 37
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsIT.java
  48. 39 39
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/TermsDocCountErrorIT.java
  49. 3 2
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/TermsShardMinDocCountIT.java
  50. 13 9
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/TermsTests.java
  51. 2 1
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java
  52. 2 1
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java
  53. 2 1
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsTests.java
  54. 2 1
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsTests.java
  55. 2 1
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsTests.java
  56. 3 2
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java
  57. 2 2
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgIT.java
  58. 3 2
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsIT.java
  59. 3 3
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksIT.java
  60. 3 3
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesIT.java
  61. 2 2
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxIT.java
  62. 2 2
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/MinIT.java
  63. 2 2
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/StatsIT.java
  64. 2 2
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/SumIT.java
  65. 3 3
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentileRanksIT.java
  66. 3 3
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentilesIT.java
  67. 4 4
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java
  68. 5 5
      core/src/test/java/org/elasticsearch/search/aggregations/pipeline/AvgBucketIT.java
  69. 6 6
      core/src/test/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java
  70. 5 5
      core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketIT.java
  71. 5 5
      core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MinBucketIT.java
  72. 7 6
      core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketIT.java
  73. 5 5
      core/src/test/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketIT.java
  74. 5 5
      core/src/test/java/org/elasticsearch/search/aggregations/pipeline/SumBucketIT.java
  75. 4 0
      docs/java-api/aggregations/bucket/datehistogram-aggregation.asciidoc
  76. 4 0
      docs/java-api/aggregations/bucket/histogram-aggregation.asciidoc
  77. 26 4
      docs/java-api/aggregations/bucket/terms-aggregation.asciidoc
  78. 7 0
      docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc
  79. 2 114
      docs/reference/aggregations/bucket/histogram-aggregation.asciidoc
  80. 2 1
      docs/reference/aggregations/bucket/terms-aggregation.asciidoc
  81. 9 1
      docs/reference/migration/migrate_6_0/java.asciidoc
  82. 63 0
      rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/10_histogram.yaml
  83. 54 0
      rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/20_terms.yaml
  84. 2 2
      test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java
  85. 3 3
      test/framework/src/main/java/org/elasticsearch/test/AbstractWireSerializingTestCase.java
  86. 29 5
      test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java

+ 127 - 0
core/src/main/java/org/elasticsearch/search/aggregations/BucketOrder.java

@@ -0,0 +1,127 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.search.aggregations;
+
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.xcontent.ToXContentObject;
+import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
+import org.elasticsearch.search.aggregations.support.AggregationPath;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * {@link Bucket} Ordering strategy.
+ */
+public abstract class BucketOrder implements ToXContentObject, Writeable {
+
+    /**
+     * Creates a bucket ordering strategy that sorts buckets by their document counts (ascending or descending).
+     *
+     * @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
+     */
+    public static BucketOrder count(boolean asc) {
+        return asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC;
+    }
+
+    /**
+     * Creates a bucket ordering strategy that sorts buckets by their keys (ascending or descending). This may be
+     * used as a tie-breaker to avoid non-deterministic ordering.
+     *
+     * @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
+     */
+    public static BucketOrder key(boolean asc) {
+        return asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC;
+    }
+
+    /**
+     * Creates a bucket ordering strategy which sorts buckets based on a single-valued sub-aggregation.
+     *
+     * @param path path to the sub-aggregation to sort on.
+     * @param asc  direction to sort by: {@code true} for ascending, {@code false} for descending.
+     * @see AggregationPath
+     */
+    public static BucketOrder aggregation(String path, boolean asc) {
+        return new InternalOrder.Aggregation(path, asc);
+    }
+
+    /**
+     * Creates a bucket ordering strategy which sorts buckets based on a metric from a multi-valued sub-aggregation.
+     *
+     * @param path       path to the sub-aggregation to sort on.
+     * @param metricName name of the value of the multi-value metric to sort on.
+     * @param asc        direction to sort by: {@code true} for ascending, {@code false} for descending.
+     * @see AggregationPath
+     */
+    public static BucketOrder aggregation(String path, String metricName, boolean asc) {
+        return new InternalOrder.Aggregation(path + "." + metricName, asc);
+    }
+
+    /**
+     * Creates a bucket ordering strategy which sorts buckets based on multiple criteria. A tie-breaker may be added to
+     * avoid non-deterministic ordering.
+     *
+     * @param orders a list of {@link BucketOrder} objects to sort on, in order of priority.
+     */
+    public static BucketOrder compound(List<BucketOrder> orders) {
+        return new InternalOrder.CompoundOrder(orders);
+    }
+
+    /**
+     * Creates a bucket ordering strategy which sorts buckets based on multiple criteria. A tie-breaker may be added to
+     * avoid non-deterministic ordering.
+     *
+     * @param orders a list of {@link BucketOrder} parameters to sort on, in order of priority.
+     */
+    public static BucketOrder compound(BucketOrder... orders) {
+        return compound(Arrays.asList(orders));
+    }
+
+    /**
+     * @return A comparator for the bucket based on the given aggregator. The comparator is used in two phases:
+     * <p>
+     * - aggregation phase, where each shard builds a list of buckets to be sent to the coordinating node.
+     * In this phase, the passed in aggregator will be the aggregator that aggregates the buckets on the
+     * shard level.
+     * <p>
+     * - reduce phase, where the coordinating node gathers all the buckets from all the shards and reduces them
+     * to a final bucket list. In this case, the passed in aggregator will be {@code null}.
+     */
+    public abstract Comparator<Bucket> comparator(Aggregator aggregator);
+
+    /**
+     * @return unique internal ID used for reading/writing this order from/to a stream.
+     * @see InternalOrder.Streams
+     */
+    abstract byte id();
+
+    @Override
+    public abstract int hashCode();
+
+    @Override
+    public abstract boolean equals(Object obj);
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        InternalOrder.Streams.writeOrder(this, out);
+    }
+}

+ 595 - 0
core/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java

@@ -0,0 +1,595 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.search.aggregations;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.common.ParsingException;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.logging.DeprecationLogger;
+import org.elasticsearch.common.logging.Loggers;
+import org.elasticsearch.common.util.Comparators;
+import org.elasticsearch.common.xcontent.XContent;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
+import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
+import org.elasticsearch.search.aggregations.support.AggregationPath;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Implementations for {@link Bucket} ordering strategies.
+ */
+public class InternalOrder extends BucketOrder {
+
+    private final byte id;
+    private final String key;
+    protected final boolean asc;
+    protected final Comparator<Bucket> comparator;
+
+    /**
+     * Creates an ordering strategy that sorts {@link Bucket}s by some property.
+     *
+     * @param id         unique ID for this ordering strategy.
+     * @param key        key of the property to sort on.
+     * @param asc        direction to sort by: {@code true} for ascending, {@code false} for descending.
+     * @param comparator determines how buckets will be ordered.
+     */
+    public InternalOrder(byte id, String key, boolean asc, Comparator<Bucket> comparator) {
+        this.id = id;
+        this.key = key;
+        this.asc = asc;
+        this.comparator = comparator;
+    }
+
+    @Override
+    byte id() {
+        return id;
+    }
+
+    @Override
+    public Comparator<Bucket> comparator(Aggregator aggregator) {
+        return comparator;
+    }
+
+    @Override
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
+    }
+
+    /**
+     * Validate a bucket ordering strategy for an {@link Aggregator}.
+     *
+     * @param order      bucket ordering strategy to sort on.
+     * @param aggregator aggregator to sort.
+     * @return unmodified bucket ordering strategy.
+     * @throws AggregationExecutionException if validation fails
+     */
+    public static BucketOrder validate(BucketOrder order, Aggregator aggregator) throws AggregationExecutionException {
+        if (order instanceof CompoundOrder) {
+            for (BucketOrder innerOrder : ((CompoundOrder) order).orderElements) {
+                validate(innerOrder, aggregator);
+            }
+        } else if (order instanceof Aggregation) {
+            ((Aggregation) order).path().validate(aggregator);
+        }
+        return order;
+    }
+
+    /**
+     * {@link Bucket} ordering strategy to sort by a sub-aggregation.
+     */
+    public static class Aggregation extends InternalOrder {
+
+        static final byte ID = 0;
+
+        /**
+         * Create a new ordering strategy to sort by a sub-aggregation.
+         *
+         * @param path path to the sub-aggregation to sort on.
+         * @param asc  direction to sort by: {@code true} for ascending, {@code false} for descending.
+         * @see AggregationPath
+         */
+        Aggregation(String path, boolean asc) {
+            super(ID, path, asc, new AggregationComparator(path, asc));
+        }
+
+        /**
+         * @return parsed path to the sub-aggregation to sort on.
+         */
+        public AggregationPath path() {
+            return ((AggregationComparator) comparator).path;
+        }
+
+        @Override
+        public Comparator<Bucket> comparator(Aggregator aggregator) {
+            if (aggregator instanceof TermsAggregator) {
+                // Internal Optimization for terms aggregation to avoid constructing buckets for ordering purposes
+                return ((TermsAggregator) aggregator).bucketComparator(path(), asc);
+            }
+            return comparator;
+        }
+
+        /**
+         * {@link Bucket} ordering strategy to sort by a sub-aggregation.
+         */
+        static class AggregationComparator implements Comparator<Bucket> {
+
+            private final AggregationPath path;
+            private final boolean asc;
+
+            /**
+             * Create a new {@link Bucket} ordering strategy to sort by a sub-aggregation.
+             *
+             * @param path path to the sub-aggregation to sort on.
+             * @param asc  direction to sort by: {@code true} for ascending, {@code false} for descending.
+             * @see AggregationPath
+             */
+            AggregationComparator(String path, boolean asc) {
+                this.asc = asc;
+                this.path = AggregationPath.parse(path);
+            }
+
+            @Override
+            public int compare(Bucket b1, Bucket b2) {
+                double v1 = path.resolveValue(b1);
+                double v2 = path.resolveValue(b2);
+                return Comparators.compareDiscardNaN(v1, v2, asc);
+            }
+        }
+    }
+
+    /**
+     * {@link Bucket} ordering strategy to sort by multiple criteria.
+     */
+    public static class CompoundOrder extends BucketOrder {
+
+        static final byte ID = -1;
+
+        final List<BucketOrder> orderElements;
+
+        /**
+         * Create a new ordering strategy to sort by multiple criteria. A tie-breaker may be added to avoid
+         * non-deterministic ordering.
+         *
+         * @param compoundOrder a list of {@link BucketOrder}s to sort on, in order of priority.
+         */
+        CompoundOrder(List<BucketOrder> compoundOrder) {
+            this(compoundOrder, true);
+        }
+
+        /**
+         * Create a new ordering strategy to sort by multiple criteria.
+         *
+         * @param compoundOrder    a list of {@link BucketOrder}s to sort on, in order of priority.
+         * @param absoluteOrdering {@code true} to add a tie-breaker to avoid non-deterministic ordering if needed,
+         *                         {@code false} otherwise.
+         */
+        CompoundOrder(List<BucketOrder> compoundOrder, boolean absoluteOrdering) {
+            this.orderElements = new LinkedList<>(compoundOrder);
+            BucketOrder lastElement = null;
+            for (BucketOrder order : orderElements) {
+                if (order instanceof CompoundOrder) {
+                    throw new IllegalArgumentException("nested compound order not supported");
+                }
+                lastElement = order;
+            }
+            if (absoluteOrdering && isKeyOrder(lastElement) == false) {
+                // add key order ascending as a tie-breaker to avoid non-deterministic ordering
+                // if all user provided comparators return 0.
+                this.orderElements.add(KEY_ASC);
+            }
+        }
+
+        @Override
+        byte id() {
+            return ID;
+        }
+
+        /**
+         * @return unmodifiable list of {@link BucketOrder}s to sort on.
+         */
+        public List<BucketOrder> orderElements() {
+            return Collections.unmodifiableList(orderElements);
+        }
+
+        @Override
+        public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+            builder.startArray();
+            for (BucketOrder order : orderElements) {
+                order.toXContent(builder, params);
+            }
+            return builder.endArray();
+        }
+
+        @Override
+        public Comparator<Bucket> comparator(Aggregator aggregator) {
+            return new CompoundOrderComparator(orderElements, aggregator);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(orderElements);
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == null) {
+                return false;
+            }
+            if (getClass() != obj.getClass()) {
+                return false;
+            }
+            CompoundOrder other = (CompoundOrder) obj;
+            return Objects.equals(orderElements, other.orderElements);
+        }
+
+        /**
+         * {@code Comparator} for sorting buckets by multiple criteria.
+         */
+        static class CompoundOrderComparator implements Comparator<Bucket> {
+
+            private List<BucketOrder> compoundOrder;
+            private Aggregator aggregator;
+
+            /**
+             * Create a new {@code Comparator} for sorting buckets by multiple criteria.
+             *
+             * @param compoundOrder a list of {@link BucketOrder}s to sort on, in order of priority.
+             * @param aggregator    {@link BucketOrder#comparator(Aggregator)}
+             */
+            CompoundOrderComparator(List<BucketOrder> compoundOrder, Aggregator aggregator) {
+                this.compoundOrder = compoundOrder;
+                this.aggregator = aggregator;
+            }
+
+            @Override
+            public int compare(Bucket b1, Bucket b2) {
+                int result = 0;
+                for (Iterator<BucketOrder> itr = compoundOrder.iterator(); itr.hasNext() && result == 0; ) {
+                    result = itr.next().comparator(aggregator).compare(b1, b2);
+                }
+                return result;
+            }
+        }
+    }
+
+    private static final byte COUNT_DESC_ID = 1;
+    private static final byte COUNT_ASC_ID = 2;
+    private static final byte KEY_DESC_ID = 3;
+    private static final byte KEY_ASC_ID = 4;
+
+    /**
+     * Order by the (higher) count of each bucket.
+     */
+    static final InternalOrder COUNT_DESC = new InternalOrder(COUNT_DESC_ID, "_count", false, comparingCounts().reversed());
+
+    /**
+     * Order by the (lower) count of each bucket.
+     */
+    static final InternalOrder COUNT_ASC = new InternalOrder(COUNT_ASC_ID, "_count", true, comparingCounts());
+
+    /**
+     * Order by the key of each bucket descending.
+     */
+    static final InternalOrder KEY_DESC = new InternalOrder(KEY_DESC_ID, "_key", false, comparingKeys().reversed());
+
+    /**
+     * Order by the key of each bucket ascending.
+     */
+    static final InternalOrder KEY_ASC = new InternalOrder(KEY_ASC_ID, "_key", true, comparingKeys());
+
+    /**
+     * @return compare by {@link Bucket#getDocCount()}.
+     */
+    private static Comparator<Bucket> comparingCounts() {
+        return Comparator.comparingLong(Bucket::getDocCount);
+    }
+
+    /**
+     * @return compare by {@link Bucket#getKey()} from the appropriate implementation.
+     */
+    @SuppressWarnings("unchecked")
+    private static Comparator<Bucket> comparingKeys() {
+        return (b1, b2) -> {
+            if (b1 instanceof KeyComparable) {
+                return ((KeyComparable) b1).compareKey(b2);
+            }
+            throw new IllegalStateException("Unexpected order bucket class [" + b1.getClass() + "]");
+        };
+    }
+
+    /**
+     * Determine if the ordering strategy is sorting on bucket count descending.
+     *
+     * @param order bucket ordering strategy to check.
+     * @return {@code true} if the ordering strategy is sorting on bucket count descending, {@code false} otherwise.
+     */
+    public static boolean isCountDesc(BucketOrder order) {
+        return isOrder(order, COUNT_DESC);
+    }
+
+    /**
+     * Determine if the ordering strategy is sorting on bucket key (ascending or descending).
+     *
+     * @param order bucket ordering strategy to check.
+     * @return {@code true} if the ordering strategy is sorting on bucket key, {@code false} otherwise.
+     */
+    public static boolean isKeyOrder(BucketOrder order) {
+        return isOrder(order, KEY_ASC) || isOrder(order, KEY_DESC);
+    }
+
+    /**
+     * Determine if the ordering strategy is sorting on bucket key ascending.
+     *
+     * @param order bucket ordering strategy to check.
+     * @return {@code true} if the ordering strategy is sorting on bucket key ascending, {@code false} otherwise.
+     */
+    public static boolean isKeyAsc(BucketOrder order) {
+        return isOrder(order, KEY_ASC);
+    }
+
+    /**
+     * Determine if the ordering strategy is sorting on bucket key descending.
+     *
+     * @param order bucket ordering strategy to check.
+     * @return {@code true} if the ordering strategy is sorting on bucket key descending, {@code false} otherwise.
+     */
+    public static boolean isKeyDesc(BucketOrder order) {
+        return isOrder(order, KEY_DESC);
+    }
+
+    /**
+     * Determine if the ordering strategy matches the expected one.
+     *
+     * @param order    bucket ordering strategy to check. If this is a {@link CompoundOrder} the first element will be
+     *                 check instead.
+     * @param expected expected  bucket ordering strategy.
+     * @return {@code true} if the order matches, {@code false} otherwise.
+     */
+    private static boolean isOrder(BucketOrder order, BucketOrder expected) {
+        if (order == expected) {
+            return true;
+        } else if (order instanceof CompoundOrder) {
+            // check if its a compound order with the first element that matches
+            List<BucketOrder> orders = ((CompoundOrder) order).orderElements;
+            if (orders.size() >= 1) {
+                return isOrder(orders.get(0), expected);
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Contains logic for reading/writing {@link BucketOrder} from/to streams.
+     */
+    public static class Streams {
+
+        /**
+         * Read a {@link BucketOrder} from a {@link StreamInput}.
+         *
+         * @param in stream with order data to read.
+         * @return order read from the stream
+         * @throws IOException on error reading from the stream.
+         */
+        public static BucketOrder readOrder(StreamInput in) throws IOException {
+            byte id = in.readByte();
+            switch (id) {
+                case COUNT_DESC_ID: return COUNT_DESC;
+                case COUNT_ASC_ID: return COUNT_ASC;
+                case KEY_DESC_ID: return KEY_DESC;
+                case KEY_ASC_ID: return KEY_ASC;
+                case Aggregation.ID:
+                    boolean asc = in.readBoolean();
+                    String key = in.readString();
+                    return new Aggregation(key, asc);
+                case CompoundOrder.ID:
+                    int size = in.readVInt();
+                    List<BucketOrder> compoundOrder = new ArrayList<>(size);
+                    for (int i = 0; i < size; i++) {
+                        compoundOrder.add(Streams.readOrder(in));
+                    }
+                    return new CompoundOrder(compoundOrder, false);
+                default:
+                    throw new RuntimeException("unknown order id [" + id + "]");
+            }
+        }
+
+        /**
+         * ONLY FOR HISTOGRAM ORDER: Backwards compatibility logic to read a {@link BucketOrder} from a {@link StreamInput}.
+         *
+         * @param in           stream with order data to read.
+         * @param bwcOrderFlag {@code true} to check {@code in.readBoolean()} in the backwards compat logic before reading
+         *                     the order. {@code false} to skip this flag (order always present).
+         * @return order read from the stream
+         * @throws IOException on error reading from the stream.
+         */
+        public static BucketOrder readHistogramOrder(StreamInput in, boolean bwcOrderFlag) throws IOException {
+            if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha2_UNRELEASED)) {
+                return Streams.readOrder(in);
+            } else { // backwards compat logic
+                if (bwcOrderFlag == false || in.readBoolean()) {
+                    // translate the old histogram order IDs to the new order objects
+                    byte id = in.readByte();
+                    switch (id) {
+                        case 1: return KEY_ASC;
+                        case 2: return KEY_DESC;
+                        case 3: return COUNT_ASC;
+                        case 4: return COUNT_DESC;
+                        case 0: // aggregation order stream logic is backwards compatible
+                            boolean asc = in.readBoolean();
+                            String key = in.readString();
+                            return new Aggregation(key, asc);
+                        default: // not expecting compound order ID
+                            throw new RuntimeException("unknown histogram order id [" + id + "]");
+                    }
+                } else { // default to _key asc if no order specified
+                    return KEY_ASC;
+                }
+            }
+        }
+
+        /**
+         * Write a {@link BucketOrder} to a {@link StreamOutput}.
+         *
+         * @param order order to write to the stream.
+         * @param out   stream to write the order to.
+         * @throws IOException on error writing to the stream.
+         */
+        public static void writeOrder(BucketOrder order, StreamOutput out) throws IOException {
+            out.writeByte(order.id());
+            if (order instanceof Aggregation) {
+                Aggregation aggregationOrder = (Aggregation) order;
+                out.writeBoolean(aggregationOrder.asc);
+                out.writeString(aggregationOrder.path().toString());
+            } else if (order instanceof CompoundOrder) {
+                CompoundOrder compoundOrder = (CompoundOrder) order;
+                out.writeVInt(compoundOrder.orderElements.size());
+                for (BucketOrder innerOrder : compoundOrder.orderElements) {
+                    innerOrder.writeTo(out);
+                }
+            }
+        }
+
+        /**
+         * ONLY FOR HISTOGRAM ORDER: Backwards compatibility logic to write a {@link BucketOrder} to a stream.
+         *
+         * @param order        order to write to the stream.
+         * @param out          stream to write the order to.
+         * @param bwcOrderFlag {@code true} to always {@code out.writeBoolean(true)} for the backwards compat logic before
+         *                     writing the order. {@code false} to skip this flag.
+         * @throws IOException on error writing to the stream.
+         */
+        public static void writeHistogramOrder(BucketOrder order, StreamOutput out, boolean bwcOrderFlag) throws IOException {
+            if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha2_UNRELEASED)) {
+                order.writeTo(out);
+            } else { // backwards compat logic
+                if(bwcOrderFlag) { // need to add flag that determines if order exists
+                    out.writeBoolean(true); // order always exists
+                }
+                if (order instanceof CompoundOrder) {
+                    // older versions do not support histogram compound order; the best we can do here is use the first order.
+                    order = ((CompoundOrder) order).orderElements.get(0);
+                }
+                if (order instanceof Aggregation) {
+                    // aggregation order stream logic is backwards compatible
+                    order.writeTo(out);
+                } else {
+                    // convert the new order IDs to the old histogram order IDs.
+                    byte id;
+                    switch (order.id()) {
+                        case COUNT_DESC_ID: id = 4; break;
+                        case COUNT_ASC_ID: id = 3; break;
+                        case KEY_DESC_ID: id = 2; break;
+                        case KEY_ASC_ID: id = 1; break;
+                        default: throw new RuntimeException("unknown order id [" + order.id() + "]");
+                    }
+                    out.writeByte(id);
+                }
+            }
+        }
+    }
+
+    /**
+     * Contains logic for parsing a {@link BucketOrder} from a {@link XContentParser}.
+     */
+    public static class Parser {
+
+        private static final DeprecationLogger DEPRECATION_LOGGER =
+            new DeprecationLogger(Loggers.getLogger(Parser.class));
+
+        /**
+         * Parse a {@link BucketOrder} from {@link XContent}.
+         *
+         * @param parser  for parsing {@link XContent} that contains the order.
+         * @param context parsing context.
+         * @return bucket ordering strategy
+         * @throws IOException on error a {@link XContent} parsing error.
+         */
+        public static BucketOrder parseOrderParam(XContentParser parser, QueryParseContext context) throws IOException {
+            XContentParser.Token token;
+            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 ParsingException(parser.getTokenLocation(),
+                            "Unknown order direction [" + dir + "]");
+                    }
+                } else {
+                    throw new ParsingException(parser.getTokenLocation(),
+                        "Unexpected token [" + token + "] for [order]");
+                }
+            }
+            if (orderKey == null) {
+                throw new ParsingException(parser.getTokenLocation(),
+                    "Must specify at least one field for [order]");
+            }
+            // _term and _time order deprecated in 6.0; replaced by _key
+            if ("_term".equals(orderKey) || "_time".equals(orderKey)) {
+                DEPRECATION_LOGGER.deprecated("Deprecated aggregation order key [{}] used, replaced by [_key]", orderKey);
+            }
+            switch (orderKey) {
+                case "_term":
+                case "_time":
+                case "_key":
+                    return orderAsc ? KEY_ASC : KEY_DESC;
+                case "_count":
+                    return orderAsc ? COUNT_ASC : COUNT_DESC;
+                default: // assume all other orders are sorting on a sub-aggregation. Validation occurs later.
+                    return aggregation(orderKey, orderAsc);
+            }
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(id, key, asc);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        InternalOrder other = (InternalOrder) obj;
+        return Objects.equals(id, other.id)
+            && Objects.equals(key, other.key)
+            && Objects.equals(asc, other.asc);
+    }
+}

+ 41 - 0
core/src/main/java/org/elasticsearch/search/aggregations/KeyComparable.java

@@ -0,0 +1,41 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.search.aggregations;
+
+import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
+
+/**
+ * Defines behavior for comparing {@link Bucket#getKey() bucket keys} to imposes a total ordering
+ * of buckets of the same type.
+ *
+ * @param <T> {@link Bucket} of the same type that also implements {@link KeyComparable}.
+ * @see BucketOrder#key(boolean)
+ */
+public interface KeyComparable<T extends Bucket & KeyComparable<T>> {
+
+    /**
+     * Compare this {@link Bucket}s {@link Bucket#getKey() key} with another bucket.
+     *
+     * @param other the bucket that contains the key to compare to.
+     * @return a negative integer, zero, or a positive integer as this buckets key
+     * is less than, equal to, or greater than the other buckets key.
+     * @see Comparable#compareTo(Object)
+     */
+    int compareKey(T other);
+}

+ 0 - 27
core/src/main/java/org/elasticsearch/search/aggregations/bucket/MultiBucketsAggregation.java

@@ -19,12 +19,10 @@
 
 package org.elasticsearch.search.aggregations.bucket;
 
-import org.elasticsearch.common.util.Comparators;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.search.aggregations.Aggregation;
 import org.elasticsearch.search.aggregations.Aggregations;
 import org.elasticsearch.search.aggregations.HasAggregations;
-import org.elasticsearch.search.aggregations.support.AggregationPath;
 
 import java.util.List;
 
@@ -58,31 +56,6 @@ public interface MultiBucketsAggregation extends Aggregation {
         @Override
         Aggregations getAggregations();
 
-        class SubAggregationComparator<B extends Bucket> implements java.util.Comparator<B> {
-
-            private final AggregationPath path;
-            private final boolean asc;
-
-            public SubAggregationComparator(String expression, boolean asc) {
-                this.asc = asc;
-                this.path = AggregationPath.parse(expression);
-            }
-
-            public boolean asc() {
-                return asc;
-            }
-
-            public AggregationPath path() {
-                return path;
-            }
-
-            @Override
-            public int compare(B b1, B b2) {
-                double v1 = path.resolveValue(b1);
-                double v2 = path.resolveValue(b2);
-                return Comparators.compareDiscardNaN(v1, v2, asc);
-            }
-        }
     }
 
     /**

+ 30 - 48
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java

@@ -19,7 +19,6 @@
 
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
-import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.rounding.DateTimeUnit;
@@ -28,10 +27,12 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.common.xcontent.XContentParser.Token;
 import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
+import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
 import org.elasticsearch.search.aggregations.support.ValueType;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
@@ -44,6 +45,7 @@ import org.elasticsearch.search.internal.SearchContext;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
@@ -113,8 +115,8 @@ public class DateHistogramAggregationBuilder
         PARSER.declareField(DateHistogramAggregationBuilder::extendedBounds, parser -> ExtendedBounds.PARSER.apply(parser, null),
                 ExtendedBounds.EXTENDED_BOUNDS_FIELD, ObjectParser.ValueType.OBJECT);
 
-        PARSER.declareField(DateHistogramAggregationBuilder::order, DateHistogramAggregationBuilder::parseOrder,
-                Histogram.ORDER_FIELD, ObjectParser.ValueType.OBJECT);
+        PARSER.declareObjectArray(DateHistogramAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
+                Histogram.ORDER_FIELD);
     }
 
     public static DateHistogramAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
@@ -125,7 +127,7 @@ public class DateHistogramAggregationBuilder
     private DateHistogramInterval dateHistogramInterval;
     private long offset = 0;
     private ExtendedBounds extendedBounds;
-    private InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
+    private BucketOrder order = BucketOrder.key(true);
     private boolean keyed = false;
     private long minDocCount = 0;
 
@@ -137,9 +139,7 @@ public class DateHistogramAggregationBuilder
     /** Read from a stream, for internal use only. */
     public DateHistogramAggregationBuilder(StreamInput in) throws IOException {
         super(in, ValuesSourceType.NUMERIC, ValueType.DATE);
-        if (in.readBoolean()) {
-            order = InternalOrder.Streams.readOrder(in);
-        }
+        order = InternalOrder.Streams.readHistogramOrder(in, true);
         keyed = in.readBoolean();
         minDocCount = in.readVLong();
         interval = in.readLong();
@@ -150,11 +150,7 @@ public class DateHistogramAggregationBuilder
 
     @Override
     protected void innerWriteTo(StreamOutput out) throws IOException {
-        boolean hasOrder = order != null;
-        out.writeBoolean(hasOrder);
-        if (hasOrder) {
-            InternalOrder.Streams.writeOrder(order, out);
-        }
+        InternalOrder.Streams.writeHistogramOrder(order, out, true);
         out.writeBoolean(keyed);
         out.writeVLong(minDocCount);
         out.writeLong(interval);
@@ -244,17 +240,34 @@ public class DateHistogramAggregationBuilder
     }
 
     /** Return the order to use to sort buckets of this histogram. */
-    public Histogram.Order order() {
+    public BucketOrder order() {
         return order;
     }
 
     /** Set a new order on this builder and return the builder so that calls
-     *  can be chained. */
-    public DateHistogramAggregationBuilder order(Histogram.Order order) {
+     *  can be chained. A tie-breaker may be added to avoid non-deterministic ordering. */
+    public DateHistogramAggregationBuilder order(BucketOrder order) {
         if (order == null) {
             throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
         }
-        this.order = (InternalOrder) order;
+        if(order instanceof CompoundOrder || InternalOrder.isKeyOrder(order)) {
+            this.order = order; // if order already contains a tie-breaker we are good to go
+        } else { // otherwise add a tie-breaker by using a compound order
+            this.order = BucketOrder.compound(order);
+        }
+        return this;
+    }
+
+    /**
+     * Sets the order in which the buckets will be returned. A tie-breaker may be added to avoid non-deterministic
+     * ordering.
+     */
+    public DateHistogramAggregationBuilder order(List<BucketOrder> orders) {
+        if (orders == null) {
+            throw new IllegalArgumentException("[orders] must not be null: [" + name + "]");
+        }
+        // if the list only contains one order use that to avoid inconsistent xcontent
+        order(orders.size() > 1 ? BucketOrder.compound(orders) : orders.get(0));
         return this;
     }
 
@@ -370,35 +383,4 @@ public class DateHistogramAggregationBuilder
                 && Objects.equals(offset, other.offset)
                 && Objects.equals(extendedBounds, other.extendedBounds);
     }
-
-    // similar to the parsing oh histogram orders, but also accepts _time as an alias for _key
-    private static InternalOrder parseOrder(XContentParser parser, QueryParseContext context) throws IOException {
-        InternalOrder order = null;
-        Token token;
-        String currentFieldName = null;
-        while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
-            if (token == XContentParser.Token.FIELD_NAME) {
-                currentFieldName = parser.currentName();
-            } else if (token == XContentParser.Token.VALUE_STRING) {
-                String dir = parser.text();
-                boolean asc = "asc".equals(dir);
-                if (!asc && !"desc".equals(dir)) {
-                    throw new ParsingException(parser.getTokenLocation(), "Unknown order direction: [" + dir
-                            + "]. Should be either [asc] or [desc]");
-                }
-                order = resolveOrder(currentFieldName, asc);
-            }
-        }
-        return order;
-    }
-
-    static InternalOrder resolveOrder(String key, boolean asc) {
-        if ("_key".equals(key) || "_time".equals(key)) {
-            return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC);
-        }
-        if ("_count".equals(key)) {
-            return (InternalOrder) (asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC);
-        }
-        return new InternalOrder.Aggregation(key, asc);
-    }
 }

+ 6 - 4
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java

@@ -33,6 +33,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
 import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.internal.SearchContext;
 
@@ -53,7 +55,7 @@ class DateHistogramAggregator extends BucketsAggregator {
     private final ValuesSource.Numeric valuesSource;
     private final DocValueFormat formatter;
     private final Rounding rounding;
-    private final InternalOrder order;
+    private final BucketOrder order;
     private final boolean keyed;
 
     private final long minDocCount;
@@ -62,7 +64,7 @@ class DateHistogramAggregator extends BucketsAggregator {
     private final LongHash bucketOrds;
     private long offset;
 
-    DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, long offset, InternalOrder order,
+    DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, long offset, BucketOrder order,
             boolean keyed,
             long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource,
             DocValueFormat formatter, SearchContext aggregationContext,
@@ -71,7 +73,7 @@ class DateHistogramAggregator extends BucketsAggregator {
         super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
         this.rounding = rounding;
         this.offset = offset;
-        this.order = order;
+        this.order = InternalOrder.validate(order, this);;
         this.keyed = keyed;
         this.minDocCount = minDocCount;
         this.extendedBounds = extendedBounds;
@@ -131,7 +133,7 @@ class DateHistogramAggregator extends BucketsAggregator {
         }
 
         // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order
-        CollectionUtil.introSort(buckets, InternalOrder.KEY_ASC.comparator());
+        CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this));
 
         // value source will be null for unmapped fields
         InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0

+ 3 - 2
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java

@@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
@@ -40,14 +41,14 @@ public final class DateHistogramAggregatorFactory
     private final DateHistogramInterval dateHistogramInterval;
     private final long interval;
     private final long offset;
-    private final InternalOrder order;
+    private final BucketOrder order;
     private final boolean keyed;
     private final long minDocCount;
     private final ExtendedBounds extendedBounds;
     private Rounding rounding;
 
     public DateHistogramAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, long interval,
-            DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount,
+            DateHistogramInterval dateHistogramInterval, long offset, BucketOrder order, boolean keyed, long minDocCount,
             Rounding rounding, ExtendedBounds extendedBounds, SearchContext context, AggregatorFactory<?> parent,
             AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
         super(name, config, context, parent, subFactoriesBuilder, metaData);

+ 0 - 81
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/Histogram.java

@@ -19,10 +19,8 @@
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
 import org.elasticsearch.common.ParseField;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
 
-import java.util.Comparator;
 import java.util.List;
 
 /**
@@ -50,83 +48,4 @@ public interface Histogram extends MultiBucketsAggregation {
     @Override
     List<? extends Bucket> getBuckets();
 
-    /**
-     * A strategy defining the order in which the buckets in this histogram are ordered.
-     */
-    abstract class Order implements ToXContent {
-
-        private static int compareKey(Histogram.Bucket b1, Histogram.Bucket b2) {
-            if (b1 instanceof InternalHistogram.Bucket) {
-                return Double.compare(((InternalHistogram.Bucket) b1).key, ((InternalHistogram.Bucket) b2).key);
-            } else if (b1 instanceof InternalDateHistogram.Bucket) {
-                return Long.compare(((InternalDateHistogram.Bucket) b1).key, ((InternalDateHistogram.Bucket) b2).key);
-            } else {
-                throw new IllegalStateException("Unexpected impl: " + b1.getClass());
-            }
-        }
-
-        public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator<Histogram.Bucket>() {
-            @Override
-            public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
-                return compareKey(b1, b2);
-            }
-        });
-
-        public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator<Histogram.Bucket>() {
-            @Override
-            public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
-                return compareKey(b2, b1);
-            }
-        });
-
-        public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator<Histogram.Bucket>() {
-            @Override
-            public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
-                int cmp = Long.compare(b1.getDocCount(), b2.getDocCount());
-                if (cmp == 0) {
-                    cmp = compareKey(b1, b2);
-                }
-                return cmp;
-            }
-        });
-
-
-        public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator<Histogram.Bucket>() {
-            @Override
-            public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
-                int cmp = Long.compare(b2.getDocCount(), b1.getDocCount());
-                if (cmp == 0) {
-                    cmp = compareKey(b1, b2);
-                }
-                return cmp;
-            }
-        });
-
-        /**
-         * Creates a bucket ordering strategy that sorts buckets based on a single-valued calc sug-aggregation
-         *
-         * @param path the name of the aggregation
-         * @param asc             The direction of the order (ascending or descending)
-         */
-        public static Order aggregation(String path, boolean asc) {
-            return new InternalOrder.Aggregation(path, asc);
-        }
-
-        /**
-         * Creates a bucket ordering strategy that sorts buckets based on a multi-valued calc sug-aggregation
-         *
-         * @param aggregationName the name of the aggregation
-         * @param valueName       The name of the value of the multi-value get by which the sorting will be applied
-         * @param asc             The direction of the order (ascending or descending)
-         */
-        public static Order aggregation(String aggregationName, String valueName, boolean asc) {
-            return new InternalOrder.Aggregation(aggregationName + "." + valueName, asc);
-        }
-
-        /**
-         * @return The bucket comparator by which the order will be applied.
-         */
-        abstract Comparator<Histogram.Bucket> comparator();
-
-    }
 }

+ 30 - 48
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java

@@ -20,16 +20,16 @@
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
 import org.elasticsearch.common.ParseField;
-import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.common.xcontent.XContentParser.Token;
 import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
+import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
 import org.elasticsearch.search.aggregations.support.ValueType;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
@@ -41,6 +41,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceType;
 import org.elasticsearch.search.internal.SearchContext;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Objects;
 
 /**
@@ -75,8 +76,8 @@ public class HistogramAggregationBuilder
             histogram.extendedBounds(extendedBounds[0], extendedBounds[1]);
         }, parser -> EXTENDED_BOUNDS_PARSER.apply(parser, null), ExtendedBounds.EXTENDED_BOUNDS_FIELD, ObjectParser.ValueType.OBJECT);
 
-        PARSER.declareField(HistogramAggregationBuilder::order, HistogramAggregationBuilder::parseOrder,
-                Histogram.ORDER_FIELD, ObjectParser.ValueType.OBJECT);
+        PARSER.declareObjectArray(HistogramAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
+            Histogram.ORDER_FIELD);
     }
 
     public static HistogramAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
@@ -87,7 +88,7 @@ public class HistogramAggregationBuilder
     private double offset = 0;
     private double minBound = Double.POSITIVE_INFINITY;
     private double maxBound = Double.NEGATIVE_INFINITY;
-    private InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
+    private BucketOrder order = BucketOrder.key(true);
     private boolean keyed = false;
     private long minDocCount = 0;
 
@@ -99,9 +100,7 @@ public class HistogramAggregationBuilder
     /** Read from a stream, for internal use only. */
     public HistogramAggregationBuilder(StreamInput in) throws IOException {
         super(in, ValuesSourceType.NUMERIC, ValueType.DOUBLE);
-        if (in.readBoolean()) {
-            order = InternalOrder.Streams.readOrder(in);
-        }
+        order = InternalOrder.Streams.readHistogramOrder(in, true);
         keyed = in.readBoolean();
         minDocCount = in.readVLong();
         interval = in.readDouble();
@@ -112,11 +111,7 @@ public class HistogramAggregationBuilder
 
     @Override
     protected void innerWriteTo(StreamOutput out) throws IOException {
-        boolean hasOrder = order != null;
-        out.writeBoolean(hasOrder);
-        if (hasOrder) {
-            InternalOrder.Streams.writeOrder(order, out);
-        }
+        InternalOrder.Streams.writeHistogramOrder(order, out, true);
         out.writeBoolean(keyed);
         out.writeVLong(minDocCount);
         out.writeDouble(interval);
@@ -185,17 +180,34 @@ public class HistogramAggregationBuilder
     }
 
     /** Return the order to use to sort buckets of this histogram. */
-    public Histogram.Order order() {
+    public BucketOrder order() {
         return order;
     }
 
     /** Set a new order on this builder and return the builder so that calls
-     *  can be chained. */
-    public HistogramAggregationBuilder order(Histogram.Order order) {
+     *  can be chained. A tie-breaker may be added to avoid non-deterministic ordering. */
+    public HistogramAggregationBuilder order(BucketOrder order) {
         if (order == null) {
             throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
         }
-        this.order = (InternalOrder) order;
+        if(order instanceof CompoundOrder || InternalOrder.isKeyOrder(order)) {
+            this.order = order; // if order already contains a tie-breaker we are good to go
+        } else { // otherwise add a tie-breaker by using a compound order
+            this.order = BucketOrder.compound(order);
+        }
+        return this;
+    }
+
+    /**
+     * Sets the order in which the buckets will be returned. A tie-breaker may be added to avoid non-deterministic
+     * ordering.
+     */
+    public HistogramAggregationBuilder order(List<BucketOrder> orders) {
+        if (orders == null) {
+            throw new IllegalArgumentException("[orders] must not be null: [" + name + "]");
+        }
+        // if the list only contains one order use that to avoid inconsistent xcontent
+        order(orders.size() > 1 ? BucketOrder.compound(orders) : orders.get(0));
         return this;
     }
 
@@ -286,34 +298,4 @@ public class HistogramAggregationBuilder
                 && Objects.equals(minBound, other.minBound)
                 && Objects.equals(maxBound, other.maxBound);
     }
-
-    private static InternalOrder parseOrder(XContentParser parser, QueryParseContext context) throws IOException {
-        InternalOrder order = null;
-        Token token;
-        String currentFieldName = null;
-        while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
-            if (token == XContentParser.Token.FIELD_NAME) {
-                currentFieldName = parser.currentName();
-            } else if (token == XContentParser.Token.VALUE_STRING) {
-                String dir = parser.text();
-                boolean asc = "asc".equals(dir);
-                if (!asc && !"desc".equals(dir)) {
-                    throw new ParsingException(parser.getTokenLocation(), "Unknown order direction: [" + dir
-                            + "]. Should be either [asc] or [desc]");
-                }
-                order = resolveOrder(currentFieldName, asc);
-            }
-        }
-        return order;
-    }
-
-    static InternalOrder resolveOrder(String key, boolean asc) {
-        if ("_key".equals(key)) {
-            return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC);
-        }
-        if ("_count".equals(key)) {
-            return (InternalOrder) (asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC);
-        }
-        return new InternalOrder.Aggregation(key, asc);
-    }
 }

+ 6 - 4
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java

@@ -34,6 +34,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
 import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
 import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.EmptyBucketInfo;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.internal.SearchContext;
 
@@ -54,7 +56,7 @@ class HistogramAggregator extends BucketsAggregator {
     private final ValuesSource.Numeric valuesSource;
     private final DocValueFormat formatter;
     private final double interval, offset;
-    private final InternalOrder order;
+    private final BucketOrder order;
     private final boolean keyed;
     private final long minDocCount;
     private final double minBound, maxBound;
@@ -62,7 +64,7 @@ class HistogramAggregator extends BucketsAggregator {
     private final LongHash bucketOrds;
 
     HistogramAggregator(String name, AggregatorFactories factories, double interval, double offset,
-            InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
+            BucketOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
             @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter,
             SearchContext context, Aggregator parent,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@@ -73,7 +75,7 @@ class HistogramAggregator extends BucketsAggregator {
         }
         this.interval = interval;
         this.offset = offset;
-        this.order = order;
+        this.order = InternalOrder.validate(order, this);
         this.keyed = keyed;
         this.minDocCount = minDocCount;
         this.minBound = minBound;
@@ -137,7 +139,7 @@ class HistogramAggregator extends BucketsAggregator {
         }
 
         // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order
-        CollectionUtil.introSort(buckets, InternalOrder.KEY_ASC.comparator());
+        CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this));
 
         EmptyBucketInfo emptyBucketInfo = null;
         if (minDocCount == 0) {

+ 4 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java

@@ -23,6 +23,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
@@ -36,13 +37,13 @@ import java.util.Map;
 public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, HistogramAggregatorFactory> {
 
     private final double interval, offset;
-    private final InternalOrder order;
+    private final BucketOrder order;
     private final boolean keyed;
     private final long minDocCount;
     private final double minBound, maxBound;
 
     HistogramAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, double interval, double offset,
-            InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
+            BucketOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
             SearchContext context, AggregatorFactory<?> parent,
             AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
         super(name, config, context, parent, subFactoriesBuilder, metaData);
@@ -80,4 +81,4 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
             throws IOException {
         return createAggregator(null, parent, pipelineAggregators, metaData);
     }
-}
+}

+ 17 - 9
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java

@@ -31,6 +31,9 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
 import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
+import org.elasticsearch.search.aggregations.KeyComparable;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 
@@ -49,7 +52,7 @@ import java.util.Objects;
 public final class InternalDateHistogram extends InternalMultiBucketAggregation<InternalDateHistogram, InternalDateHistogram.Bucket>
         implements Histogram, HistogramFactory {
 
-    public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket {
+    public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket, KeyComparable<Bucket> {
 
         final long key;
         final long docCount;
@@ -151,6 +154,11 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
             return builder;
         }
 
+        @Override
+        public int compareKey(Bucket other) {
+            return Long.compare(key, other.key);
+        }
+
         public DocValueFormat getFormatter() {
             return format;
         }
@@ -206,14 +214,14 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
     }
 
     private final List<Bucket> buckets;
-    private final InternalOrder order;
+    private final BucketOrder order;
     private final DocValueFormat format;
     private final boolean keyed;
     private final long minDocCount;
     private final long offset;
     private final EmptyBucketInfo emptyBucketInfo;
 
-    InternalDateHistogram(String name, List<Bucket> buckets, InternalOrder order, long minDocCount, long offset,
+    InternalDateHistogram(String name, List<Bucket> buckets, BucketOrder order, long minDocCount, long offset,
             EmptyBucketInfo emptyBucketInfo,
             DocValueFormat formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData) {
@@ -233,7 +241,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
      */
     public InternalDateHistogram(StreamInput in) throws IOException {
         super(in);
-        order = InternalOrder.Streams.readOrder(in);
+        order = InternalOrder.Streams.readHistogramOrder(in, false);
         minDocCount = in.readVLong();
         if (minDocCount == 0) {
             emptyBucketInfo = new EmptyBucketInfo(in);
@@ -248,7 +256,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
 
     @Override
     protected void doWriteTo(StreamOutput out) throws IOException {
-        InternalOrder.Streams.writeOrder(order, out);
+        InternalOrder.Streams.writeHistogramOrder(order, out, false);
         out.writeVLong(minDocCount);
         if (minDocCount == 0) {
             emptyBucketInfo.writeTo(out);
@@ -416,18 +424,18 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
             addEmptyBuckets(reducedBuckets, reduceContext);
         }
 
-        if (order == InternalOrder.KEY_ASC || reduceContext.isFinalReduce() == false) {
+        if (InternalOrder.isKeyAsc(order) || reduceContext.isFinalReduce() == false) {
             // nothing to do, data are already sorted since shards return
             // sorted buckets and the merge-sort performed by reduceBuckets
             // maintains order
-        } else if (order == InternalOrder.KEY_DESC) {
+        } else if (InternalOrder.isKeyDesc(order)) {
             // we just need to reverse here...
             List<Bucket> reverse = new ArrayList<>(reducedBuckets);
             Collections.reverse(reverse);
             reducedBuckets = reverse;
         } else {
-            // sorted by sub-aggregation, need to fall back to a costly n*log(n) sort
-            CollectionUtil.introSort(reducedBuckets, order.comparator());
+            // sorted by compound order or sub-aggregation, need to fall back to a costly n*log(n) sort
+            CollectionUtil.introSort(reducedBuckets, order.comparator(null));
         }
 
         return new InternalDateHistogram(getName(), reducedBuckets, order, minDocCount, offset, emptyBucketInfo,

+ 17 - 9
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java

@@ -30,6 +30,9 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
 import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
+import org.elasticsearch.search.aggregations.KeyComparable;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -45,7 +48,7 @@ import java.util.Objects;
  */
 public final class InternalHistogram extends InternalMultiBucketAggregation<InternalHistogram, InternalHistogram.Bucket>
         implements Histogram, HistogramFactory {
-    public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket {
+    public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket, KeyComparable<Bucket> {
 
         final double key;
         final long docCount;
@@ -147,6 +150,11 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
             return builder;
         }
 
+        @Override
+        public int compareKey(Bucket other) {
+            return Double.compare(key, other.key);
+        }
+
         public DocValueFormat getFormatter() {
             return format;
         }
@@ -201,13 +209,13 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
     }
 
     private final List<Bucket> buckets;
-    private final InternalOrder order;
+    private final BucketOrder order;
     private final DocValueFormat format;
     private final boolean keyed;
     private final long minDocCount;
     private final EmptyBucketInfo emptyBucketInfo;
 
-    InternalHistogram(String name, List<Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
+    InternalHistogram(String name, List<Bucket> buckets, BucketOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
             DocValueFormat formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData) {
         super(name, pipelineAggregators, metaData);
@@ -225,7 +233,7 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
      */
     public InternalHistogram(StreamInput in) throws IOException {
         super(in);
-        order = InternalOrder.Streams.readOrder(in);
+        order = InternalOrder.Streams.readHistogramOrder(in, false);
         minDocCount = in.readVLong();
         if (minDocCount == 0) {
             emptyBucketInfo = new EmptyBucketInfo(in);
@@ -239,7 +247,7 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
 
     @Override
     protected void doWriteTo(StreamOutput out) throws IOException {
-        InternalOrder.Streams.writeOrder(order, out);
+        InternalOrder.Streams.writeHistogramOrder(order, out, false);
         out.writeVLong(minDocCount);
         if (minDocCount == 0) {
             emptyBucketInfo.writeTo(out);
@@ -400,18 +408,18 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
             addEmptyBuckets(reducedBuckets, reduceContext);
         }
 
-        if (order == InternalOrder.KEY_ASC || reduceContext.isFinalReduce() == false) {
+        if (InternalOrder.isKeyAsc(order) || reduceContext.isFinalReduce() == false) {
             // nothing to do, data are already sorted since shards return
             // sorted buckets and the merge-sort performed by reduceBuckets
             // maintains order
-        } else if (order == InternalOrder.KEY_DESC) {
+        } else if (InternalOrder.isKeyDesc(order)) {
             // we just need to reverse here...
             List<Bucket> reverse = new ArrayList<>(reducedBuckets);
             Collections.reverse(reverse);
             reducedBuckets = reverse;
         } else {
-            // sorted by sub-aggregation, need to fall back to a costly n*log(n) sort
-            CollectionUtil.introSort(reducedBuckets, order.comparator());
+            // sorted by compound order or sub-aggregation, need to fall back to a costly n*log(n) sort
+            CollectionUtil.introSort(reducedBuckets, order.comparator(null));
         }
 
         return new InternalHistogram(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, format, keyed, pipelineAggregators(),

+ 0 - 135
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalOrder.java

@@ -1,135 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.elasticsearch.search.aggregations.bucket.histogram;
-
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
-
-import java.io.IOException;
-import java.util.Comparator;
-import java.util.Objects;
-
-/**
- * An internal {@link Histogram.Order} strategy which is identified by a unique id.
- */
-class InternalOrder extends Histogram.Order {
-
-    final byte id;
-    final String key;
-    final boolean asc;
-    final Comparator<Histogram.Bucket> comparator;
-
-    InternalOrder(byte id, String key, boolean asc, Comparator<Histogram.Bucket> comparator) {
-        this.id = id;
-        this.key = key;
-        this.asc = asc;
-        this.comparator = comparator;
-    }
-
-    byte id() {
-        return id;
-    }
-
-    String key() {
-        return key;
-    }
-
-    boolean asc() {
-        return asc;
-    }
-
-    @Override
-    Comparator<Histogram.Bucket> comparator() {
-        return comparator;
-    }
-
-    @Override
-    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
-        return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
-    }
-    
-    @Override
-    public int hashCode() {
-        return Objects.hash(id, key, asc);
-    }
-    
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        InternalOrder other = (InternalOrder) obj;
-        return Objects.equals(id, other.id)
-                && Objects.equals(key, other.key)
-                && Objects.equals(asc, other.asc);
-    }
-
-    static class Aggregation extends InternalOrder {
-
-        static final byte ID = 0;
-
-        Aggregation(String key, boolean asc) {
-            super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<Histogram.Bucket>(key, asc));
-        }
-
-    }
-
-    static class Streams {
-
-        /**
-         * Writes the given order to the given output (based on the id of the order).
-         */
-        public static void writeOrder(InternalOrder order, StreamOutput out) throws IOException {
-            out.writeByte(order.id());
-            if (order instanceof InternalOrder.Aggregation) {
-                out.writeBoolean(order.asc());
-                out.writeString(order.key());
-            }
-        }
-
-        /**
-         * Reads an order from the given input (based on the id of the order).
-         *
-         * @see Streams#writeOrder(InternalOrder, org.elasticsearch.common.io.stream.StreamOutput)
-         */
-        public static InternalOrder readOrder(StreamInput in) throws IOException {
-            byte id = in.readByte();
-            switch (id) {
-                case 1: return (InternalOrder) Histogram.Order.KEY_ASC;
-                case 2: return (InternalOrder) Histogram.Order.KEY_DESC;
-                case 3: return (InternalOrder) Histogram.Order.COUNT_ASC;
-                case 4: return (InternalOrder) Histogram.Order.COUNT_DESC;
-                case 0:
-                    boolean asc = in.readBoolean();
-                    String key = in.readString();
-                    return new InternalOrder.Aggregation(key, asc);
-                default:
-                    throw new RuntimeException("unknown histogram order");
-            }
-        }
-
-    }
-
-
-}

+ 2 - 1
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java

@@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.internal.SearchContext;
 
 import java.io.IOException;
@@ -37,7 +38,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
     protected final boolean showTermDocCountError;
 
     AbstractStringTermsAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
-            Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode,
+            BucketOrder order, DocValueFormat format, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode,
             boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
         super(name, factories, context, parent, bucketCountThresholds, order, format, subAggCollectMode, pipelineAggregators, metaData);
         this.showTermDocCountError = showTermDocCountError;

+ 4 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java

@@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -76,8 +77,8 @@ public class DoubleTerms extends InternalMappedTerms<DoubleTerms, DoubleTerms.Bu
         }
 
         @Override
-        public int compareTerm(Terms.Bucket other) {
-            return Double.compare(term, ((Number) other.getKey()).doubleValue());
+        public int compareKey(Bucket other) {
+            return Double.compare(term, other.term);
         }
 
         @Override
@@ -105,7 +106,7 @@ public class DoubleTerms extends InternalMappedTerms<DoubleTerms, DoubleTerms.Bu
         }
     }
 
-    public DoubleTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
+    public DoubleTerms(String name, BucketOrder order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData, DocValueFormat format, int shardSize, boolean showTermDocCountError, long otherDocCount,
             List<Bucket> buckets, long docCountError) {
         super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData, format, shardSize, showTermDocCountError,

+ 2 - 1
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java

@@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
 import org.elasticsearch.search.internal.SearchContext;
@@ -39,7 +40,7 @@ import java.util.stream.Collectors;
 public class DoubleTermsAggregator extends LongTermsAggregator {
 
     public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format,
-            Terms.Order order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
+            BucketOrder order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
             SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
         super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode,

+ 12 - 11
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java

@@ -43,6 +43,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
 import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.internal.SearchContext;
 
@@ -70,7 +71,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
     protected SortedSetDocValues globalOrds;
 
     public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource,
-           Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
+           BucketOrder order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
            IncludeExclude.OrdinalsFilter includeExclude, SearchContext context, Aggregator parent,
            SubAggCollectionMode collectionMode, boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators,
            Map<String, Object> metaData) throws IOException {
@@ -122,8 +123,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
                 public void collect(int doc, long bucket) throws IOException {
                     assert bucket == 0;
                     if (ords.advanceExact(doc)) {
-                        for (long globalOrd = ords.nextOrd(); 
-                                globalOrd != SortedSetDocValues.NO_MORE_ORDS; 
+                        for (long globalOrd = ords.nextOrd();
+                                globalOrd != SortedSetDocValues.NO_MORE_ORDS;
                                 globalOrd = ords.nextOrd()) {
                             collectExistingBucket(sub, doc, globalOrd);
                         }
@@ -218,8 +219,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
         }
 
         @Override
-        public int compareTerm(Terms.Bucket other) {
-            return Long.compare(globalOrd, ((OrdBucket) other).globalOrd);
+        public int compareKey(OrdBucket other) {
+            return Long.compare(globalOrd, other.globalOrd);
         }
 
         @Override
@@ -261,7 +262,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
 
         private final LongHash bucketOrds;
 
-        public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource, Terms.Order order,
+        public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource, BucketOrder order,
                 DocValueFormat format, BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude,
                 SearchContext context, Aggregator parent, SubAggCollectionMode collectionMode,
                 boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
@@ -296,8 +297,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
                     @Override
                     public void collect(int doc, long bucket) throws IOException {
                         if (ords.advanceExact(doc)) {
-                            for (long globalOrd = ords.nextOrd(); 
-                                    globalOrd != SortedSetDocValues.NO_MORE_ORDS; 
+                            for (long globalOrd = ords.nextOrd();
+                                    globalOrd != SortedSetDocValues.NO_MORE_ORDS;
                                     globalOrd = ords.nextOrd()) {
                                 long bucketOrd = bucketOrds.add(globalOrd);
                                 if (bucketOrd < 0) {
@@ -337,7 +338,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
         private SortedSetDocValues segmentOrds;
 
         public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource,
-                Terms.Order order, DocValueFormat format,
+                BucketOrder order, DocValueFormat format,
                 BucketCountThresholds bucketCountThresholds, SearchContext context, Aggregator parent,
                 SubAggCollectionMode collectionMode, boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators,
                 Map<String, Object> metaData) throws IOException {
@@ -371,8 +372,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
                     public void collect(int doc, long bucket) throws IOException {
                         assert bucket == 0;
                         if (ords.advanceExact(doc)) {
-                            for (long segmentOrd = ords.nextOrd(); 
-                                    segmentOrd != SortedSetDocValues.NO_MORE_ORDS; 
+                            for (long segmentOrd = ords.nextOrd();
+                                    segmentOrd != SortedSetDocValues.NO_MORE_ORDS;
                                     segmentOrd = ords.nextOrd()) {
                                 segmentDocCounts.increment(segmentOrd + 1, 1);
                             }

+ 3 - 2
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedTerms.java

@@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -46,7 +47,7 @@ public abstract class InternalMappedTerms<A extends InternalTerms<A, B>, B exten
 
     protected long docCountError;
 
-    protected InternalMappedTerms(String name, Terms.Order order, int requiredSize, long minDocCount,
+    protected InternalMappedTerms(String name, BucketOrder order, int requiredSize, long minDocCount,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData, DocValueFormat format, int shardSize,
             boolean showTermDocCountError, long otherDocCount, List<B> buckets, long docCountError) {
         super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData);
@@ -83,7 +84,7 @@ public abstract class InternalMappedTerms<A extends InternalTerms<A, B>, B exten
 
     @Override
     protected void setDocCountError(long docCountError) {
-        this.docCountError = docCountError; 
+        this.docCountError = docCountError;
     }
 
     @Override

+ 0 - 385
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalOrder.java

@@ -1,385 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.elasticsearch.search.aggregations.bucket.terms;
-
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.util.Comparators;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-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.bucket.terms.Terms.Order;
-import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
-import org.elasticsearch.search.aggregations.support.AggregationPath;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Objects;
-
-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(COUNT_DESC_ID, "_count", false, new Comparator<Terms.Bucket>() {
-        @Override
-        public int compare(Terms.Bucket o1, Terms.Bucket o2) {
-            return  Long.compare(o2.getDocCount(), o1.getDocCount());
-        }
-    });
-
-    /**
-     * Order by the (lower) count of each term.
-     */
-    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) {
-            return Long.compare(o1.getDocCount(), o2.getDocCount());
-        }
-    });
-
-    /**
-     * Order by the terms.
-     */
-    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 o2.compareTerm(o1);
-        }
-    });
-
-    /**
-     * Order by the terms.
-     */
-    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) {
-            return o1.compareTerm(o2);
-        }
-    });
-
-    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.orderElements.size() == 2 && compoundOrder.orderElements.get(0) == COUNT_DESC && compoundOrder.orderElements.get(1) == TERM_ASC) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    public static boolean isTermOrder(Terms.Order order) {
-        if (order == TERM_ASC) {
-            return true;
-        } else if (order == TERM_DESC) {
-            return true;
-        } else if (order instanceof CompoundOrder) {
-            // check if its a compound order with only a single element ordering
-            // by term
-            CompoundOrder compoundOrder = (CompoundOrder) order;
-            if (compoundOrder.orderElements.size() == 1 && compoundOrder.orderElements.get(0) == TERM_ASC
-                    || compoundOrder.orderElements.get(0) == TERM_DESC) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    final byte id;
-    final String key;
-    final boolean asc;
-    protected final Comparator<Terms.Bucket> comparator;
-
-    InternalOrder(byte id, String key, boolean asc, Comparator<Terms.Bucket> comparator) {
-        this.id = id;
-        this.key = key;
-        this.asc = asc;
-        this.comparator = comparator;
-    }
-
-    @Override
-    byte id() {
-        return id;
-    }
-
-    @Override
-    protected Comparator<Terms.Bucket> comparator(Aggregator aggregator) {
-        return comparator;
-    }
-
-    @Override
-    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
-        return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
-    }
-
-    public static Terms.Order validate(Terms.Order order, Aggregator termsAggregator) {
-        if (order instanceof CompoundOrder) {
-            for (Terms.Order innerOrder : ((CompoundOrder)order).orderElements) {
-                validate(innerOrder, termsAggregator);
-            }
-            return order;
-        } else if (!(order instanceof Aggregation)) {
-            return order;
-        }
-        AggregationPath path = ((Aggregation) order).path();
-        path.validate(termsAggregator);
-        return order;
-    }
-
-    static class Aggregation extends InternalOrder {
-
-        static final byte ID = 0;
-
-        Aggregation(String key, boolean asc) {
-            super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<Terms.Bucket>(key, asc));
-        }
-
-        AggregationPath path() {
-            return ((MultiBucketsAggregation.Bucket.SubAggregationComparator) comparator).path();
-        }
-
-        @Override
-        protected Comparator<Terms.Bucket> comparator(Aggregator termsAggregator) {
-            if (termsAggregator == null) {
-                return comparator;
-            }
-
-            // Internal Optimization:
-            //
-            // in this phase, if the order is based on sub-aggregations, we need to use a different comparator
-            // to avoid constructing buckets for ordering purposes (we can potentially have a lot of buckets and building
-            // them will cause loads of redundant object constructions). The "special" comparators here will fetch the
-            // sub aggregation values directly from the sub aggregators bypassing bucket creation. Note that the comparator
-            // attached to the order will still be used in the reduce phase of the Aggregation.
-
-            AggregationPath path = path();
-            final Aggregator aggregator = path.resolveAggregator(termsAggregator);
-            final String key = path.lastPathElement().key;
-
-            if (aggregator instanceof SingleBucketAggregator) {
-                assert key == null : "this should be picked up before the aggregation is executed - on validate";
-                return new Comparator<Terms.Bucket>() {
-                    @Override
-                    public int compare(Terms.Bucket o1, Terms.Bucket o2) {
-                        int mul = asc ? 1 : -1;
-                        int v1 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) o1).bucketOrd);
-                        int v2 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) o2).bucketOrd);
-                        return mul * (v1 - v2);
-                    }
-                };
-            }
-
-            // with only support single-bucket aggregators
-            assert !(aggregator instanceof BucketsAggregator) : "this should be picked up before the aggregation is executed - on validate";
-
-            if (aggregator instanceof NumericMetricsAggregator.MultiValue) {
-                assert key != null : "this should be picked up before the aggregation is executed - on validate";
-                return new Comparator<Terms.Bucket>() {
-                    @Override
-                    public int compare(Terms.Bucket o1, Terms.Bucket o2) {
-                        double v1 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) o1).bucketOrd);
-                        double v2 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) o2).bucketOrd);
-                        // some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
-                        // the bottom
-                        return Comparators.compareDiscardNaN(v1, v2, asc);
-                    }
-                };
-            }
-
-            // single-value metrics agg
-            return new Comparator<Terms.Bucket>() {
-                @Override
-                public int compare(Terms.Bucket o1, Terms.Bucket o2) {
-                    double v1 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) o1).bucketOrd);
-                    double v2 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) o2).bucketOrd);
-                    // some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
-                    // the bottom
-                    return Comparators.compareDiscardNaN(v1, v2, asc);
-                }
-            };
-        }
-    }
-
-    static class CompoundOrder extends Terms.Order {
-
-        static final byte ID = -1;
-
-        private final List<Terms.Order> orderElements;
-
-        CompoundOrder(List<Terms.Order> compoundOrder) {
-            this(compoundOrder, true);
-        }
-
-        CompoundOrder(List<Terms.Order> compoundOrder, boolean absoluteOrdering) {
-            this.orderElements = new LinkedList<>(compoundOrder);
-            Terms.Order lastElement = compoundOrder.get(compoundOrder.size() - 1);
-            if (absoluteOrdering && !(InternalOrder.TERM_ASC == lastElement || InternalOrder.TERM_DESC == lastElement)) {
-                // add term order ascending as a tie-breaker to avoid non-deterministic ordering
-                // if all user provided comparators return 0.
-                this.orderElements.add(Order.term(true));
-            }
-        }
-
-        @Override
-        byte id() {
-            return ID;
-        }
-
-        List<Terms.Order> orderElements() {
-            return Collections.unmodifiableList(orderElements);
-        }
-
-        @Override
-        public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
-            builder.startArray();
-            for (Terms.Order order : orderElements) {
-                order.toXContent(builder, params);
-            }
-            return builder.endArray();
-        }
-
-        @Override
-        protected Comparator<Bucket> comparator(Aggregator aggregator) {
-            return new CompoundOrderComparator(orderElements, aggregator);
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(orderElements);
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            if (obj == null) {
-                return false;
-            }
-            if (getClass() != obj.getClass()) {
-                return false;
-            }
-            CompoundOrder other = (CompoundOrder) obj;
-            return Objects.equals(orderElements, other.orderElements);
-        }
-
-        public static class CompoundOrderComparator implements Comparator<Terms.Bucket> {
-
-            private List<Terms.Order> compoundOrder;
-            private Aggregator aggregator;
-
-            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(Terms.Order order, StreamOutput out) throws IOException {
-            if (order instanceof Aggregation) {
-                out.writeByte(order.id());
-                Aggregation aggregationOrder = (Aggregation) order;
-                out.writeBoolean(((MultiBucketsAggregation.Bucket.SubAggregationComparator) aggregationOrder.comparator).asc());
-                AggregationPath path = ((Aggregation) order).path();
-                out.writeString(path.toString());
-            } else if (order instanceof CompoundOrder) {
-                CompoundOrder compoundOrder = (CompoundOrder) order;
-                    out.writeByte(order.id());
-                    out.writeVInt(compoundOrder.orderElements.size());
-                    for (Terms.Order innerOrder : compoundOrder.orderElements) {
-                        Streams.writeOrder(innerOrder, out);
-                    }
-            } else {
-                out.writeByte(order.id());
-            }
-        }
-
-        public static Terms.Order readOrder(StreamInput in) throws IOException {
-            return readOrder(in, false);
-        }
-
-        public static Terms.Order readOrder(StreamInput in, boolean absoluteOrder) throws IOException {
-            byte id = in.readByte();
-            switch (id) {
-                case COUNT_DESC_ID: return absoluteOrder ? new CompoundOrder(Collections.singletonList((Terms.Order) InternalOrder.COUNT_DESC)) : InternalOrder.COUNT_DESC;
-                case COUNT_ASC_ID: return absoluteOrder ? new CompoundOrder(Collections.singletonList((Terms.Order) InternalOrder.COUNT_ASC)) : 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();
-                    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, false));
-                    }
-                    return new CompoundOrder(compoundOrder, absoluteOrder);
-                default:
-                    throw new RuntimeException("unknown terms order");
-            }
-        }
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(id, asc);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-        InternalOrder other = (InternalOrder) obj;
-        return Objects.equals(id, other.id)
-                && Objects.equals(asc, other.asc);
-    }
-}

+ 10 - 7
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java

@@ -31,6 +31,9 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
 import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
+import org.elasticsearch.search.aggregations.KeyComparable;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -46,8 +49,8 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
     protected static final ParseField DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME = new ParseField("doc_count_error_upper_bound");
     protected static final ParseField SUM_OF_OTHER_DOC_COUNTS = new ParseField("sum_other_doc_count");
 
-    public abstract static class Bucket<B extends Bucket<B>> extends InternalMultiBucketAggregation.InternalBucket implements Terms.Bucket {
-
+    public abstract static class Bucket<B extends Bucket<B>> extends InternalMultiBucketAggregation.InternalBucket
+        implements Terms.Bucket, KeyComparable<B> {
         /**
          * Reads a bucket. Should be a constructor reference.
          */
@@ -177,11 +180,11 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
         }
     }
 
-    protected final Terms.Order order;
+    protected final BucketOrder order;
     protected final int requiredSize;
     protected final long minDocCount;
 
-    protected InternalTerms(String name, Terms.Order order, int requiredSize, long minDocCount,
+    protected InternalTerms(String name, BucketOrder order, int requiredSize, long minDocCount,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
         super(name, pipelineAggregators, metaData);
         this.order = order;
@@ -201,7 +204,7 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
 
     @Override
     protected final void doWriteTo(StreamOutput out) throws IOException {
-        InternalOrder.Streams.writeOrder(order, out);
+        order.writeTo(out);
         writeSize(requiredSize, out);
         out.writeVLong(minDocCount);
         writeTermTypeInfoTo(out);
@@ -238,9 +241,9 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
             }
             otherDocCount += terms.getSumOfOtherDocCounts();
             final long thisAggDocCountError;
-            if (terms.getBuckets().size() < getShardSize() || InternalOrder.isTermOrder(order)) {
+            if (terms.getBuckets().size() < getShardSize() || InternalOrder.isKeyOrder(order)) {
                 thisAggDocCountError = 0;
-            } else if (InternalOrder.isCountDesc(this.order)) {
+            } else if (InternalOrder.isCountDesc(order)) {
                 if (terms.getDocCountError() > 0) {
                     // If there is an existing docCountError for this agg then
                     // use this as the error for this aggregation

+ 4 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTerms.java

@@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -76,8 +77,8 @@ public class LongTerms extends InternalMappedTerms<LongTerms, LongTerms.Bucket>
         }
 
         @Override
-        public int compareTerm(Terms.Bucket other) {
-            return Long.compare(term, ((Number) other.getKey()).longValue());
+        public int compareKey(Bucket other) {
+            return Long.compare(term, other.term);
         }
 
         @Override
@@ -105,7 +106,7 @@ public class LongTerms extends InternalMappedTerms<LongTerms, LongTerms.Bucket>
         }
     }
 
-    public LongTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
+    public LongTerms(String name, BucketOrder order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData, DocValueFormat format, int shardSize, boolean showTermDocCountError, long otherDocCount,
             List<Bucket> buckets, long docCountError) {
         super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData, format, shardSize, showTermDocCountError,

+ 4 - 2
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java

@@ -32,6 +32,8 @@ import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriority
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude.LongFilter;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.internal.SearchContext;
 
@@ -50,7 +52,7 @@ public class LongTermsAggregator extends TermsAggregator {
     private LongFilter longFilter;
 
     public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format,
-            Terms.Order order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
+            BucketOrder order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
             SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
         super(name, factories, aggregationContext, parent, bucketCountThresholds, order, format, subAggCollectMode, pipelineAggregators, metaData);
@@ -106,7 +108,7 @@ public class LongTermsAggregator extends TermsAggregator {
     public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
         assert owningBucketOrdinal == 0;
 
-        if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
+        if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
             // we need to fill-in the blanks
             for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) {
                 final SortedNumericDocValues values = getValues(valuesSource, ctx);

+ 4 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTerms.java

@@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.io.IOException;
 import java.util.List;
@@ -75,8 +76,8 @@ public class StringTerms extends InternalMappedTerms<StringTerms, StringTerms.Bu
         }
 
         @Override
-        public int compareTerm(Terms.Bucket other) {
-            return termBytes.compareTo(((Bucket) other).termBytes);
+        public int compareKey(Bucket other) {
+            return termBytes.compareTo(other.termBytes);
         }
 
         @Override
@@ -100,7 +101,7 @@ public class StringTerms extends InternalMappedTerms<StringTerms, StringTerms.Bu
         }
     }
 
-    public StringTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
+    public StringTerms(String name, BucketOrder order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData, DocValueFormat format, int shardSize, boolean showTermDocCountError, long otherDocCount,
             List<Bucket> buckets, long docCountError) {
         super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData, format,

+ 4 - 2
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java

@@ -33,6 +33,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
 import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.internal.SearchContext;
 
@@ -51,7 +53,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
     private final IncludeExclude.StringFilter includeExclude;
 
     public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
-            Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
+            BucketOrder order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
             IncludeExclude.StringFilter includeExclude, SearchContext context,
             Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@@ -110,7 +112,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
     public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
         assert owningBucketOrdinal == 0;
 
-        if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
+        if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
             // we need to fill-in the blanks
             for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) {
                 final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);

+ 0 - 86
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/Terms.java

@@ -18,12 +18,8 @@
  */
 package org.elasticsearch.search.aggregations.bucket.terms;
 
-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;
 
 /**
@@ -39,8 +35,6 @@ public interface Terms extends MultiBucketsAggregation {
 
         Number getKeyAsNumber();
 
-        int compareTerm(Terms.Bucket other);
-
         long getDocCountError();
     }
 
@@ -65,84 +59,4 @@ public interface Terms extends MultiBucketsAggregation {
      * it to the top buckets.
      */
     long getSumOfOtherDocCounts();
-
-    /**
-     * Determines the order by which the term buckets will be sorted
-     */
-    abstract class Order implements ToXContent {
-
-        /**
-         * @return a bucket ordering strategy that sorts buckets by their document counts (ascending or descending)
-         */
-        public static Order count(boolean asc) {
-            return asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC;
-        }
-
-        /**
-         * @return a bucket ordering strategy that sorts buckets by their terms (ascending or descending)
-         */
-        public static Order term(boolean asc) {
-            return asc ? InternalOrder.TERM_ASC : InternalOrder.TERM_DESC;
-        }
-
-        /**
-         * Creates a bucket ordering strategy which sorts buckets based on a single-valued calc get
-         *
-         * @param   path the name of the get
-         * @param   asc             The direction of the order (ascending or descending)
-         */
-        public static Order aggregation(String path, boolean asc) {
-            return new InternalOrder.Aggregation(path, asc);
-        }
-
-        /**
-         * Creates a bucket ordering strategy which sorts buckets based on a multi-valued calc get
-         *
-         * @param   aggregationName the name of the get
-         * @param   metricName       The name of the value of the multi-value get by which the sorting will be applied
-         * @param   asc             The direction of the order (ascending or descending)
-         */
-        public static Order aggregation(String aggregationName, String metricName, boolean 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:
-         *
-         *          - aggregation phase, where each shard builds a list of term buckets to be sent to the coordinating node.
-         *            In this phase, the passed in aggregator will be the terms aggregator that aggregates the buckets on the
-         *            shard level.
-         *
-         *          - reduce phase, where the coordinating node gathers all the buckets from all the shards and reduces them
-         *            to a final bucket list. In this case, the passed in aggregator will be {@code null}
-         */
-        protected abstract Comparator<Bucket> comparator(Aggregator aggregator);
-
-        abstract byte id();
-
-        @Override
-        public abstract int hashCode();
-
-        @Override
-        public abstract boolean equals(Object obj);
-
-    }
 }

+ 20 - 56
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java

@@ -19,20 +19,20 @@
 package org.elasticsearch.search.aggregations.bucket.terms;
 
 import org.elasticsearch.common.ParseField;
-import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
+import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
 import org.elasticsearch.search.aggregations.support.ValueType;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@@ -82,7 +82,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
                 (p, c) -> SubAggCollectionMode.parse(p.text()),
                 SubAggCollectionMode.KEY, ObjectParser.ValueType.STRING);
 
-        PARSER.declareObjectArray(TermsAggregationBuilder::order, TermsAggregationBuilder::parseOrderParam,
+        PARSER.declareObjectArray(TermsAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
                 TermsAggregationBuilder.ORDER_FIELD);
 
         PARSER.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())),
@@ -96,7 +96,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
         return PARSER.parse(context.parser(), new TermsAggregationBuilder(aggregationName, null), context);
     }
 
-    private Terms.Order order = Terms.Order.compound(Terms.Order.count(false), Terms.Order.term(true));
+    private BucketOrder order = BucketOrder.compound(BucketOrder.count(false)); // automatically adds tie-breaker key asc order
     private IncludeExclude includeExclude = null;
     private String executionHint = null;
     private SubAggCollectionMode collectMode = null;
@@ -132,7 +132,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
         out.writeOptionalWriteable(collectMode);
         out.writeOptionalString(executionHint);
         out.writeOptionalWriteable(includeExclude);
-        InternalOrder.Streams.writeOrder(order, out);
+        order.writeTo(out);
         out.writeBoolean(showTermDocCountError);
     }
 
@@ -189,32 +189,37 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
         return this;
     }
 
-    /**
-     * Sets the order in which the buckets will be returned.
-     */
-    public TermsAggregationBuilder order(Terms.Order order) {
+    /** Set a new order on this builder and return the builder so that calls
+     *  can be chained. A tie-breaker may be added to avoid non-deterministic ordering. */
+    public TermsAggregationBuilder order(BucketOrder order) {
         if (order == null) {
             throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
         }
-        this.order = order;
+        if(order instanceof CompoundOrder || InternalOrder.isKeyOrder(order)) {
+            this.order = order; // if order already contains a tie-breaker we are good to go
+        } else { // otherwise add a tie-breaker by using a compound order
+            this.order = BucketOrder.compound(order);
+        }
         return this;
     }
 
     /**
-     * Sets the order in which the buckets will be returned.
+     * Sets the order in which the buckets will be returned. A tie-breaker may be added to avoid non-deterministic
+     * ordering.
      */
-    public TermsAggregationBuilder order(List<Terms.Order> orders) {
+    public TermsAggregationBuilder order(List<BucketOrder> orders) {
         if (orders == null) {
             throw new IllegalArgumentException("[orders] must not be null: [" + name + "]");
         }
-        order(Terms.Order.compound(orders));
+        // if the list only contains one order use that to avoid inconsistent xcontent
+        order(orders.size() > 1 ? BucketOrder.compound(orders) : orders.get(0));
         return this;
     }
 
     /**
      * Gets the order in which the buckets will be returned.
      */
-    public Terms.Order order() {
+    public BucketOrder order() {
         return order;
     }
 
@@ -327,45 +332,4 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
         return NAME;
     }
 
-    private static Terms.Order parseOrderParam(XContentParser parser, QueryParseContext context) throws IOException {
-        XContentParser.Token token;
-        Terms.Order 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 ParsingException(parser.getTokenLocation(),
-                            "Unknown terms order direction [" + dir + "]");
-                }
-            } else {
-                throw new ParsingException(parser.getTokenLocation(),
-                        "Unexpected token " + token + " for [order]");
-            }
-        }
-        if (orderKey == null) {
-            throw new ParsingException(parser.getTokenLocation(),
-                    "Must specify at least one field for [order]");
-        } else {
-            orderParam = resolveOrder(orderKey, orderAsc);
-        }
-        return orderParam;
-    }
-
-    static Terms.Order resolveOrder(String key, boolean asc) {
-        if ("_term".equals(key)) {
-            return Order.term(asc);
-        }
-        if ("_count".equals(key)) {
-            return Order.count(asc);
-        }
-        return Order.aggregation(key, asc);
-    }
 }

+ 64 - 5
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java

@@ -24,19 +24,26 @@ import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.util.Comparators;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
-import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.Aggregation;
-import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.CompoundOrder;
+import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
+import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
+import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 import org.elasticsearch.search.aggregations.support.AggregationPath;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
+import org.elasticsearch.search.aggregations.InternalOrder.Aggregation;
+import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
 import org.elasticsearch.search.internal.SearchContext;
 
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -168,12 +175,12 @@ public abstract class TermsAggregator extends BucketsAggregator {
 
     protected final DocValueFormat format;
     protected final BucketCountThresholds bucketCountThresholds;
-    protected final Terms.Order order;
+    protected final BucketOrder order;
     protected final Set<Aggregator> aggsUsedForSorting = new HashSet<>();
     protected final SubAggCollectionMode collectMode;
 
     public TermsAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
-            BucketCountThresholds bucketCountThresholds, Terms.Order order, DocValueFormat format, SubAggCollectionMode collectMode,
+            BucketCountThresholds bucketCountThresholds, BucketOrder order, DocValueFormat format, SubAggCollectionMode collectMode,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
         super(name, factories, context, parent, pipelineAggregators, metaData);
         this.bucketCountThresholds = bucketCountThresholds;
@@ -186,7 +193,7 @@ public abstract class TermsAggregator extends BucketsAggregator {
             aggsUsedForSorting.add(path.resolveTopmostAggregator(this));
         } else if (order instanceof CompoundOrder) {
             CompoundOrder compoundOrder = (CompoundOrder) order;
-            for (Terms.Order orderElement : compoundOrder.orderElements()) {
+            for (BucketOrder orderElement : compoundOrder.orderElements()) {
                 if (orderElement instanceof Aggregation) {
                     AggregationPath path = ((Aggregation) orderElement).path();
                     aggsUsedForSorting.add(path.resolveTopmostAggregator(this));
@@ -195,6 +202,58 @@ public abstract class TermsAggregator extends BucketsAggregator {
         }
     }
 
+    /**
+     * Internal Optimization for ordering {@link InternalTerms.Bucket}s by a sub aggregation.
+     * <p>
+     * in this phase, if the order is based on sub-aggregations, we need to use a different comparator
+     * to avoid constructing buckets for ordering purposes (we can potentially have a lot of buckets and building
+     * them will cause loads of redundant object constructions). The "special" comparators here will fetch the
+     * sub aggregation values directly from the sub aggregators bypassing bucket creation. Note that the comparator
+     * attached to the order will still be used in the reduce phase of the Aggregation.
+     *
+     * @param path determines which sub aggregation to use for ordering.
+     * @param asc  {@code true} for ascending order, {@code false} for descending.
+     * @return {@code Comparator} to order {@link InternalTerms.Bucket}s in the desired order.
+     */
+    public Comparator<Bucket> bucketComparator(AggregationPath path, boolean asc) {
+
+        final Aggregator aggregator = path.resolveAggregator(this);
+        final String key = path.lastPathElement().key;
+
+        if (aggregator instanceof SingleBucketAggregator) {
+            assert key == null : "this should be picked up before the aggregation is executed - on validate";
+            return (b1, b2) -> {
+                int mul = asc ? 1 : -1;
+                int v1 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) b1).bucketOrd);
+                int v2 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) b2).bucketOrd);
+                return mul * (v1 - v2);
+            };
+        }
+
+        // with only support single-bucket aggregators
+        assert !(aggregator instanceof BucketsAggregator) : "this should be picked up before the aggregation is executed - on validate";
+
+        if (aggregator instanceof NumericMetricsAggregator.MultiValue) {
+            assert key != null : "this should be picked up before the aggregation is executed - on validate";
+            return (b1, b2) -> {
+                double v1 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) b1).bucketOrd);
+                double v2 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) b2).bucketOrd);
+                // some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
+                // the bottom
+                return Comparators.compareDiscardNaN(v1, v2, asc);
+            };
+        }
+
+        // single-value metrics agg
+        return (b1, b2) -> {
+            double v1 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) b1).bucketOrd);
+            double v2 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) b2).bucketOrd);
+            // some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
+            // the bottom
+            return Comparators.compareDiscardNaN(v1, v2, asc);
+        };
+    }
+
     @Override
     protected boolean shouldDefer(Aggregator aggregator) {
         return collectMode == SubAggCollectionMode.BREADTH_FIRST

+ 11 - 9
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java

@@ -33,6 +33,8 @@ import org.elasticsearch.search.aggregations.bucket.BucketUtils;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
+import org.elasticsearch.search.aggregations.InternalOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
 import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
@@ -44,14 +46,14 @@ import java.util.Map;
 
 public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, TermsAggregatorFactory> {
 
-    private final Terms.Order order;
+    private final BucketOrder order;
     private final IncludeExclude includeExclude;
     private final String executionHint;
     private final SubAggCollectionMode collectMode;
     private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
     private boolean showTermDocCountError;
 
-    public TermsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, Terms.Order order,
+    public TermsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, BucketOrder order,
             IncludeExclude includeExclude, String executionHint, SubAggCollectionMode collectMode,
             TermsAggregator.BucketCountThresholds bucketCountThresholds, boolean showTermDocCountError, SearchContext context,
             AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
@@ -90,7 +92,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
             return asMultiBucketAggregator(this, context, parent);
         }
         BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds);
-        if (!(order == InternalOrder.TERM_ASC || order == InternalOrder.TERM_DESC)
+        if (InternalOrder.isKeyOrder(order) == false
                 && bucketCountThresholds.getShardSize() == TermsAggregationBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) {
             // The user has not made a shardSize selection. Use default
             // heuristic to avoid any wrong-ranking caused by distributed
@@ -129,7 +131,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
                 // to be unbounded and most instances may only aggregate few
                 // documents, so use hashed based
                 // global ordinals to keep the bucket ords dense.
-                // Additionally, if using partitioned terms the regular global 
+                // Additionally, if using partitioned terms the regular global
                 // ordinals would be sparse so we opt for hash
                 if (Aggregator.descendsFromBucketAggregator(parent) ||
                         (includeExclude != null && includeExclude.isPartitionBased())) {
@@ -223,7 +225,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
         MAP(new ParseField("map")) {
 
             @Override
-            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
+            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
                     DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
                     SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
                     boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
@@ -242,7 +244,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
         GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
 
             @Override
-            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
+            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
                     DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
                     SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
                     boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
@@ -262,7 +264,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
         GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) {
 
             @Override
-            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
+            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
                     DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
                     SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
                     boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
@@ -281,7 +283,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
         GLOBAL_ORDINALS_LOW_CARDINALITY(new ParseField("global_ordinals_low_cardinality")) {
 
             @Override
-            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
+            Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
                     DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
                     SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
                     boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
@@ -319,7 +321,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
             this.parseField = parseField;
         }
 
-        abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
+        abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
                 DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
                 SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
                 boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)

+ 2 - 1
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/UnmappedTerms.java

@@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -50,7 +51,7 @@ public class UnmappedTerms extends InternalTerms<UnmappedTerms, UnmappedTerms.Bu
         }
     }
 
-    public UnmappedTerms(String name, Terms.Order order, int requiredSize, long minDocCount,
+    public UnmappedTerms(String name, BucketOrder order, int requiredSize, long minDocCount,
             List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
         super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData);
     }

+ 15 - 14
core/src/main/java/org/elasticsearch/search/aggregations/support/AggregationPath.java

@@ -33,7 +33,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * A path that can be used to sort/order buckets (in some multi-bucket aggregations, eg terms &amp; histogram) based on
+ * A path that can be used to sort/order buckets (in some multi-bucket aggregations, e.g. terms &amp; histogram) based on
  * sub-aggregations. The path may point to either a single-bucket aggregation or a metrics aggregation. If the path
  * points to a single-bucket aggregation, the sort will be applied based on the {@code doc_count} of the bucket. If this
  * path points to a metrics aggregation, if it's a single-value metrics (eg. avg, max, min, etc..) the sort will be
@@ -281,14 +281,15 @@ public class AggregationPath {
     /**
      * Validates this path over the given aggregator as a point of reference.
      *
-     * @param root  The point of reference of this path
+     * @param root The point of reference of this path
+     * @throws AggregationExecutionException on validation error
      */
-    public void validate(Aggregator root) {
+    public void validate(Aggregator root) throws AggregationExecutionException {
         Aggregator aggregator = root;
         for (int i = 0; i < pathElements.size(); i++) {
             aggregator = aggregator.subAggregator(pathElements.get(i).name);
             if (aggregator == null) {
-                throw new AggregationExecutionException("Invalid term-aggregator order path [" + this + "]. Unknown aggregation ["
+                throw new AggregationExecutionException("Invalid aggregator order path [" + this + "]. Unknown aggregation ["
                         + pathElements.get(i).name + "]");
             }
             if (i < pathElements.size() - 1) {
@@ -296,16 +297,16 @@ public class AggregationPath {
                 // we're in the middle of the path, so the aggregator can only be a single-bucket aggregator
 
                 if (!(aggregator instanceof SingleBucketAggregator)) {
-                    throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
-                            "]. Terms buckets can only be sorted on a sub-aggregator path " +
+                    throw new AggregationExecutionException("Invalid aggregation order path [" + this +
+                            "]. Buckets can only be sorted on a sub-aggregator path " +
                             "that is built out of zero or more single-bucket aggregations within the path and a final " +
                             "single-bucket or a metrics aggregation at the path end. Sub-path [" +
                             subPath(0, i + 1) + "] points to non single-bucket aggregation");
                 }
 
                 if (pathElements.get(i).key != null) {
-                    throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
-                            "]. Terms buckets can only be sorted on a sub-aggregator path " +
+                    throw new AggregationExecutionException("Invalid aggregation order path [" + this +
+                            "]. Buckets can only be sorted on a sub-aggregator path " +
                             "that is built out of zero or more single-bucket aggregations within the path and a " +
                             "final single-bucket or a metrics aggregation at the path end. Sub-path [" +
                             subPath(0, i + 1) + "] points to non single-bucket aggregation");
@@ -314,8 +315,8 @@ public class AggregationPath {
         }
         boolean singleBucket = aggregator instanceof SingleBucketAggregator;
         if (!singleBucket && !(aggregator instanceof NumericMetricsAggregator)) {
-            throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
-                    "]. Terms buckets can only be sorted on a sub-aggregator path " +
+            throw new AggregationExecutionException("Invalid aggregation order path [" + this +
+                    "]. Buckets can only be sorted on a sub-aggregator path " +
                     "that is built out of zero or more single-bucket aggregations within the path and a final " +
                     "single-bucket or a metrics aggregation at the path end.");
         }
@@ -324,7 +325,7 @@ public class AggregationPath {
 
         if (singleBucket) {
             if (lastToken.key != null && !"doc_count".equals(lastToken.key)) {
-                throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
+                throw new AggregationExecutionException("Invalid aggregation order path [" + this +
                         "]. Ordering on a single-bucket aggregation can only be done on its doc_count. " +
                         "Either drop the key (a la \"" + lastToken.name + "\") or change it to \"doc_count\" (a la \"" + lastToken.name + ".doc_count\")");
             }
@@ -333,7 +334,7 @@ public class AggregationPath {
 
         if (aggregator instanceof NumericMetricsAggregator.SingleValue) {
             if (lastToken.key != null && !"value".equals(lastToken.key)) {
-                throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
+                throw new AggregationExecutionException("Invalid aggregation order path [" + this +
                         "]. Ordering on a single-value metrics aggregation can only be done on its value. " +
                         "Either drop the key (a la \"" + lastToken.name + "\") or change it to \"value\" (a la \"" + lastToken.name + ".value\")");
             }
@@ -342,12 +343,12 @@ public class AggregationPath {
 
         // the aggregator must be of a multi-value metrics type
         if (lastToken.key == null) {
-            throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
+            throw new AggregationExecutionException("Invalid aggregation order path [" + this +
                     "]. When ordering on a multi-value metrics aggregation a metric name must be specified");
         }
 
         if (!((NumericMetricsAggregator.MultiValue) aggregator).hasMetric(lastToken.key)) {
-            throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
+            throw new AggregationExecutionException("Invalid aggregation order path [" + this +
                     "]. Unknown metric name [" + lastToken.key + "] on multi-value metrics aggregation [" + lastToken.name + "]");
         }
     }

+ 13 - 0
core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java

@@ -32,6 +32,7 @@ import org.elasticsearch.action.termvectors.MultiTermVectorsRequest;
 import org.elasticsearch.action.termvectors.MultiTermVectorsResponse;
 import org.elasticsearch.action.termvectors.TermVectorsRequest;
 import org.elasticsearch.action.termvectors.TermVectorsResponse;
+import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.lucene.search.MoreLikeThisQuery;
@@ -64,6 +65,8 @@ import static org.hamcrest.Matchers.instanceOf;
 
 public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase<MoreLikeThisQueryBuilder> {
 
+    private static final String[] SHUFFLE_PROTECTED_FIELDS = new String[]{Item.Field.DOC.getPreferredName()};
+
     private static String[] randomFields;
     private static Item[] randomLikeItems;
     private static Item[] randomUnlikeItems;
@@ -204,6 +207,16 @@ public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase<MoreLik
         return queryBuilder;
     }
 
+    /**
+     * we don't want to shuffle the "doc" field internally in {@link #testFromXContent()} because even though the
+     * documents would be functionally the same, their {@link BytesReference} representation isn't and thats what we
+     * compare when check for equality of the original and the shuffled builder
+     */
+    @Override
+    protected String[] shuffleProtectedFields() {
+        return SHUFFLE_PROTECTED_FIELDS;
+    }
+
     @Override
     protected Set<String> getObjectsHoldingArbitraryContent() {
         //doc contains arbitrary content, anything can be added to it and no exception will be thrown

+ 9 - 0
core/src/test/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilderTests.java

@@ -83,6 +83,9 @@ import static org.hamcrest.Matchers.nullValue;
 
 public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<FunctionScoreQueryBuilder> {
 
+    private static final String[] SHUFFLE_PROTECTED_FIELDS = new String[] {Script.PARAMS_PARSE_FIELD.getPreferredName(),
+        ExponentialDecayFunctionBuilder.NAME, LinearDecayFunctionBuilder.NAME, GaussDecayFunctionBuilder.NAME};
+
     @Override
     protected Collection<Class<? extends Plugin>> getPlugins() {
         return Collections.singleton(TestPlugin.class);
@@ -106,6 +109,12 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
         return functionScoreQueryBuilder;
     }
 
+    @Override
+    protected String[] shuffleProtectedFields() {
+        // do not shuffle fields that may contain arbitrary content
+        return SHUFFLE_PROTECTED_FIELDS;
+    }
+
     @Override
     protected Set<String> getObjectsHoldingArbitraryContent() {
         //script_score.script.params can contain arbitrary parameters. no error is expected when adding additional objects

+ 158 - 0
core/src/test/java/org/elasticsearch/search/aggregations/InternalOrderTests.java

@@ -0,0 +1,158 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.search.aggregations;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.io.stream.BytesStreamOutput;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.common.xcontent.XContentParser.Token;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
+import org.elasticsearch.test.AbstractSerializingTestCase;
+import org.elasticsearch.test.VersionUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class InternalOrderTests extends AbstractSerializingTestCase<BucketOrder> {
+
+    @Override
+    protected BucketOrder createTestInstance() {
+        if (randomBoolean()) {
+            return getRandomOrder();
+        } else {
+            List<BucketOrder> orders = new ArrayList<>();
+            for (int i = 0; i < randomInt(3); i++) {
+                orders.add(getRandomOrder());
+            }
+            return BucketOrder.compound(orders);
+        }
+    }
+
+    private BucketOrder getRandomOrder() {
+        switch(randomInt(2)) {
+            case 0: return BucketOrder.key(randomBoolean());
+            case 1: return BucketOrder.count(randomBoolean());
+            default: return BucketOrder.aggregation(randomAlphaOfLength(10), randomBoolean());
+        }
+    }
+
+    @Override
+    protected Reader<BucketOrder> instanceReader() {
+        return InternalOrder.Streams::readOrder;
+    }
+
+    @Override
+    protected BucketOrder doParseInstance(XContentParser parser) throws IOException {
+        Token token = parser.nextToken();
+        if (token == Token.START_OBJECT) {
+            return InternalOrder.Parser.parseOrderParam(parser, null);
+        }
+        if (token == Token.START_ARRAY) {
+            List<BucketOrder> orders = new ArrayList<>();
+            while (parser.nextToken() == Token.START_OBJECT) {
+                orders.add(InternalOrder.Parser.parseOrderParam(parser, null));
+            }
+            return BucketOrder.compound(orders);
+        }
+        return null;
+    }
+
+    @Override
+    protected BucketOrder assertSerialization(BucketOrder testInstance) throws IOException {
+        // identical behavior to AbstractWireSerializingTestCase, except assertNotSame is only called for
+        // compound and aggregation order because _key and _count orders are static instances.
+        BucketOrder deserializedInstance = copyInstance(testInstance);
+        assertEquals(testInstance, deserializedInstance);
+        assertEquals(testInstance.hashCode(), deserializedInstance.hashCode());
+        if(testInstance instanceof CompoundOrder || testInstance instanceof InternalOrder.Aggregation) {
+            assertNotSame(testInstance, deserializedInstance);
+        }
+        return deserializedInstance;
+    }
+
+    @Override
+    protected void assertParsedInstance(XContentType xContentType, BytesReference instanceAsBytes, BucketOrder expectedInstance)
+        throws IOException {
+        // identical behavior to AbstractSerializingTestCase, except assertNotSame is only called for
+        // compound and aggregation order because _key and _count orders are static instances.
+        XContentParser parser = createParser(XContentFactory.xContent(xContentType), instanceAsBytes);
+        BucketOrder newInstance = parseInstance(parser);
+        assertEquals(expectedInstance, newInstance);
+        assertEquals(expectedInstance.hashCode(), newInstance.hashCode());
+        if(expectedInstance instanceof CompoundOrder || expectedInstance instanceof InternalOrder.Aggregation) {
+            assertNotSame(newInstance, expectedInstance);
+        }
+    }
+
+    public void testHistogramOrderBwc() throws IOException {
+        for (int runs = 0; runs < NUMBER_OF_TEST_RUNS; runs++) {
+            BucketOrder order = createTestInstance();
+            Version bwcVersion = VersionUtils.randomVersionBetween(random(), VersionUtils.getFirstVersion(),
+                VersionUtils.getPreviousVersion(Version.V_6_0_0_alpha2_UNRELEASED));
+            boolean bwcOrderFlag = randomBoolean();
+            try (BytesStreamOutput out = new BytesStreamOutput()) {
+                out.setVersion(bwcVersion);
+                InternalOrder.Streams.writeHistogramOrder(order, out, bwcOrderFlag);
+                try (StreamInput in = out.bytes().streamInput()) {
+                    in.setVersion(bwcVersion);
+                    BucketOrder actual = InternalOrder.Streams.readHistogramOrder(in, bwcOrderFlag);
+                    BucketOrder expected = order;
+                    if (order instanceof CompoundOrder) {
+                        expected = ((CompoundOrder) order).orderElements.get(0);
+                    }
+                    assertEquals(expected, actual);
+                }
+            }
+        }
+    }
+
+    public void testAggregationOrderEqualsAndHashCode() {
+        String path = randomAlphaOfLength(10);
+        boolean asc = randomBoolean();
+        BucketOrder o1 = BucketOrder.aggregation(path, asc);
+        BucketOrder o2 = BucketOrder.aggregation(path + "test", asc);
+        BucketOrder o3 = BucketOrder.aggregation(path, !asc);
+        BucketOrder o4 = BucketOrder.aggregation(path, asc);
+        assertNotEquals(o1, o2);
+        assertNotEquals(o1.hashCode(), o2.hashCode());
+        assertNotEquals(o1, o3);
+        assertNotEquals(o1.hashCode(), o3.hashCode());
+        assertEquals(o1, o4);
+        assertEquals(o1.hashCode(), o4.hashCode());
+
+        o1 = InternalOrder.compound(o1);
+        o2 = InternalOrder.compound(o2);
+        o3 = InternalOrder.compound(o3);
+        assertNotEquals(o1, o2);
+        assertNotEquals(o1.hashCode(), o2.hashCode());
+        assertNotEquals(o1, o2);
+        assertNotEquals(o1.hashCode(), o2.hashCode());
+        assertNotEquals(o1, o3);
+        assertNotEquals(o1.hashCode(), o3.hashCode());
+        assertNotEquals(o1, o4);
+        assertNotEquals(o1.hashCode(), o4.hashCode());
+    }
+
+}

+ 194 - 10
core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java

@@ -18,7 +18,9 @@
  */
 package org.elasticsearch.search.aggregations.bucket;
 
+import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.action.search.SearchPhaseExecutionException;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.joda.DateMathParser;
 import org.elasticsearch.common.joda.Joda;
@@ -30,13 +32,16 @@ import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.script.ScriptType;
+import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.DateScriptMocks.DateScriptsMockPlugin;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
+import org.elasticsearch.search.aggregations.metrics.avg.Avg;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.hamcrest.Matchers;
 import org.joda.time.DateTime;
@@ -57,6 +62,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
@@ -73,6 +79,8 @@ import static org.hamcrest.core.IsNull.notNullValue;
 @ESIntegTestCase.SuiteScopeTestCase
 public class DateHistogramIT extends ESIntegTestCase {
 
+    static Map<DateTime, Map<String, Object>> expectedMultiSortBuckets;
+
     private DateTime date(int month, int day) {
         return new DateTime(2012, month, day, 0, 0, DateTimeZone.UTC);
     }
@@ -98,6 +106,7 @@ public class DateHistogramIT extends ESIntegTestCase {
         return client().prepareIndex("idx", "type").setSource(jsonBuilder()
                 .startObject()
                 .field("value", value)
+                .field("constant", 1)
                 .field("date", date(month, day))
                 .startArray("dates").value(date(month, day)).value(date(month + 1, day + 1)).endArray()
                 .endObject());
@@ -115,6 +124,9 @@ public class DateHistogramIT extends ESIntegTestCase {
                     .field("value", i * 2)
                     .endObject()));
         }
+
+        getMultiSortDocs(builders);
+
         builders.addAll(Arrays.asList(
                 indexDoc(1, 2, 1),  // date: Jan 2, dates: Jan 2, Feb 3
                 indexDoc(2, 2, 2),  // date: Feb 2, dates: Feb 2, Mar 3
@@ -126,6 +138,50 @@ public class DateHistogramIT extends ESIntegTestCase {
         ensureSearchable();
     }
 
+    private void addExpectedBucket(DateTime key, long docCount, double avg, double sum) {
+        Map<String, Object> bucketProps = new HashMap<>();
+        bucketProps.put("_count", docCount);
+        bucketProps.put("avg_l", avg);
+        bucketProps.put("sum_d", sum);
+        expectedMultiSortBuckets.put(key, bucketProps);
+    }
+
+    private void getMultiSortDocs(List<IndexRequestBuilder> builders) throws IOException {
+        expectedMultiSortBuckets = new HashMap<>();
+        addExpectedBucket(date(1, 1), 3, 1, 6);
+        addExpectedBucket(date(1, 2), 3, 2, 6);
+        addExpectedBucket(date(1, 3), 2, 3, 3);
+        addExpectedBucket(date(1, 4), 2, 3, 4);
+        addExpectedBucket(date(1, 5), 2, 5, 3);
+        addExpectedBucket(date(1, 6), 1, 5, 1);
+        addExpectedBucket(date(1, 7), 1, 5, 1);
+
+        assertAcked(client().admin().indices().prepareCreate("sort_idx")
+            .addMapping("type", "date", "type=date").get());
+        for (int i = 1; i <= 3; i++) {
+            builders.add(client().prepareIndex("sort_idx", "type").setSource(
+                jsonBuilder().startObject().field("date", date(1, 1)).field("l", 1).field("d", i).endObject()));
+            builders.add(client().prepareIndex("sort_idx", "type").setSource(
+                jsonBuilder().startObject().field("date", date(1, 2)).field("l", 2).field("d", i).endObject()));
+        }
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 3)).field("l", 3).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 3).plusHours(1)).field("l", 3).field("d", 2).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 4)).field("l", 3).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 4).plusHours(2)).field("l", 3).field("d", 3).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 5)).field("l", 5).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 5).plusHours(12)).field("l", 5).field("d", 2).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 6)).field("l", 5).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field("date", date(1, 7)).field("l", 5).field("d", 1).endObject()));
+    }
+
     @Override
     protected Collection<Class<? extends Plugin>> nodePlugins() {
         return Arrays.asList(
@@ -281,7 +337,7 @@ public class DateHistogramIT extends ESIntegTestCase {
                 .addAggregation(dateHistogram("histo")
                         .field("date")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.KEY_ASC))
+                        .order(BucketOrder.key(true)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -304,7 +360,7 @@ public class DateHistogramIT extends ESIntegTestCase {
                 .addAggregation(dateHistogram("histo")
                         .field("date")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.KEY_DESC))
+                        .order(BucketOrder.key(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -326,7 +382,7 @@ public class DateHistogramIT extends ESIntegTestCase {
                 .addAggregation(dateHistogram("histo")
                         .field("date")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.COUNT_ASC))
+                        .order(BucketOrder.count(true)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -348,7 +404,7 @@ public class DateHistogramIT extends ESIntegTestCase {
                 .addAggregation(dateHistogram("histo")
                         .field("date")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.COUNT_DESC))
+                        .order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -428,7 +484,7 @@ public class DateHistogramIT extends ESIntegTestCase {
                 .addAggregation(dateHistogram("histo")
                         .field("date")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.aggregation("sum", true))
+                        .order(BucketOrder.aggregation("sum", true))
                         .subAggregation(max("sum").field("value")))
                 .execute().actionGet();
 
@@ -451,7 +507,7 @@ public class DateHistogramIT extends ESIntegTestCase {
                 .addAggregation(dateHistogram("histo")
                         .field("date")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.aggregation("sum", false))
+                        .order(BucketOrder.aggregation("sum", false))
                         .subAggregation(max("sum").field("value")))
                 .execute().actionGet();
 
@@ -474,7 +530,7 @@ public class DateHistogramIT extends ESIntegTestCase {
                 .addAggregation(dateHistogram("histo")
                         .field("date")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.aggregation("stats", "sum", false))
+                        .order(BucketOrder.aggregation("stats", "sum", false))
                         .subAggregation(stats("stats").field("value")))
                 .execute().actionGet();
 
@@ -492,6 +548,60 @@ public class DateHistogramIT extends ESIntegTestCase {
         }
     }
 
+    public void testSingleValuedFieldOrderedByTieBreaker() throws Exception {
+        SearchResponse response = client().prepareSearch("idx")
+            .addAggregation(dateHistogram("histo")
+                .field("date")
+                .dateHistogramInterval(DateHistogramInterval.MONTH)
+                .order(BucketOrder.aggregation("max_constant", randomBoolean()))
+                .subAggregation(max("max_constant").field("constant")))
+            .execute().actionGet();
+
+        assertSearchResponse(response);
+
+        Histogram histo = response.getAggregations().get("histo");
+        assertThat(histo, notNullValue());
+        assertThat(histo.getName(), equalTo("histo"));
+        assertThat(histo.getBuckets().size(), equalTo(3));
+
+        int i = 1;
+        for (Histogram.Bucket bucket : histo.getBuckets()) {
+            assertThat(bucket.getKey(), equalTo(date(i, 1)));
+            i++;
+        }
+    }
+
+    public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception {
+        boolean asc = true;
+        try {
+            client()
+                .prepareSearch("idx")
+                .addAggregation(
+                    dateHistogram("histo").field("date")
+                        .dateHistogramInterval(DateHistogramInterval.MONTH)
+                        .order(BucketOrder.aggregation("inner_histo>avg", asc))
+                        .subAggregation(dateHistogram("inner_histo")
+                            .dateHistogramInterval(DateHistogramInterval.MONTH)
+                            .field("dates")
+                            .subAggregation(avg("avg").field("value"))))
+                .execute().actionGet();
+            fail("Expected an exception");
+        } catch (SearchPhaseExecutionException e) {
+            ElasticsearchException[] rootCauses = e.guessRootCauses();
+            if (rootCauses.length == 1) {
+                ElasticsearchException rootCause = rootCauses[0];
+                if (rootCause instanceof AggregationExecutionException) {
+                    AggregationExecutionException aggException = (AggregationExecutionException) rootCause;
+                    assertThat(aggException.getMessage(), Matchers.startsWith("Invalid aggregation order path"));
+                } else {
+                    throw e;
+                }
+            } else {
+                throw e;
+            }
+        }
+    }
+
     public void testSingleValuedFieldWithValueScript() throws Exception {
         Map<String, Object> params = new HashMap<>();
         params.put("fieldname", "date");
@@ -583,12 +693,12 @@ public class DateHistogramIT extends ESIntegTestCase {
         assertThat(bucket.getDocCount(), equalTo(3L));
     }
 
-    public void testMultiValuedFieldOrderedByKeyDesc() throws Exception {
+    public void testMultiValuedFieldOrderedByCountDesc() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(dateHistogram("histo")
                         .field("dates")
                         .dateHistogramInterval(DateHistogramInterval.MONTH)
-                        .order(Histogram.Order.COUNT_DESC))
+                        .order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -598,23 +708,26 @@ public class DateHistogramIT extends ESIntegTestCase {
         assertThat(histo.getName(), equalTo("histo"));
         assertThat(histo.getBuckets().size(), equalTo(4));
 
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
 
         Histogram.Bucket bucket = buckets.get(0);
         assertThat(bucket, notNullValue());
+        assertThat(bucket.getKey(), equalTo(date(3, 1)));
         assertThat(bucket.getDocCount(), equalTo(5L));
 
         bucket = buckets.get(1);
         assertThat(bucket, notNullValue());
+        assertThat(bucket.getKey(), equalTo(date(2, 1)));
         assertThat(bucket.getDocCount(), equalTo(3L));
 
         bucket = buckets.get(2);
         assertThat(bucket, notNullValue());
+        assertThat(bucket.getKey(), equalTo(date(4, 1)));
         assertThat(bucket.getDocCount(), equalTo(3L));
 
         bucket = buckets.get(3);
         assertThat(bucket, notNullValue());
+        assertThat(bucket.getKey(), equalTo(date(1, 1)));
         assertThat(bucket.getDocCount(), equalTo(1L));
     }
 
@@ -1236,4 +1349,75 @@ public class DateHistogramIT extends ESIntegTestCase {
         assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
                 .getMissCount(), equalTo(1L));
     }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyDesc() throws Exception {
+        int[] expectedDays = new int[] { 1, 2, 4, 3, 7, 6, 5 };
+        assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyAsc() throws Exception {
+        int[] expectedDays = new int[]  { 1, 2, 3, 4, 5, 6, 7 };
+        assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndKeyAsc() throws Exception {
+        int[] expectedDays = new int[]  { 5, 6, 7, 3, 4, 2, 1 };
+        assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
+    }
+
+    public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
+        int[] expectedDays = new int[]  { 6, 7, 3, 4, 5, 1, 2 };
+        assertMultiSortResponse(expectedDays, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
+        int[] expectedDays = new int[]  { 6, 7, 3, 5, 4, 1, 2 };
+        assertMultiSortResponse(expectedDays, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
+    }
+
+    public void testSingleValuedFieldOrderedByThreeCriteria() throws Exception {
+        int[] expectedDays = new int[]  { 2, 1, 4, 5, 3, 6, 7 };
+        assertMultiSortResponse(expectedDays, BucketOrder.count(false), BucketOrder.aggregation("sum_d", false),
+            BucketOrder.aggregation("avg_l", false));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
+        int[] expectedDays = new int[]  { 1, 2, 3, 4, 5, 6, 7 };
+        assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", true));
+    }
+
+    private void assertMultiSortResponse(int[] expectedDays, BucketOrder... order) {
+        DateTime[] expectedKeys = Arrays.stream(expectedDays).mapToObj(d -> date(1, d)).toArray(DateTime[]::new);
+        SearchResponse response = client()
+            .prepareSearch("sort_idx")
+            .setTypes("type")
+            .addAggregation(
+                dateHistogram("histo").field("date").dateHistogramInterval(DateHistogramInterval.DAY).order(BucketOrder.compound(order))
+                    .subAggregation(avg("avg_l").field("l")).subAggregation(sum("sum_d").field("d"))).execute().actionGet();
+
+        assertSearchResponse(response);
+
+        Histogram histogram = response.getAggregations().get("histo");
+        assertThat(histogram, notNullValue());
+        assertThat(histogram.getName(), equalTo("histo"));
+        assertThat(histogram.getBuckets().size(), equalTo(expectedKeys.length));
+
+        int i = 0;
+        for (Histogram.Bucket bucket : histogram.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++;
+        }
+    }
+
+    private DateTime key(Histogram.Bucket bucket) {
+        return (DateTime) bucket.getKey();
+    }
 }

+ 28 - 13
core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java

@@ -20,10 +20,13 @@
 package org.elasticsearch.search.aggregations.bucket;
 
 import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
-import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBoundsTests;
+import org.elasticsearch.search.aggregations.BucketOrder;
+
+import java.util.ArrayList;
+import java.util.List;
 
 public class DateHistogramTests extends BaseAggregationTestCase<DateHistogramAggregationBuilder> {
 
@@ -80,29 +83,41 @@ public class DateHistogramTests extends BaseAggregationTestCase<DateHistogramAgg
             factory.offset(randomIntBetween(0, 100000));
         }
         if (randomBoolean()) {
-            int branch = randomInt(5);
-            switch (branch) {
+            List<BucketOrder> order = randomOrder();
+            if(order.size() == 1 && randomBoolean()) {
+                factory.order(order.get(0));
+            } else {
+                factory.order(order);
+            }
+        }
+        return factory;
+    }
+
+    private List<BucketOrder> randomOrder() {
+        List<BucketOrder> orders = new ArrayList<>();
+        switch (randomInt(4)) {
             case 0:
-                factory.order(Order.COUNT_ASC);
+                orders.add(BucketOrder.key(randomBoolean()));
                 break;
             case 1:
-                factory.order(Order.COUNT_DESC);
+                orders.add(BucketOrder.count(randomBoolean()));
                 break;
             case 2:
-                factory.order(Order.KEY_ASC);
+                orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomBoolean()));
                 break;
             case 3:
-                factory.order(Order.KEY_DESC);
+                orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomAlphaOfLengthBetween(3, 20), randomBoolean()));
                 break;
             case 4:
-                factory.order(Order.aggregation("foo", true));
-                break;
-            case 5:
-                factory.order(Order.aggregation("foo", false));
+                int numOrders = randomIntBetween(1, 3);
+                for (int i = 0; i < numOrders; i++) {
+                    orders.addAll(randomOrder());
+                }
                 break;
-            }
+            default:
+                fail();
         }
-        return factory;
+        return orders;
     }
 
 }

+ 2 - 1
core/src/test/java/org/elasticsearch/search/aggregations/bucket/DiversifiedSamplerIT.java

@@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.max.Max;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.Collection;
@@ -103,7 +104,7 @@ public class DiversifiedSamplerIT extends ESIntegTestCase {
         SearchResponse response = client().prepareSearch("test").setTypes("book").setSearchType(SearchType.QUERY_THEN_FETCH)
                 .addAggregation(terms("genres")
                         .field("genre")
-                        .order(Terms.Order.aggregation("sample>max_price.value", asc))
+                        .order(BucketOrder.aggregation("sample>max_price.value", asc))
                         .subAggregation(sampler("sample").shardSize(100)
                                 .subAggregation(max("max_price").field("price")))
                 ).execute().actionGet();

+ 59 - 30
core/src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsIT.java

@@ -41,6 +41,7 @@ import org.elasticsearch.search.aggregations.metrics.max.Max;
 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.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.hamcrest.Matchers;
 
@@ -134,6 +135,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                     .startObject()
                     .field(SINGLE_VALUED_FIELD_NAME, (double) i)
                     .field("num_tag", i < NUM_DOCS/2 + 1 ? 1 : 0) // used to test order by single-bucket sub agg
+                    .field("constant", 1)
                     .startArray(MULTI_VALUED_FIELD_NAME).value((double) i).value(i + 1d).endArray()
                     .endObject()));
 
@@ -315,7 +317,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .size(20)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
+                        .order(BucketOrder.key(true))) // we need to sort by terms cause we're checking the first 20 values
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -363,15 +365,15 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
             assertThat(bucket.getDocCount(), equalTo(1L));
         }
     }
-    
+
     public void testSingleValueFieldWithPartitionedFiltering() throws Exception {
         runTestFieldWithPartitionedFiltering(SINGLE_VALUED_FIELD_NAME);
     }
-    
+
     public void testMultiValueFieldWithPartitionedFiltering() throws Exception {
         runTestFieldWithPartitionedFiltering(MULTI_VALUED_FIELD_NAME);
     }
-    
+
     private void runTestFieldWithPartitionedFiltering(String field) throws Exception {
         // Find total number of unique terms
         SearchResponse allResponse = client().prepareSearch("idx")
@@ -399,14 +401,14 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
             }
         }
         assertEquals(expectedCardinality, foundTerms.size());
-    }   
+    }
 
     public void testSingleValueFieldOrderedByTermAsc() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.term(true)))
+                        .order(BucketOrder.key(true)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -432,7 +434,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.term(false)))
+                        .order(BucketOrder.key(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -453,6 +455,33 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
         }
     }
 
+    public void testSingleValueFieldOrderedByTieBreaker() throws Exception {
+        SearchResponse response = client().prepareSearch("idx").setTypes("type")
+            .addAggregation(terms("terms")
+                .field(SINGLE_VALUED_FIELD_NAME)
+                .collectMode(randomFrom(SubAggCollectionMode.values()))
+                .order(BucketOrder.aggregation("max_constant", randomBoolean()))
+                .subAggregation(max("max_constant").field("constant")))
+            .execute().actionGet();
+
+        assertSearchResponse(response);
+
+
+        Terms terms = response.getAggregations().get("terms");
+        assertThat(terms, notNullValue());
+        assertThat(terms.getName(), equalTo("terms"));
+        assertThat(terms.getBuckets().size(), equalTo(5));
+
+        int i = 0;
+        for (Terms.Bucket bucket : terms.getBuckets()) {
+            assertThat(bucket, notNullValue());
+            assertThat(key(bucket), equalTo("" + (double)i));
+            assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
+            assertThat(bucket.getDocCount(), equalTo(1L));
+            i++;
+        }
+    }
+
     public void testSingleValuedFieldWithSubAggregation() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(terms("terms")
@@ -759,7 +788,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 .prepareSearch("idx")
                 .addAggregation(
                         terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
+                                .order(BucketOrder.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
 
@@ -789,7 +818,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                         terms("terms")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("avg_i", asc))
+                                .order(BucketOrder.aggregation("avg_i", asc))
                                 .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                                 .subAggregation(
                                         terms("subTerms").field(MULTI_VALUED_FIELD_NAME).collectMode(
@@ -831,7 +860,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 .prepareSearch("idx")
                 .addAggregation(
                         terms("num_tags").field("num_tag").collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("filter", asc))
+                                .order(BucketOrder.aggregation("filter", asc))
                                 .subAggregation(filter("filter", QueryBuilders.matchAllQuery()))).execute().actionGet();
 
 
@@ -869,7 +898,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                         terms("tags")
                                 .field("num_tag")
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("filter1>filter2>max", asc))
+                                .order(BucketOrder.aggregation("filter1>filter2>max", asc))
                                 .subAggregation(
                                         filter("filter1", QueryBuilders.matchAllQuery()).subAggregation(
                                                 filter("filter2", QueryBuilders.matchAllQuery()).subAggregation(
@@ -923,7 +952,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 client().prepareSearch(index)
                         .addAggregation(
                                 terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("avg_i", true))).execute().actionGet();
+                                        .order(BucketOrder.aggregation("avg_i", true))).execute().actionGet();
 
                 fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
 
@@ -941,7 +970,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                                 terms("terms")
                                         .field(SINGLE_VALUED_FIELD_NAME)
                                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("num_tags", true))
+                                        .order(BucketOrder.aggregation("num_tags", true))
                                         .subAggregation(
                                                 terms("num_tags").field("num_tags").collectMode(randomFrom(SubAggCollectionMode.values()))))
                         .execute().actionGet();
@@ -960,7 +989,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 client().prepareSearch(index)
                         .addAggregation(
                                 terms("terms").field(SINGLE_VALUED_FIELD_NAME + "2").collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("stats.foo", true))
+                                        .order(BucketOrder.aggregation("stats.foo", true))
                                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
 
                 fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
@@ -978,7 +1007,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 client().prepareSearch(index)
                         .addAggregation(
                                 terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("stats", true))
+                                        .order(BucketOrder.aggregation("stats", true))
                                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
 
                 fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
@@ -996,7 +1025,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 .prepareSearch("idx")
                 .addAggregation(
                         terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
+                                .order(BucketOrder.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
 
@@ -1026,7 +1055,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 .prepareSearch("idx")
                 .addAggregation(
                         terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("stats.avg", asc))
+                                .order(BucketOrder.aggregation("stats.avg", asc))
                                 .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1054,7 +1083,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 .prepareSearch("idx")
                 .addAggregation(
                         terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("stats.avg", asc))
+                                .order(BucketOrder.aggregation("stats.avg", asc))
                                 .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1082,7 +1111,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
                 .prepareSearch("idx")
                 .addAggregation(
                         terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("stats.variance", asc))
+                                .order(BucketOrder.aggregation("stats.variance", asc))
                                 .subAggregation(extendedStats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1139,48 +1168,48 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsDesc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndTermsAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
     }
 
     public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
     }
 
     public void testSingleValuedFieldOrderedByThreeCriteria() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(false),
+                BucketOrder.aggregation("sum_d", false),
+                BucketOrder.aggregation("avg_l", false));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
         double[] expectedKeys = new double[] { 1, 2, 3, 4, 5, 6, 7 };
-        assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true));
     }
 
-    private void assertMultiSortResponse(double[] expectedKeys, Terms.Order... order) {
+    private void assertMultiSortResponse(double[] expectedKeys, BucketOrder... 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"))
+                                .order(BucketOrder.compound(order)).subAggregation(avg("avg_l").field("l"))
                                 .subAggregation(sum("sum_d").field("d"))).execute().actionGet();
 
         assertSearchResponse(response);

+ 192 - 20
core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramIT.java

@@ -19,7 +19,9 @@
 package org.elasticsearch.search.aggregations.bucket;
 
 import com.carrotsearch.hppc.LongHashSet;
+import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.action.search.SearchPhaseExecutionException;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.query.QueryBuilders;
@@ -27,16 +29,20 @@ import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.script.MockScriptPlugin;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.script.ScriptType;
+import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
+import org.elasticsearch.search.aggregations.metrics.avg.Avg;
 import org.elasticsearch.search.aggregations.metrics.max.Max;
 import org.elasticsearch.search.aggregations.metrics.stats.Stats;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.hamcrest.Matchers;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -48,6 +54,7 @@ import java.util.function.Function;
 import static java.util.Collections.emptyMap;
 import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
@@ -72,6 +79,7 @@ public class HistogramIT extends ESIntegTestCase {
     static int interval;
     static int numValueBuckets, numValuesBuckets;
     static long[] valueCounts, valuesCounts;
+    static Map<Long, Map<String, Object>> expectedMultiSortBuckets;
 
     @Override
     protected Collection<Class<? extends Plugin>> nodePlugins() {
@@ -130,16 +138,18 @@ public class HistogramIT extends ESIntegTestCase {
         }
         List<IndexRequestBuilder> builders = new ArrayList<>();
 
-
         for (int i = 0; i < numDocs; i++) {
             builders.add(client().prepareIndex("idx", "type").setSource(jsonBuilder()
                     .startObject()
                     .field(SINGLE_VALUED_FIELD_NAME, i + 1)
                     .startArray(MULTI_VALUED_FIELD_NAME).value(i + 1).value(i + 2).endArray()
                     .field("tag", "tag" + i)
+                    .field("constant", 1)
                     .endObject()));
         }
 
+        getMultiSortDocs(builders);
+
         assertAcked(prepareCreate("empty_bucket_idx").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=integer"));
         for (int i = 0; i < 2; i++) {
             builders.add(client().prepareIndex("empty_bucket_idx", "type", "" + i).setSource(jsonBuilder()
@@ -151,6 +161,51 @@ public class HistogramIT extends ESIntegTestCase {
         ensureSearchable();
     }
 
+    private void addExpectedBucket(long key, long docCount, double avg, double sum) {
+        Map<String, Object> bucketProps = new HashMap<>();
+        bucketProps.put("key", key);
+        bucketProps.put("_count", docCount);
+        bucketProps.put("avg_l", avg);
+        bucketProps.put("sum_d", sum);
+        expectedMultiSortBuckets.put(key, bucketProps);
+    }
+
+    private void getMultiSortDocs(List<IndexRequestBuilder> builders) throws IOException {
+        expectedMultiSortBuckets = new HashMap<>();
+        addExpectedBucket(1, 3, 1, 6);
+        addExpectedBucket(2, 3, 2, 6);
+        addExpectedBucket(3, 2, 3, 3);
+        addExpectedBucket(4, 2, 3, 4);
+        addExpectedBucket(5, 2, 5, 3);
+        addExpectedBucket(6, 1, 5, 1);
+        addExpectedBucket(7, 1, 5, 1);
+
+        assertAcked(client().admin().indices().prepareCreate("sort_idx")
+            .addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=double").get());
+        for (int i = 1; i <= 3; i++) {
+            builders.add(client().prepareIndex("sort_idx", "type").setSource(
+                jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 1).field("l", 1).field("d", i).endObject()));
+            builders.add(client().prepareIndex("sort_idx", "type").setSource(
+                jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 2).field("l", 2).field("d", i).endObject()));
+        }
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 3).field("l", 3).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 3.8).field("l", 3).field("d", 2).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 4).field("l", 3).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 4.4).field("l", 3).field("d", 3).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 5).field("l", 5).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 5.1).field("l", 5).field("d", 2).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 6).field("l", 5).field("d", 1).endObject()));
+        builders.add(client().prepareIndex("sort_idx", "type").setSource(
+            jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 7).field("l", 5).field("d", 1).endObject()));
+    }
+
     public void testSingleValuedField() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval))
@@ -241,7 +296,7 @@ public class HistogramIT extends ESIntegTestCase {
 
     public void testSingleValuedFieldOrderedByKeyAsc() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.KEY_ASC))
+                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.key(true)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -252,7 +307,6 @@ public class HistogramIT extends ESIntegTestCase {
         assertThat(histo.getName(), equalTo("histo"));
         assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
 
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(i);
@@ -264,7 +318,7 @@ public class HistogramIT extends ESIntegTestCase {
 
     public void testsingleValuedFieldOrderedByKeyDesc() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.KEY_DESC))
+                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.key(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -275,7 +329,6 @@ public class HistogramIT extends ESIntegTestCase {
         assertThat(histo.getName(), equalTo("histo"));
         assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
 
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(numValueBuckets - i - 1);
@@ -287,7 +340,7 @@ public class HistogramIT extends ESIntegTestCase {
 
     public void testSingleValuedFieldOrderedByCountAsc() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.COUNT_ASC))
+                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.count(true)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -299,7 +352,6 @@ public class HistogramIT extends ESIntegTestCase {
         assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
 
         LongHashSet buckets = new LongHashSet();
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> histoBuckets = new ArrayList<>(histo.getBuckets());
         long previousCount = Long.MIN_VALUE;
         for (int i = 0; i < numValueBuckets; ++i) {
@@ -316,7 +368,7 @@ public class HistogramIT extends ESIntegTestCase {
 
     public void testSingleValuedFieldOrderedByCountDesc() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.COUNT_DESC))
+                .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -328,7 +380,6 @@ public class HistogramIT extends ESIntegTestCase {
         assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
 
         LongHashSet buckets = new LongHashSet();
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> histoBuckets = new ArrayList<>(histo.getBuckets());
         long previousCount = Long.MAX_VALUE;
         for (int i = 0; i < numValueBuckets; ++i) {
@@ -361,7 +412,6 @@ public class HistogramIT extends ESIntegTestCase {
         Object[] propertiesDocCounts = (Object[]) ((InternalAggregation)histo).getProperty("_count");
         Object[] propertiesCounts = (Object[]) ((InternalAggregation)histo).getProperty("sum.value");
 
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(i);
@@ -390,7 +440,7 @@ public class HistogramIT extends ESIntegTestCase {
                         histogram("histo")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .interval(interval)
-                                .order(Histogram.Order.aggregation("sum", true))
+                                .order(BucketOrder.aggregation("sum", true))
                         .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
@@ -404,7 +454,6 @@ public class HistogramIT extends ESIntegTestCase {
 
         LongHashSet visited = new LongHashSet();
         double previousSum = Double.NEGATIVE_INFINITY;
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(i);
@@ -434,7 +483,7 @@ public class HistogramIT extends ESIntegTestCase {
                         histogram("histo")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .interval(interval)
-                                .order(Histogram.Order.aggregation("sum", false))
+                                .order(BucketOrder.aggregation("sum", false))
                         .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
@@ -448,7 +497,6 @@ public class HistogramIT extends ESIntegTestCase {
 
         LongHashSet visited = new LongHashSet();
         double previousSum = Double.POSITIVE_INFINITY;
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(i);
@@ -478,7 +526,7 @@ public class HistogramIT extends ESIntegTestCase {
                         histogram("histo")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .interval(interval)
-                                .order(Histogram.Order.aggregation("stats.sum", false))
+                                .order(BucketOrder.aggregation("stats.sum", false))
                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
@@ -492,7 +540,7 @@ public class HistogramIT extends ESIntegTestCase {
 
         LongHashSet visited = new LongHashSet();
         double previousSum = Double.POSITIVE_INFINITY;
-        // TODO: use diamond once JI-9019884 is fixed
+
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(i);
@@ -523,7 +571,7 @@ public class HistogramIT extends ESIntegTestCase {
                         histogram("histo")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .interval(interval)
-                                .order(Histogram.Order.aggregation("filter>max", asc))
+                                .order(BucketOrder.aggregation("filter>max", asc))
                         .subAggregation(filter("filter", matchAllQuery())
                         .subAggregation(max("max").field(SINGLE_VALUED_FIELD_NAME))))
                 .execute().actionGet();
@@ -538,7 +586,6 @@ public class HistogramIT extends ESIntegTestCase {
 
         LongHashSet visited = new LongHashSet();
         double prevMax = asc ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY;
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(i);
@@ -558,6 +605,62 @@ public class HistogramIT extends ESIntegTestCase {
         }
     }
 
+    public void testSingleValuedFieldOrderedByTieBreaker() throws Exception {
+        SearchResponse response = client().prepareSearch("idx")
+            .addAggregation(histogram("histo")
+                .field(SINGLE_VALUED_FIELD_NAME)
+                .interval(interval)
+                .order(BucketOrder.aggregation("max_constant", randomBoolean()))
+                .subAggregation(max("max_constant").field("constant")))
+            .execute().actionGet();
+
+        assertSearchResponse(response);
+
+        Histogram histo = response.getAggregations().get("histo");
+        assertThat(histo, notNullValue());
+        assertThat(histo.getName(), equalTo("histo"));
+        assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
+
+        List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
+        for (int i = 0; i < numValueBuckets; ++i) {
+            Histogram.Bucket bucket = buckets.get(i);
+            assertThat(bucket, notNullValue());
+            assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) i * interval));
+            assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
+        }
+    }
+
+    public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception {
+        boolean asc = true;
+        try {
+            client()
+                .prepareSearch("idx")
+                .addAggregation(
+                    histogram("histo").field(SINGLE_VALUED_FIELD_NAME)
+                        .interval(interval)
+                        .order(BucketOrder.aggregation("inner_histo>avg", asc))
+                        .subAggregation(histogram("inner_histo")
+                            .interval(interval)
+                            .field(MULTI_VALUED_FIELD_NAME)
+                            .subAggregation(avg("avg").field("value"))))
+                .execute().actionGet();
+            fail("Expected an exception");
+        } catch (SearchPhaseExecutionException e) {
+            ElasticsearchException[] rootCauses = e.guessRootCauses();
+            if (rootCauses.length == 1) {
+                ElasticsearchException rootCause = rootCauses[0];
+                if (rootCause instanceof AggregationExecutionException) {
+                    AggregationExecutionException aggException = (AggregationExecutionException) rootCause;
+                    assertThat(aggException.getMessage(), Matchers.startsWith("Invalid aggregation order path"));
+                } else {
+                    throw e;
+                }
+            } else {
+                throw e;
+            }
+        }
+    }
+
     public void testSingleValuedFieldWithValueScript() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(
@@ -614,7 +717,7 @@ public class HistogramIT extends ESIntegTestCase {
 
     public void testMultiValuedFieldOrderedByKeyDesc() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(histogram("histo").field(MULTI_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.KEY_DESC))
+                .addAggregation(histogram("histo").field(MULTI_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.key(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -625,7 +728,6 @@ public class HistogramIT extends ESIntegTestCase {
         assertThat(histo.getName(), equalTo("histo"));
         assertThat(histo.getBuckets().size(), equalTo(numValuesBuckets));
 
-        // TODO: use diamond once JI-9019884 is fixed
         List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
         for (int i = 0; i < numValuesBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(numValuesBuckets - i - 1);
@@ -1036,4 +1138,74 @@ public class HistogramIT extends ESIntegTestCase {
         assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
                 .getMissCount(), equalTo(1L));
     }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyDesc() throws Exception {
+        long[] expectedKeys = new long[] { 1, 2, 4, 3, 7, 6, 5 };
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyAsc() throws Exception {
+        long[] expectedKeys = new long[]  { 1, 2, 3, 4, 5, 6, 7 };
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndKeyAsc() throws Exception {
+        long[] expectedKeys = new long[]  { 5, 6, 7, 3, 4, 2, 1 };
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
+    }
+
+    public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
+        long[] expectedKeys = new long[]  { 6, 7, 3, 4, 5, 1, 2 };
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
+        long[] expectedKeys = new long[]  { 6, 7, 3, 5, 4, 1, 2 };
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
+    }
+
+    public void testSingleValuedFieldOrderedByThreeCriteria() throws Exception {
+        long[] expectedKeys = new long[]  { 2, 1, 4, 5, 3, 6, 7 };
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(false), BucketOrder.aggregation("sum_d", false),
+            BucketOrder.aggregation("avg_l", false));
+    }
+
+    public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
+        long[] expectedKeys = new long[]  { 1, 2, 3, 4, 5, 6, 7 };
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true));
+    }
+
+    private void assertMultiSortResponse(long[] expectedKeys, BucketOrder... order) {
+        SearchResponse response = client()
+            .prepareSearch("sort_idx")
+            .setTypes("type")
+            .addAggregation(
+                histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).order(BucketOrder.compound(order))
+                    .subAggregation(avg("avg_l").field("l")).subAggregation(sum("sum_d").field("d"))).execute().actionGet();
+
+        assertSearchResponse(response);
+
+        Histogram histogram = response.getAggregations().get("histo");
+        assertThat(histogram, notNullValue());
+        assertThat(histogram.getName(), equalTo("histo"));
+        assertThat(histogram.getBuckets().size(), equalTo(expectedKeys.length));
+
+        int i = 0;
+        for (Histogram.Bucket bucket : histogram.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++;
+        }
+    }
+
+    private long key(Histogram.Bucket bucket) {
+        return ((Number) bucket.getKey()).longValue();
+    }
 }

+ 36 - 21
core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java

@@ -21,7 +21,10 @@ package org.elasticsearch.search.aggregations.bucket;
 
 import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
 import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
+import org.elasticsearch.search.aggregations.BucketOrder;
+
+import java.util.ArrayList;
+import java.util.List;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.startsWith;
@@ -54,26 +57,11 @@ public class HistogramTests extends BaseAggregationTestCase<HistogramAggregation
             factory.offset(randomIntBetween(0, 100000));
         }
         if (randomBoolean()) {
-            int branch = randomInt(5);
-            switch (branch) {
-            case 0:
-                factory.order(Order.COUNT_ASC);
-                break;
-            case 1:
-                factory.order(Order.COUNT_DESC);
-                break;
-            case 2:
-                factory.order(Order.KEY_ASC);
-                break;
-            case 3:
-                factory.order(Order.KEY_DESC);
-                break;
-            case 4:
-                factory.order(Order.aggregation("foo", true));
-                break;
-            case 5:
-                factory.order(Order.aggregation("foo", false));
-                break;
+            List<BucketOrder> order = randomOrder();
+            if(order.size() == 1 && randomBoolean()) {
+                factory.order(order.get(0));
+            } else {
+                factory.order(order);
             }
         }
         return factory;
@@ -102,4 +90,31 @@ public class HistogramTests extends BaseAggregationTestCase<HistogramAggregation
         assertThat(ex.getMessage(), equalTo("maxBound [0.4] must be greater than minBound [0.5]"));
     }
 
+    private List<BucketOrder> randomOrder() {
+        List<BucketOrder> orders = new ArrayList<>();
+        switch (randomInt(4)) {
+            case 0:
+                orders.add(BucketOrder.key(randomBoolean()));
+                break;
+            case 1:
+                orders.add(BucketOrder.count(randomBoolean()));
+                break;
+            case 2:
+                orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomBoolean()));
+                break;
+            case 3:
+                orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomAlphaOfLengthBetween(3, 20), randomBoolean()));
+                break;
+            case 4:
+                int numOrders = randomIntBetween(1, 3);
+                for (int i = 0; i < numOrders; i++) {
+                    orders.addAll(randomOrder());
+                }
+                break;
+            default:
+                fail();
+        }
+        return orders;
+    }
+
 }

+ 53 - 26
core/src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsIT.java

@@ -40,6 +40,7 @@ import org.elasticsearch.search.aggregations.metrics.max.Max;
 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.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.hamcrest.Matchers;
 
@@ -121,6 +122,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                     .field(SINGLE_VALUED_FIELD_NAME, i)
                     .startArray(MULTI_VALUED_FIELD_NAME).value(i).value(i + 1).endArray()
                     .field("num_tag", i < lowCardBuilders.length / 2 + 1 ? 1 : 0) // used to test order by single-bucket sub agg
+                    .field("constant", 1)
                     .endObject());
         }
         indexRandom(true, lowCardBuilders);
@@ -392,7 +394,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .size(20)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
+                        .order(BucketOrder.key(true))) // we need to sort by terms cause we're checking the first 20 values
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -417,7 +419,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.term(true)))
+                        .order(BucketOrder.key(true)))
                 .execute().actionGet();
         assertSearchResponse(response);
 
@@ -441,7 +443,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.term(false)))
+                        .order(BucketOrder.key(false)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -462,6 +464,31 @@ public class LongTermsIT extends AbstractTermsTestCase {
         }
     }
 
+    public void testSingleValueFieldOrderedByTieBreaker() throws Exception {
+        SearchResponse response = client().prepareSearch("idx").setTypes("type")
+            .addAggregation(terms("terms")
+                .field(SINGLE_VALUED_FIELD_NAME)
+                .collectMode(randomFrom(SubAggCollectionMode.values()))
+                .order(BucketOrder.aggregation("max_constant", randomBoolean()))
+                .subAggregation(max("max_constant").field("constant")))
+            .execute().actionGet();
+        assertSearchResponse(response);
+
+        Terms terms = response.getAggregations().get("terms");
+        assertThat(terms, notNullValue());
+        assertThat(terms.getName(), equalTo("terms"));
+        assertThat(terms.getBuckets().size(), equalTo(5));
+
+        int i = 0;
+        for (Terms.Bucket bucket : terms.getBuckets()) {
+            assertThat(bucket, notNullValue());
+            assertThat(key(bucket), equalTo("" + i));
+            assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
+            assertThat(bucket.getDocCount(), equalTo(1L));
+            i++;
+        }
+    }
+
     public void testSingleValuedFieldWithSubAggregation() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(terms("terms")
@@ -769,7 +796,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("avg_i", asc))
+                        .order(BucketOrder.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
 
@@ -798,7 +825,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("avg_i", asc))
+                        .order(BucketOrder.aggregation("avg_i", asc))
                         .subAggregation(
                                 avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                         .subAggregation(
@@ -842,7 +869,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("num_tags")
                         .field("num_tag")
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("filter", asc))
+                        .order(BucketOrder.aggregation("filter", asc))
                         .subAggregation(filter("filter", QueryBuilders.matchAllQuery()))
                 ).get();
 
@@ -879,7 +906,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("tags")
                         .field("num_tag")
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("filter1>filter2>max", asc))
+                        .order(BucketOrder.aggregation("filter1>filter2>max", asc))
                 .subAggregation(filter("filter1", QueryBuilders.matchAllQuery()).subAggregation(
                         filter("filter2", QueryBuilders.matchAllQuery())
                                         .subAggregation(max("max").field(SINGLE_VALUED_FIELD_NAME))))
@@ -934,7 +961,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                         .addAggregation(terms("terms")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("avg_i", true))
+                                .order(BucketOrder.aggregation("avg_i", true))
                         ).execute().actionGet();
 
                 fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
@@ -952,7 +979,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                         .addAggregation(terms("terms")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("num_tags", true))
+                                .order(BucketOrder.aggregation("num_tags", true))
                                 .subAggregation(terms("num_tags").field("num_tags")
                                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                         ).execute().actionGet();
@@ -972,7 +999,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                         .addAggregation(terms("terms")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("stats.foo", true))
+                                .order(BucketOrder.aggregation("stats.foo", true))
                                 .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                         ).execute().actionGet();
 
@@ -992,7 +1019,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                         .addAggregation(terms("terms")
                                 .field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("stats", true))
+                                .order(BucketOrder.aggregation("stats", true))
                                 .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                         ).execute().actionGet();
 
@@ -1011,7 +1038,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("avg_i", asc))
+                        .order(BucketOrder.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
 
@@ -1043,7 +1070,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("stats.avg", asc))
+                        .order(BucketOrder.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
 
@@ -1073,7 +1100,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("stats.avg", asc))
+                        .order(BucketOrder.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
 
@@ -1103,7 +1130,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                        .order(Terms.Order.aggregation("stats.variance", asc))
+                        .order(BucketOrder.aggregation("stats.variance", asc))
                         .subAggregation(extendedStats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
 
@@ -1129,47 +1156,47 @@ public class LongTermsIT extends AbstractTermsTestCase {
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsDesc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndTermsAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
     }
 
     public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
     }
 
     public void testSingleValuedFieldOrderedByThreeCriteria() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(false),
+                BucketOrder.aggregation("sum_d", false),
+                BucketOrder.aggregation("avg_l", false));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
         long[] expectedKeys = new long[] { 1, 2, 3, 4, 5, 6, 7 };
-        assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true));
     }
 
-    private void assertMultiSortResponse(long[] expectedKeys, Terms.Order... order) {
+    private void assertMultiSortResponse(long[] expectedKeys, BucketOrder... 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))
+                        .order(BucketOrder.compound(order))
                         .subAggregation(avg("avg_l").field("l"))
                         .subAggregation(sum("sum_d").field("d"))
                 ).execute().actionGet();

+ 41 - 40
core/src/test/java/org/elasticsearch/search/aggregations/bucket/MinDocCountIT.java

@@ -37,6 +37,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
@@ -190,122 +191,122 @@ public class MinDocCountIT extends AbstractTermsTestCase {
     }
 
     public void testStringTermAsc() throws Exception {
-        testMinDocCountOnTerms("s", Script.NO, Terms.Order.term(true));
+        testMinDocCountOnTerms("s", Script.NO, BucketOrder.key(true));
     }
 
     public void testStringScriptTermAsc() throws Exception {
-        testMinDocCountOnTerms("s", Script.YES, Terms.Order.term(true));
+        testMinDocCountOnTerms("s", Script.YES, BucketOrder.key(true));
     }
 
     public void testStringTermDesc() throws Exception {
-        testMinDocCountOnTerms("s", Script.NO, Terms.Order.term(false));
+        testMinDocCountOnTerms("s", Script.NO, BucketOrder.key(false));
     }
 
     public void testStringScriptTermDesc() throws Exception {
-        testMinDocCountOnTerms("s", Script.YES, Terms.Order.term(false));
+        testMinDocCountOnTerms("s", Script.YES, BucketOrder.key(false));
     }
 
     public void testStringCountAsc() throws Exception {
-        testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(true));
+        testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(true));
     }
 
     public void testStringScriptCountAsc() throws Exception {
-        testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(true));
+        testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(true));
     }
 
     public void testStringCountDesc() throws Exception {
-        testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(false));
+        testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(false));
     }
 
     public void testStringScriptCountDesc() throws Exception {
-        testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(false));
+        testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(false));
     }
 
     public void testStringCountAscWithInclude() throws Exception {
-        testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(true), ".*a.*", true);
+        testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(true), ".*a.*", true);
     }
 
     public void testStringScriptCountAscWithInclude() throws Exception {
-        testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(true), ".*a.*", true);
+        testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(true), ".*a.*", true);
     }
 
     public void testStringCountDescWithInclude() throws Exception {
-        testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(false), ".*a.*", true);
+        testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(false), ".*a.*", true);
     }
 
     public void testStringScriptCountDescWithInclude() throws Exception {
-        testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(false), ".*a.*", true);
+        testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(false), ".*a.*", true);
     }
 
     public void testLongTermAsc() throws Exception {
-        testMinDocCountOnTerms("l", Script.NO, Terms.Order.term(true));
+        testMinDocCountOnTerms("l", Script.NO, BucketOrder.key(true));
     }
 
     public void testLongScriptTermAsc() throws Exception {
-        testMinDocCountOnTerms("l", Script.YES, Terms.Order.term(true));
+        testMinDocCountOnTerms("l", Script.YES, BucketOrder.key(true));
     }
 
     public void testLongTermDesc() throws Exception {
-        testMinDocCountOnTerms("l", Script.NO, Terms.Order.term(false));
+        testMinDocCountOnTerms("l", Script.NO, BucketOrder.key(false));
     }
 
     public void testLongScriptTermDesc() throws Exception {
-        testMinDocCountOnTerms("l", Script.YES, Terms.Order.term(false));
+        testMinDocCountOnTerms("l", Script.YES, BucketOrder.key(false));
     }
 
     public void testLongCountAsc() throws Exception {
-        testMinDocCountOnTerms("l", Script.NO, Terms.Order.count(true));
+        testMinDocCountOnTerms("l", Script.NO, BucketOrder.count(true));
     }
 
     public void testLongScriptCountAsc() throws Exception {
-        testMinDocCountOnTerms("l", Script.YES, Terms.Order.count(true));
+        testMinDocCountOnTerms("l", Script.YES, BucketOrder.count(true));
     }
 
     public void testLongCountDesc() throws Exception {
-        testMinDocCountOnTerms("l", Script.NO, Terms.Order.count(false));
+        testMinDocCountOnTerms("l", Script.NO, BucketOrder.count(false));
     }
 
     public void testLongScriptCountDesc() throws Exception {
-        testMinDocCountOnTerms("l", Script.YES, Terms.Order.count(false));
+        testMinDocCountOnTerms("l", Script.YES, BucketOrder.count(false));
     }
 
     public void testDoubleTermAsc() throws Exception {
-        testMinDocCountOnTerms("d", Script.NO, Terms.Order.term(true));
+        testMinDocCountOnTerms("d", Script.NO, BucketOrder.key(true));
     }
 
     public void testDoubleScriptTermAsc() throws Exception {
-        testMinDocCountOnTerms("d", Script.YES, Terms.Order.term(true));
+        testMinDocCountOnTerms("d", Script.YES, BucketOrder.key(true));
     }
 
     public void testDoubleTermDesc() throws Exception {
-        testMinDocCountOnTerms("d", Script.NO, Terms.Order.term(false));
+        testMinDocCountOnTerms("d", Script.NO, BucketOrder.key(false));
     }
 
     public void testDoubleScriptTermDesc() throws Exception {
-        testMinDocCountOnTerms("d", Script.YES, Terms.Order.term(false));
+        testMinDocCountOnTerms("d", Script.YES, BucketOrder.key(false));
     }
 
     public void testDoubleCountAsc() throws Exception {
-        testMinDocCountOnTerms("d", Script.NO, Terms.Order.count(true));
+        testMinDocCountOnTerms("d", Script.NO, BucketOrder.count(true));
     }
 
     public void testDoubleScriptCountAsc() throws Exception {
-        testMinDocCountOnTerms("d", Script.YES, Terms.Order.count(true));
+        testMinDocCountOnTerms("d", Script.YES, BucketOrder.count(true));
     }
 
     public void testDoubleCountDesc() throws Exception {
-        testMinDocCountOnTerms("d", Script.NO, Terms.Order.count(false));
+        testMinDocCountOnTerms("d", Script.NO, BucketOrder.count(false));
     }
 
     public void testDoubleScriptCountDesc() throws Exception {
-        testMinDocCountOnTerms("d", Script.YES, Terms.Order.count(false));
+        testMinDocCountOnTerms("d", Script.YES, BucketOrder.count(false));
     }
 
-    private void testMinDocCountOnTerms(String field, Script script, Terms.Order order) throws Exception {
+    private void testMinDocCountOnTerms(String field, Script script, BucketOrder order) throws Exception {
         testMinDocCountOnTerms(field, script, order, null, true);
     }
 
-    private void testMinDocCountOnTerms(String field, Script script, Terms.Order order, String include, boolean retry) throws Exception {
+    private void testMinDocCountOnTerms(String field, Script script, BucketOrder order, String include, boolean retry) throws Exception {
         // all terms
         final SearchResponse allTermsResponse = client().prepareSearch("idx").setTypes("type")
                 .setSize(0)
@@ -342,38 +343,38 @@ public class MinDocCountIT extends AbstractTermsTestCase {
     }
 
     public void testHistogramCountAsc() throws Exception {
-        testMinDocCountOnHistogram(Histogram.Order.COUNT_ASC);
+        testMinDocCountOnHistogram(BucketOrder.count(true));
     }
 
     public void testHistogramCountDesc() throws Exception {
-        testMinDocCountOnHistogram(Histogram.Order.COUNT_DESC);
+        testMinDocCountOnHistogram(BucketOrder.count(false));
     }
 
     public void testHistogramKeyAsc() throws Exception {
-        testMinDocCountOnHistogram(Histogram.Order.KEY_ASC);
+        testMinDocCountOnHistogram(BucketOrder.key(true));
     }
 
     public void testHistogramKeyDesc() throws Exception {
-        testMinDocCountOnHistogram(Histogram.Order.KEY_DESC);
+        testMinDocCountOnHistogram(BucketOrder.key(false));
     }
 
     public void testDateHistogramCountAsc() throws Exception {
-        testMinDocCountOnDateHistogram(Histogram.Order.COUNT_ASC);
+        testMinDocCountOnDateHistogram(BucketOrder.count(true));
     }
 
     public void testDateHistogramCountDesc() throws Exception {
-        testMinDocCountOnDateHistogram(Histogram.Order.COUNT_DESC);
+        testMinDocCountOnDateHistogram(BucketOrder.count(false));
     }
 
     public void testDateHistogramKeyAsc() throws Exception {
-        testMinDocCountOnDateHistogram(Histogram.Order.KEY_ASC);
+        testMinDocCountOnDateHistogram(BucketOrder.key(true));
     }
 
     public void testDateHistogramKeyDesc() throws Exception {
-        testMinDocCountOnDateHistogram(Histogram.Order.KEY_DESC);
+        testMinDocCountOnDateHistogram(BucketOrder.key(false));
     }
 
-    private void testMinDocCountOnHistogram(Histogram.Order order) throws Exception {
+    private void testMinDocCountOnHistogram(BucketOrder order) throws Exception {
         final int interval = randomIntBetween(1, 3);
         final SearchResponse allResponse = client().prepareSearch("idx").setTypes("type")
                 .setSize(0)
@@ -393,7 +394,7 @@ public class MinDocCountIT extends AbstractTermsTestCase {
         }
     }
 
-    private void testMinDocCountOnDateHistogram(Histogram.Order order) throws Exception {
+    private void testMinDocCountOnDateHistogram(BucketOrder order) throws Exception {
         final SearchResponse allResponse = client().prepareSearch("idx").setTypes("type")
                 .setSize(0)
                 .setQuery(QUERY)

+ 3 - 2
core/src/test/java/org/elasticsearch/search/aggregations/bucket/NaNSortingIT.java

@@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.metrics.avg.Avg;
 import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
 import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsAggregationBuilder;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
 import org.elasticsearch.test.ESIntegTestCase;
@@ -150,7 +151,7 @@ public class NaNSortingIT extends ESIntegTestCase {
         final boolean asc = randomBoolean();
         SubAggregation agg = randomFrom(SubAggregation.values());
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(terms("terms").field(fieldName).collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(agg.builder()).order(Terms.Order.aggregation(agg.sortKey(), asc)))
+                .addAggregation(terms("terms").field(fieldName).collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(agg.builder()).order(BucketOrder.aggregation(agg.sortKey(), asc)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -175,7 +176,7 @@ public class NaNSortingIT extends ESIntegTestCase {
         SubAggregation agg = randomFrom(SubAggregation.values());
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(histogram("histo")
-                        .field("long_value").interval(randomIntBetween(1, 2)).subAggregation(agg.builder()).order(Histogram.Order.aggregation(agg.sortKey(), asc)))
+                        .field("long_value").interval(randomIntBetween(1, 2)).subAggregation(agg.builder()).order(BucketOrder.aggregation(agg.sortKey(), asc)))
                 .execute().actionGet();
 
         assertSearchResponse(response);

+ 3 - 2
core/src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedIT.java

@@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.bucket.nested.Nested;
 import org.elasticsearch.search.aggregations.bucket.nested.ReverseNested;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -349,13 +350,13 @@ public class ReverseNestedIT extends ESIntegTestCase {
         SearchResponse response = client().prepareSearch("idx2")
                 .addAggregation(nested("nested1", "nested1.nested2")
                                 .subAggregation(
-                                        terms("field2").field("nested1.nested2.field2").order(Terms.Order.term(true))
+                                        terms("field2").field("nested1.nested2.field2").order(BucketOrder.key(true))
                                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
                                                 .size(10000)
                                                 .subAggregation(
                                                         reverseNested("nested1_to_field1").path("nested1")
                                                                 .subAggregation(
-                                                                        terms("field1").field("nested1.field1").order(Terms.Order.term(true))
+                                                                        terms("field1").field("nested1.field1").order(BucketOrder.key(true))
                                                                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
                                                                 )
                                                 )

+ 2 - 1
core/src/test/java/org/elasticsearch/search/aggregations/bucket/SamplerIT.java

@@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregationBu
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
 import org.elasticsearch.search.aggregations.metrics.max.Max;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.Collection;
@@ -99,7 +100,7 @@ public class SamplerIT extends ESIntegTestCase {
         SearchResponse response = client().prepareSearch("test").setTypes("book").setSearchType(SearchType.QUERY_THEN_FETCH)
                 .addAggregation(terms("genres")
                         .field("genre")
-                        .order(Terms.Order.aggregation("sample>max_price.value", asc))
+                        .order(BucketOrder.aggregation("sample>max_price.value", asc))
                         .subAggregation(sampler("sample").shardSize(100)
                                 .subAggregation(max("max_price").field("price")))
                 ).execute().actionGet();

+ 16 - 15
core/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardSizeTermsIT.java

@@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.HashMap;
 import java.util.List;
@@ -39,7 +40,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms  terms = response.getAggregations().get("keys");
@@ -62,7 +63,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3).shardSize(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms  terms = response.getAggregations().get("keys");
@@ -86,7 +87,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -110,7 +111,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting(routing1)
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -133,7 +134,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.term(true)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.key(true)))
                 .execute().actionGet();
 
         Terms  terms = response.getAggregations().get("keys");
@@ -156,7 +157,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -179,7 +180,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3).shardSize(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -202,7 +203,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -226,7 +227,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting(routing1)
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -249,7 +250,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.term(true)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.key(true)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -272,7 +273,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -295,7 +296,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3).shardSize(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -318,7 +319,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -341,7 +342,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting(routing1)
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(BucketOrder.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -364,7 +365,7 @@ public class ShardSizeTermsIT extends ShardSizeTestCase {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
                 .addAggregation(terms("keys").field("key").size(3)
-                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.term(true)))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.key(true)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");

+ 74 - 37
core/src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsIT.java

@@ -42,10 +42,12 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory.ExecutionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.metrics.avg.Avg;
+import org.elasticsearch.search.aggregations.metrics.max.Max;
 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.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.hamcrest.Matchers;
 
@@ -72,6 +74,7 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.count;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
 import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
@@ -129,9 +132,15 @@ public class StringTermsIT extends AbstractTermsTestCase {
         List<IndexRequestBuilder> builders = new ArrayList<>();
         for (int i = 0; i < 5; i++) {
             builders.add(client().prepareIndex("idx", "type").setSource(
-                    jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, "val" + i).field("i", i)
-                            .field("tag", i < 5 / 2 + 1 ? "more" : "less").startArray(MULTI_VALUED_FIELD_NAME).value("val" + i)
-                            .value("val" + (i + 1)).endArray().endObject()));
+                    jsonBuilder().startObject()
+                            .field(SINGLE_VALUED_FIELD_NAME, "val" + i)
+                            .field("i", i)
+                            .field("constant", 1)
+                            .field("tag", i < 5 / 2 + 1 ? "more" : "less")
+                            .startArray(MULTI_VALUED_FIELD_NAME)
+                                .value("val" + i)
+                                .value("val" + (i + 1))
+                            .endArray().endObject()));
         }
 
         getMultiSortDocs(builders);
@@ -456,15 +465,15 @@ public class StringTermsIT extends AbstractTermsTestCase {
         }
 
     }
-        
+
     public void testSingleValueFieldWithPartitionedFiltering() throws Exception {
         runTestFieldWithPartitionedFiltering(SINGLE_VALUED_FIELD_NAME);
     }
-    
+
     public void testMultiValueFieldWithPartitionedFiltering() throws Exception {
         runTestFieldWithPartitionedFiltering(MULTI_VALUED_FIELD_NAME);
     }
-    
+
     private void runTestFieldWithPartitionedFiltering(String field) throws Exception {
         // Find total number of unique terms
         SearchResponse allResponse = client().prepareSearch("idx").setTypes("type")
@@ -492,8 +501,8 @@ public class StringTermsIT extends AbstractTermsTestCase {
             }
         }
         assertEquals(expectedCardinality, foundTerms.size());
-    }    
-    
+    }
+
 
     public void testSingleValueFieldWithMaxSize() throws Exception {
         SearchResponse response = client()
@@ -503,7 +512,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                                 .executionHint(randomExecutionHint())
                                 .field(SINGLE_VALUED_FIELD_NAME).size(20)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
+                                .order(BucketOrder.key(true))) // we need to sort by terms cause we're checking the first 20 values
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -527,7 +536,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .setTypes("type")
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.term(true))).execute()
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.key(true))).execute()
                 .actionGet();
 
         assertSearchResponse(response);
@@ -552,7 +561,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .setTypes("type")
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.term(false))).execute()
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.key(false))).execute()
                 .actionGet();
 
         assertSearchResponse(response);
@@ -944,7 +953,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .setTypes("type")
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.aggregation("avg_i", asc))
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.aggregation("avg_i", asc))
                                 .subAggregation(avg("avg_i").field("i"))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -966,6 +975,34 @@ public class StringTermsIT extends AbstractTermsTestCase {
         }
     }
 
+    public void testSingleValuedFieldOrderedByTieBreaker() throws Exception {
+        SearchResponse response = client()
+            .prepareSearch("idx")
+            .setTypes("type")
+            .addAggregation(
+                terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
+                    .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.aggregation("max_constant", randomBoolean()))
+                    .subAggregation(max("max_constant").field("constant"))).execute().actionGet();
+
+        assertSearchResponse(response);
+
+        Terms terms = response.getAggregations().get("terms");
+        assertThat(terms, notNullValue());
+        assertThat(terms.getName(), equalTo("terms"));
+        assertThat(terms.getBuckets().size(), equalTo(5));
+
+        int i = 0;
+        for (Terms.Bucket bucket : terms.getBuckets()) {
+            assertThat(bucket, notNullValue());
+            assertThat(key(bucket), equalTo("val" + i));
+            assertThat(bucket.getDocCount(), equalTo(1L));
+            Max max = bucket.getAggregations().get("max_constant");
+            assertThat(max, notNullValue());
+            assertThat(max.getValue(), equalTo((double) 1));
+            i++;
+        }
+    }
+
     public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception {
         boolean asc = true;
         try {
@@ -975,7 +1012,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("inner_terms>avg", asc))
+                                .order(BucketOrder.aggregation("inner_terms>avg", asc))
                                 .subAggregation(terms("inner_terms").field(MULTI_VALUED_FIELD_NAME).subAggregation(avg("avg").field("i"))))
                 .execute().actionGet();
             fail("Expected an exception");
@@ -985,7 +1022,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 ElasticsearchException rootCause = rootCauses[0];
                 if (rootCause instanceof AggregationExecutionException) {
                     AggregationExecutionException aggException = (AggregationExecutionException) rootCause;
-                    assertThat(aggException.getMessage(), Matchers.startsWith("Invalid terms aggregation order path"));
+                    assertThat(aggException.getMessage(), Matchers.startsWith("Invalid aggregation order path"));
                 } else {
                     throw e;
                 }
@@ -1002,7 +1039,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .setTypes("type")
                 .addAggregation(
                         terms("tags").executionHint(randomExecutionHint()).field("tag")
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.aggregation("filter", asc))
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.aggregation("filter", asc))
                                 .subAggregation(filter("filter", QueryBuilders.matchAllQuery()))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1041,7 +1078,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                                 .executionHint(randomExecutionHint())
                                 .field("tag")
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("filter1>filter2>stats.max", asc))
+                                .order(BucketOrder.aggregation("filter1>filter2>stats.max", asc))
                                 .subAggregation(
                                         filter("filter1", QueryBuilders.matchAllQuery()).subAggregation(
                                                 filter("filter2", QueryBuilders.matchAllQuery()).subAggregation(
@@ -1104,7 +1141,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                                 .executionHint(randomExecutionHint())
                                 .field("tag")
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("filter1>" + filter2Name + ">" + statsName + ".max", asc))
+                                .order(BucketOrder.aggregation("filter1>" + filter2Name + ">" + statsName + ".max", asc))
                                 .subAggregation(
                                         filter("filter1", QueryBuilders.matchAllQuery()).subAggregation(
                                                 filter(filter2Name, QueryBuilders.matchAllQuery()).subAggregation(
@@ -1167,7 +1204,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                                 .executionHint(randomExecutionHint())
                                 .field("tag")
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("filter1>" + filter2Name + ">" + statsName + "[max]", asc))
+                                .order(BucketOrder.aggregation("filter1>" + filter2Name + ">" + statsName + "[max]", asc))
                                 .subAggregation(
                                         filter("filter1", QueryBuilders.matchAllQuery()).subAggregation(
                                                 filter(filter2Name, QueryBuilders.matchAllQuery()).subAggregation(
@@ -1222,7 +1259,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                         .addAggregation(
                                 terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
                                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("avg_i", true))).execute().actionGet();
+                                        .order(BucketOrder.aggregation("avg_i", true))).execute().actionGet();
 
                 fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
 
@@ -1240,7 +1277,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                         .addAggregation(
                                 terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
                                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("values", true))
+                                        .order(BucketOrder.aggregation("values", true))
                                         .subAggregation(terms("values").field("i").collectMode(randomFrom(SubAggCollectionMode.values()))))
                         .execute().actionGet();
 
@@ -1262,7 +1299,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                         .addAggregation(
                                 terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
                                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("stats.foo", true)).subAggregation(stats("stats").field("i")))
+                                        .order(BucketOrder.aggregation("stats.foo", true)).subAggregation(stats("stats").field("i")))
                         .execute().actionGet();
                 fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation "
                         + "with an unknown specified metric to order by. response had " + response.getFailedShards() + " failed shards.");
@@ -1281,7 +1318,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                         .addAggregation(
                                 terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
                                         .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                        .order(Terms.Order.aggregation("stats", true)).subAggregation(stats("stats").field("i"))).execute()
+                                        .order(BucketOrder.aggregation("stats", true)).subAggregation(stats("stats").field("i"))).execute()
                         .actionGet();
 
                 fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation "
@@ -1300,7 +1337,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .setTypes("type")
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.aggregation("avg_i", asc))
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.aggregation("avg_i", asc))
                                 .subAggregation(avg("avg_i").field("i"))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1331,7 +1368,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .setTypes("type")
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.aggregation("stats.avg", asc))
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.aggregation("stats.avg", asc))
                                 .subAggregation(stats("stats").field("i"))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1361,7 +1398,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .setTypes("type")
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.aggregation("stats.avg", asc))
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.aggregation("stats.avg", asc))
                                 .subAggregation(stats("stats").field("i"))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1393,7 +1430,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("stats.sum_of_squares", asc))
+                                .order(BucketOrder.aggregation("stats.sum_of_squares", asc))
                                 .subAggregation(extendedStats("stats").field("i"))).execute().actionGet();
 
         assertSearchResponse(response);
@@ -1425,7 +1462,7 @@ public class StringTermsIT extends AbstractTermsTestCase {
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
                                 .collectMode(randomFrom(SubAggCollectionMode.values()))
-                                .order(Terms.Order.aggregation("stats.sum_of_squares", asc))
+                                .order(BucketOrder.aggregation("stats.sum_of_squares", asc))
                                 .subAggregation(extendedStats("stats").field("i"))
                                 .subAggregation(terms("subTerms").field("s_values").collectMode(randomFrom(SubAggCollectionMode.values()))))
                 .execute().actionGet();
@@ -1464,46 +1501,46 @@ public class StringTermsIT extends AbstractTermsTestCase {
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsDesc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndTermsAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
     }
 
     public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
     }
 
     public void testSingleValuedFieldOrderedByThreeCriteria() 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));
+        assertMultiSortResponse(expectedKeys, BucketOrder.count(false), BucketOrder.aggregation("sum_d", false),
+                BucketOrder.aggregation("avg_l", false));
     }
 
     public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
         String[] expectedKeys = new String[] { "val1", "val2", "val3", "val4", "val5", "val6", "val7" };
-        assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
+        assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true));
     }
 
-    private void assertMultiSortResponse(String[] expectedKeys, Terms.Order... order) {
+    private void assertMultiSortResponse(String[] expectedKeys, BucketOrder... order) {
         SearchResponse response = client()
                 .prepareSearch("sort_idx")
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field(SINGLE_VALUED_FIELD_NAME)
-                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.compound(order))
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(BucketOrder.compound(order))
                                 .subAggregation(avg("avg_l").field("l")).subAggregation(sum("sum_d").field("d"))).execute().actionGet();
 
         assertSearchResponse(response);

+ 39 - 39
core/src/test/java/org/elasticsearch/search/aggregations/bucket/TermsDocCountErrorIT.java

@@ -27,8 +27,8 @@ import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory.ExecutionMode;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -347,7 +347,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(STRING_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.count(true))
+                        .order(BucketOrder.count(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -360,7 +360,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.count(true))
+                        .order(BucketOrder.count(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -378,7 +378,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(STRING_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.term(true))
+                        .order(BucketOrder.key(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -391,7 +391,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.term(true))
+                        .order(BucketOrder.key(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -409,7 +409,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(STRING_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.term(false))
+                        .order(BucketOrder.key(false))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -422,7 +422,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.term(false))
+                        .order(BucketOrder.key(false))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -440,7 +440,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(STRING_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.aggregation("sortAgg", true))
+                        .order(BucketOrder.aggregation("sortAgg", true))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -454,7 +454,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.aggregation("sortAgg", true))
+                        .order(BucketOrder.aggregation("sortAgg", true))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -473,7 +473,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(STRING_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.aggregation("sortAgg", false))
+                        .order(BucketOrder.aggregation("sortAgg", false))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -487,7 +487,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.aggregation("sortAgg", false))
+                        .order(BucketOrder.aggregation("sortAgg", false))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -583,7 +583,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(LONG_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.count(true))
+                        .order(BucketOrder.count(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -596,7 +596,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.count(true))
+                        .order(BucketOrder.count(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -614,7 +614,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(LONG_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.term(true))
+                        .order(BucketOrder.key(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -627,7 +627,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.term(true))
+                        .order(BucketOrder.key(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -645,7 +645,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(LONG_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.term(false))
+                        .order(BucketOrder.key(false))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -658,7 +658,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.term(false))
+                        .order(BucketOrder.key(false))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -676,7 +676,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(LONG_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.aggregation("sortAgg", true))
+                        .order(BucketOrder.aggregation("sortAgg", true))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -690,7 +690,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.aggregation("sortAgg", true))
+                        .order(BucketOrder.aggregation("sortAgg", true))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -709,7 +709,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(LONG_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.aggregation("sortAgg", false))
+                        .order(BucketOrder.aggregation("sortAgg", false))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(DOUBLE_FIELD_NAME)))
                 .execute().actionGet();
@@ -723,7 +723,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.aggregation("sortAgg", false))
+                        .order(BucketOrder.aggregation("sortAgg", false))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(DOUBLE_FIELD_NAME)))
                 .execute().actionGet();
@@ -819,7 +819,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(DOUBLE_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.count(true))
+                        .order(BucketOrder.count(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -832,7 +832,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.count(true))
+                        .order(BucketOrder.count(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -850,7 +850,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(DOUBLE_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.term(true))
+                        .order(BucketOrder.key(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -863,7 +863,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.term(true))
+                        .order(BucketOrder.key(true))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -881,7 +881,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(DOUBLE_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.term(false))
+                        .order(BucketOrder.key(false))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -894,7 +894,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.term(false))
+                        .order(BucketOrder.key(false))
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
@@ -912,7 +912,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(DOUBLE_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.aggregation("sortAgg", true))
+                        .order(BucketOrder.aggregation("sortAgg", true))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -926,7 +926,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.aggregation("sortAgg", true))
+                        .order(BucketOrder.aggregation("sortAgg", true))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -945,7 +945,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .field(DOUBLE_FIELD_NAME)
                         .showTermDocCountError(true)
                         .size(10000).shardSize(10000)
-                        .order(Order.aggregation("sortAgg", false))
+                        .order(BucketOrder.aggregation("sortAgg", false))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -959,7 +959,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .showTermDocCountError(true)
                         .size(size)
                         .shardSize(shardSize)
-                        .order(Order.aggregation("sortAgg", false))
+                        .order(BucketOrder.aggregation("sortAgg", false))
                         .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sortAgg").field(LONG_FIELD_NAME)))
                 .execute().actionGet();
@@ -968,7 +968,7 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
 
         assertUnboundedDocCountError(size, accurateResponse, testResponse);
     }
-    
+
     /**
      * Test a case where we know exactly how many of each term is on each shard
      * so we know the exact error value for each term. To do this we search over
@@ -984,39 +984,39 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
                         .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
         assertSearchResponse(response);
-        
+
         Terms terms = response.getAggregations().get("terms");
         assertThat(terms, notNullValue());
         assertThat(terms.getDocCountError(), equalTo(46L));
         List<? extends Bucket> buckets = terms.getBuckets();
         assertThat(buckets, notNullValue());
         assertThat(buckets.size(), equalTo(5));
-        
+
         Bucket bucket = buckets.get(0);
         assertThat(bucket, notNullValue());
         assertThat(bucket.getKey(), equalTo("A"));
         assertThat(bucket.getDocCount(), equalTo(100L));
         assertThat(bucket.getDocCountError(), equalTo(0L));
-        
+
         bucket = buckets.get(1);
         assertThat(bucket, notNullValue());
         assertThat(bucket.getKey(), equalTo("Z"));
         assertThat(bucket.getDocCount(), equalTo(52L));
         assertThat(bucket.getDocCountError(), equalTo(2L));
-        
+
         bucket = buckets.get(2);
         assertThat(bucket, notNullValue());
         assertThat(bucket.getKey(), equalTo("C"));
         assertThat(bucket.getDocCount(), equalTo(50L));
         assertThat(bucket.getDocCountError(), equalTo(15L));
-        
-        
+
+
         bucket = buckets.get(3);
         assertThat(bucket, notNullValue());
         assertThat(bucket.getKey(), equalTo("G"));
         assertThat(bucket.getDocCount(), equalTo(45L));
         assertThat(bucket.getDocCountError(), equalTo(2L));
-        
+
         bucket = buckets.get(4);
         assertThat(bucket, notNullValue());
         assertThat(bucket.getKey(), equalTo("B"));

+ 3 - 2
core/src/test/java/org/elasticsearch/search/aggregations/bucket/TermsShardMinDocCountIT.java

@@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter;
 import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms;
 import org.elasticsearch.search.aggregations.bucket.significant.SignificantTermsAggregatorFactory;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -129,7 +130,7 @@ public class TermsShardMinDocCountIT extends ESIntegTestCase {
         // first, check that indeed when not setting the shardMinDocCount parameter 0 terms are returned
         SearchResponse response = client().prepareSearch(index)
                 .addAggregation(
-                        terms("myTerms").field("text").minDocCount(2).size(2).executionHint(randomExecutionHint()).order(Terms.Order.term(true))
+                        terms("myTerms").field("text").minDocCount(2).size(2).executionHint(randomExecutionHint()).order(BucketOrder.key(true))
                 )
                 .execute()
                 .actionGet();
@@ -140,7 +141,7 @@ public class TermsShardMinDocCountIT extends ESIntegTestCase {
 
         response = client().prepareSearch(index)
                 .addAggregation(
-                        terms("myTerms").field("text").minDocCount(2).shardMinDocCount(2).size(2).executionHint(randomExecutionHint()).order(Terms.Order.term(true))
+                        terms("myTerms").field("text").minDocCount(2).shardMinDocCount(2).size(2).executionHint(randomExecutionHint()).order(BucketOrder.key(true))
                 )
                 .execute()
                 .actionGet();

+ 13 - 9
core/src/test/java/org/elasticsearch/search/aggregations/bucket/TermsTests.java

@@ -23,10 +23,10 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.RegExp;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory.ExecutionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -155,8 +155,12 @@ public class TermsTests extends BaseAggregationTestCase<TermsAggregationBuilder>
             factory.includeExclude(incExc);
         }
         if (randomBoolean()) {
-            List<Terms.Order> order = randomOrder();
-            factory.order(order);
+            List<BucketOrder> order = randomOrder();
+            if(order.size() == 1 && randomBoolean()) {
+                factory.order(order.get(0));
+            } else {
+                factory.order(order);
+            }
         }
         if (randomBoolean()) {
             factory.showTermDocCountError(randomBoolean());
@@ -164,20 +168,20 @@ public class TermsTests extends BaseAggregationTestCase<TermsAggregationBuilder>
         return factory;
     }
 
-    private List<Terms.Order> randomOrder() {
-        List<Terms.Order> orders = new ArrayList<>();
+    private List<BucketOrder> randomOrder() {
+        List<BucketOrder> orders = new ArrayList<>();
         switch (randomInt(4)) {
         case 0:
-            orders.add(Terms.Order.term(randomBoolean()));
+            orders.add(BucketOrder.key(randomBoolean()));
             break;
         case 1:
-            orders.add(Terms.Order.count(randomBoolean()));
+            orders.add(BucketOrder.count(randomBoolean()));
             break;
         case 2:
-            orders.add(Terms.Order.aggregation(randomAlphaOfLengthBetween(3, 20), randomBoolean()));
+            orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomBoolean()));
             break;
         case 3:
-            orders.add(Terms.Order.aggregation(randomAlphaOfLengthBetween(3, 20), randomAlphaOfLengthBetween(3, 20), randomBoolean()));
+            orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomAlphaOfLengthBetween(3, 20), randomBoolean()));
             break;
         case 4:
             int numOrders = randomIntBetween(1, 3);

+ 2 - 1
core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.InternalAggregationTestCase;
 import org.joda.time.DateTime;
 
@@ -55,7 +56,7 @@ public class InternalDateHistogramTests extends InternalAggregationTestCase<Inte
             buckets.add(i, new InternalDateHistogram.Bucket(key, randomIntBetween(1, 100), keyed, format, InternalAggregations.EMPTY));
         }
 
-        InternalOrder order = (InternalOrder) randomFrom(InternalHistogram.Order.KEY_ASC, InternalHistogram.Order.KEY_DESC);
+        BucketOrder order = randomFrom(BucketOrder.key(true), BucketOrder.key(false));
         return new InternalDateHistogram(name, buckets, order, 1, 0L, null, format, keyed, pipelineAggregators, metaData);
     }
 

+ 2 - 1
core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java

@@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 import org.elasticsearch.test.InternalAggregationTestCase;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -46,7 +47,7 @@ public class InternalHistogramTests extends InternalAggregationTestCase<Internal
             final int docCount = TestUtil.nextInt(random(), 1, 50);
             buckets.add(new InternalHistogram.Bucket(base + i * interval, docCount, keyed, format, InternalAggregations.EMPTY));
         }
-        return new InternalHistogram(name, buckets, (InternalOrder) InternalHistogram.Order.KEY_ASC,
+        return new InternalHistogram(name, buckets, BucketOrder.key(true),
                 1, null, format, keyed, pipelineAggregators, metaData);
     }
 

+ 2 - 1
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.io.stream.Writeable.Reader;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -37,7 +38,7 @@ public class DoubleTermsTests extends InternalTermsTestCase {
             String name,
             List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData) {
-        Terms.Order order = Terms.Order.count(false);
+        BucketOrder order = BucketOrder.count(false);
         long minDocCount = 1;
         int requiredSize = 3;
         int shardSize = requiredSize + 2;

+ 2 - 1
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.io.stream.Writeable.Reader;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -37,7 +38,7 @@ public class LongTermsTests extends InternalTermsTestCase {
             String name,
             List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData) {
-        Terms.Order order = Terms.Order.count(false);
+        BucketOrder order = BucketOrder.count(false);
         long minDocCount = 1;
         int requiredSize = 3;
         int shardSize = requiredSize + 2;

+ 2 - 1
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsTests.java

@@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.Writeable.Reader;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.InternalAggregations;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -38,7 +39,7 @@ public class StringTermsTests extends InternalTermsTestCase {
             String name,
             List<PipelineAggregator> pipelineAggregators,
             Map<String, Object> metaData) {
-        Terms.Order order = Terms.Order.count(false);
+        BucketOrder order = BucketOrder.count(false);
         long minDocCount = 1;
         int requiredSize = 3;
         int shardSize = requiredSize + 2;

+ 3 - 2
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java

@@ -37,6 +37,7 @@ import org.elasticsearch.index.mapper.NumberFieldMapper;
 import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
 import org.elasticsearch.search.aggregations.AggregatorTestCase;
 import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.aggregations.support.ValueType;
 
 import java.io.IOException;
@@ -70,7 +71,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
             TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.STRING)
                 .executionHint(executionMode.toString())
                 .field("string")
-                .order(Terms.Order.term(true));
+                .order(BucketOrder.key(true));
             MappedFieldType fieldType = new KeywordFieldMapper.KeywordFieldType();
             fieldType.setName("string");
             fieldType.setHasDocValues(true );
@@ -99,7 +100,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
             TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.LONG)
                 .executionHint(executionMode.toString())
                 .field("number")
-                .order(Terms.Order.term(true));
+                .order(BucketOrder.key(true));
             List<InternalAggregation> aggs = new ArrayList<> ();
             int numLongs = randomIntBetween(1, 3);
             for (int i = 0; i < numLongs; i++) {

+ 2 - 2
core/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgIT.java

@@ -36,8 +36,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.metrics.avg.Avg;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.lookup.LeafSearchLookup;
 import org.elasticsearch.search.lookup.SearchLookup;
 
@@ -326,7 +326,7 @@ public class AvgIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Order.compound(Order.aggregation("filter>avg", true)))
+                .addAggregation(terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>avg", true)))
                         .subAggregation(filter("filter", termQuery("value", 100)).subAggregation(avg("avg").field("value"))))
                 .get();
 

+ 3 - 2
core/src/test/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsIT.java

@@ -30,9 +30,9 @@ import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.missing.Missing;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
 import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats.Bounds;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -595,7 +595,8 @@ public class ExtendedStatsIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Order.compound(Order.aggregation("filter>extendedStats.avg", true)))
+                .addAggregation(terms("terms").field("value")
+                    .order(BucketOrder.compound(BucketOrder.aggregation("filter>extendedStats.avg", true)))
                         .subAggregation(
                                 filter("filter", termQuery("value", 100)).subAggregation(extendedStats("extendedStats").field("value"))))
                 .get();

+ 3 - 3
core/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksIT.java

@@ -29,11 +29,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanks;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -485,7 +485,7 @@ public class HDRPercentileRanksIT extends AbstractNumericTestCase {
                                 .subAggregation(
                                         percentileRanks("percentile_ranks").field("value").method(PercentilesMethod.HDR)
                                 .numberOfSignificantValueDigits(sigDigits).values(99))
-                                .order(Order.aggregation("percentile_ranks", "99", asc))).execute().actionGet();
+                                .order(BucketOrder.aggregation("percentile_ranks", "99", asc))).execute().actionGet();
 
         assertHitCount(searchResponse, 10);
 
@@ -506,7 +506,7 @@ public class HDRPercentileRanksIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Terms.Order.compound(Terms.Order.aggregation("filter>ranks.99", true)))
+                .addAggregation(terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>ranks.99", true)))
                         .subAggregation(filter("filter", termQuery("value", 100))
                                 .subAggregation(percentileRanks("ranks").method(PercentilesMethod.HDR).values(99).field("value"))))
                 .get();

+ 3 - 3
core/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesIT.java

@@ -30,11 +30,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
 import org.elasticsearch.search.aggregations.metrics.percentiles.Percentiles;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -474,7 +474,7 @@ public class HDRPercentilesIT extends AbstractNumericTestCase {
                                                 .method(PercentilesMethod.HDR)
                                                 .numberOfSignificantValueDigits(sigDigits)
                                                 .percentiles(99))
-                                .order(Order.aggregation("percentiles", "99", asc))).execute().actionGet();
+                                .order(BucketOrder.aggregation("percentiles", "99", asc))).execute().actionGet();
 
         assertHitCount(searchResponse, 10);
 
@@ -497,7 +497,7 @@ public class HDRPercentilesIT extends AbstractNumericTestCase {
         SearchResponse searchResponse = client().prepareSearch("idx")
                 .setQuery(matchAllQuery())
                 .addAggregation(
-                        terms("terms").field("value").order(Terms.Order.compound(Terms.Order.aggregation("filter>percentiles.99", true)))
+                        terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>percentiles.99", true)))
                                 .subAggregation(filter("filter", termQuery("value", 100))
                                         .subAggregation(percentiles("percentiles").method(PercentilesMethod.HDR).field("value"))))
                 .get();

+ 2 - 2
core/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxIT.java

@@ -29,8 +29,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.metrics.max.Max;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -328,7 +328,7 @@ public class MaxIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Order.compound(Order.aggregation("filter>max", true)))
+                .addAggregation(terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>max", true)))
                         .subAggregation(filter("filter", termQuery("value", 100)).subAggregation(max("max").field("value"))))
                 .get();
 

+ 2 - 2
core/src/test/java/org/elasticsearch/search/aggregations/metrics/MinIT.java

@@ -29,8 +29,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.metrics.min.Min;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -340,7 +340,7 @@ public class MinIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Order.compound(Order.aggregation("filter>min", true)))
+                .addAggregation(terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>min", true)))
                         .subAggregation(filter("filter", termQuery("value", 100)).subAggregation(min("min").field("value"))))
                 .get();
 

+ 2 - 2
core/src/test/java/org/elasticsearch/search/aggregations/metrics/StatsIT.java

@@ -32,8 +32,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.metrics.stats.Stats;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -447,7 +447,7 @@ public class StatsIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Order.compound(Order.aggregation("filter>stats.avg", true)))
+                .addAggregation(terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>stats.avg", true)))
                         .subAggregation(filter("filter", termQuery("value", 100)).subAggregation(stats("stats").field("value"))))
                 .get();
 

+ 2 - 2
core/src/test/java/org/elasticsearch/search/aggregations/metrics/SumIT.java

@@ -36,8 +36,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.lookup.LeafSearchLookup;
 import org.elasticsearch.search.lookup.SearchLookup;
 
@@ -325,7 +325,7 @@ public class SumIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Order.compound(Order.aggregation("filter>sum", true)))
+                .addAggregation(terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>sum", true)))
                         .subAggregation(filter("filter", termQuery("value", 100)).subAggregation(sum("sum").field("value"))))
                 .get();
 

+ 3 - 3
core/src/test/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentileRanksIT.java

@@ -30,12 +30,12 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanks;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -435,7 +435,7 @@ public class TDigestPercentileRanksIT extends AbstractNumericTestCase {
                 .addAggregation(
                         histogram("histo").field("value").interval(2L)
                             .subAggregation(randomCompression(percentileRanks("percentile_ranks").field("value").values(99)))
-                            .order(Order.aggregation("percentile_ranks", "99", asc)))
+                            .order(BucketOrder.aggregation("percentile_ranks", "99", asc)))
                 .execute().actionGet();
 
         assertHitCount(searchResponse, 10);
@@ -457,7 +457,7 @@ public class TDigestPercentileRanksIT extends AbstractNumericTestCase {
     @Override
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
-                .addAggregation(terms("terms").field("value").order(Terms.Order.compound(Terms.Order.aggregation("filter>ranks.99", true)))
+                .addAggregation(terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>ranks.99", true)))
                         .subAggregation(filter("filter", termQuery("value", 100))
                                 .subAggregation(percentileRanks("ranks").method(PercentilesMethod.TDIGEST).values(99).field("value"))))
                 .get();

+ 3 - 3
core/src/test/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentilesIT.java

@@ -30,12 +30,12 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
 import org.elasticsearch.search.aggregations.metrics.percentiles.Percentiles;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
+import org.elasticsearch.search.aggregations.BucketOrder;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -419,7 +419,7 @@ public class TDigestPercentilesIT extends AbstractNumericTestCase {
                 .addAggregation(
                         histogram("histo").field("value").interval(2L)
                         .subAggregation(randomCompression(percentiles("percentiles").field("value").percentiles(99)))
-                            .order(Order.aggregation("percentiles", "99", asc)))
+                            .order(BucketOrder.aggregation("percentiles", "99", asc)))
                 .execute().actionGet();
 
         assertHitCount(searchResponse, 10);
@@ -442,7 +442,7 @@ public class TDigestPercentilesIT extends AbstractNumericTestCase {
     public void testOrderByEmptyAggregation() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("idx").setQuery(matchAllQuery())
                 .addAggregation(
-                        terms("terms").field("value").order(Terms.Order.compound(Terms.Order.aggregation("filter>percentiles.99", true)))
+                        terms("terms").field("value").order(BucketOrder.compound(BucketOrder.aggregation("filter>percentiles.99", true)))
                                 .subAggregation(filter("filter", termQuery("value", 100))
                                         .subAggregation(percentiles("percentiles").method(PercentilesMethod.TDIGEST).field("value"))))
                 .get();

+ 4 - 4
core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java

@@ -47,6 +47,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory.ExecutionMode;
 import org.elasticsearch.search.aggregations.metrics.max.Max;
 import org.elasticsearch.search.aggregations.metrics.tophits.TopHits;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
 import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
 import org.elasticsearch.search.rescore.RescoreBuilder;
@@ -398,7 +399,7 @@ public class TopHitsIT extends ESIntegTestCase {
                 .executionHint(randomExecutionHint())
                 .collectMode(SubAggCollectionMode.BREADTH_FIRST)
                 .field(TERMS_AGGS_FIELD)
-                .order(Terms.Order.aggregation("max", false))
+                .order(BucketOrder.aggregation("max", false))
                 .subAggregation(max("max").field(SORT_FIELD))
                 .subAggregation(topHits("hits").size(3))
             ).get();
@@ -494,7 +495,7 @@ public class TopHitsIT extends ESIntegTestCase {
                 .addAggregation(terms("terms")
                                 .executionHint(randomExecutionHint())
                                 .field(TERMS_AGGS_FIELD)
-                                .order(Terms.Order.aggregation("max_sort", false))
+                                .order(BucketOrder.aggregation("max_sort", false))
                                 .subAggregation(
                                         topHits("hits").sort(SortBuilders.fieldSort(SORT_FIELD).order(SortOrder.DESC)).trackScores(true)
                                 )
@@ -535,7 +536,7 @@ public class TopHitsIT extends ESIntegTestCase {
                 .setQuery(matchQuery("text", "term rare"))
                 .addAggregation(
                         terms("terms").executionHint(randomExecutionHint()).field("group")
-                                .order(Terms.Order.aggregation("max_score", false)).subAggregation(topHits("hits").size(1))
+                                .order(BucketOrder.aggregation("max_score", false)).subAggregation(topHits("hits").size(1))
                                 .subAggregation(max("max_score").field("value"))).get();
         assertSearchResponse(response);
 
@@ -908,7 +909,6 @@ public class TopHitsIT extends ESIntegTestCase {
                         histogram("dates")
                                 .field("date")
                                 .interval(5)
-                                .order(Histogram.Order.aggregation("to-comments", true))
                                 .subAggregation(
                                         nested("to-comments", "comments")
                                                 .subAggregation(topHits("comments")

+ 5 - 5
core/src/test/java/org/elasticsearch/search/aggregations/pipeline/AvgBucketIT.java

@@ -24,10 +24,10 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -129,7 +129,7 @@ public class AvgBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -211,7 +211,7 @@ public class AvgBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -264,7 +264,7 @@ public class AvgBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -337,7 +337,7 @@ public class AvgBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))

+ 6 - 6
core/src/test/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java

@@ -26,12 +26,12 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats.Bounds;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
 import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended.ExtendedStatsBucket;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -200,7 +200,7 @@ public class ExtendedStatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -300,7 +300,7 @@ public class ExtendedStatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -362,7 +362,7 @@ public class ExtendedStatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -445,7 +445,7 @@ public class ExtendedStatsBucketIT extends ESIntegTestCase {
                     .addAggregation(
                             terms("terms")
                                     .field("tag")
-                                    .order(Order.term(true))
+                                    .order(BucketOrder.key(true))
                                     .subAggregation(
                                             histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                     .extendedBounds(minRandomValue, maxRandomValue)
@@ -475,7 +475,7 @@ public class ExtendedStatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))

+ 5 - 5
core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketIT.java

@@ -25,11 +25,11 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
 import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -138,7 +138,7 @@ public class MaxBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -230,7 +230,7 @@ public class MaxBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -335,7 +335,7 @@ public class MaxBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -413,7 +413,7 @@ public class MaxBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                 .extendedBounds(minRandomValue, maxRandomValue))

+ 5 - 5
core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MinBucketIT.java

@@ -24,11 +24,11 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
 import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -135,7 +135,7 @@ public class MinBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -227,7 +227,7 @@ public class MinBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -285,7 +285,7 @@ public class MinBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -363,7 +363,7 @@ public class MinBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))

+ 7 - 6
core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketIT.java

@@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude
 import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
 import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.PercentilesBucket;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -133,7 +134,7 @@ public class PercentilesBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Terms.Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -248,7 +249,7 @@ public class PercentilesBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Terms.Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -301,7 +302,7 @@ public class PercentilesBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Terms.Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -435,7 +436,7 @@ public class PercentilesBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Terms.Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -466,7 +467,7 @@ public class PercentilesBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Terms.Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -525,7 +526,7 @@ public class PercentilesBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Terms.Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))

+ 5 - 5
core/src/test/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketIT.java

@@ -24,11 +24,11 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
 import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.StatsBucket;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -136,7 +136,7 @@ public class StatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -230,7 +230,7 @@ public class StatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -289,7 +289,7 @@ public class StatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -368,7 +368,7 @@ public class StatsBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))

+ 5 - 5
core/src/test/java/org/elasticsearch/search/aggregations/pipeline/SumBucketIT.java

@@ -24,10 +24,10 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.metrics.sum.Sum;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
+import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.test.ESIntegTestCase;
 
 import java.util.ArrayList;
@@ -126,7 +126,7 @@ public class SumBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))
@@ -202,7 +202,7 @@ public class SumBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -252,7 +252,7 @@ public class SumBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue)
@@ -322,7 +322,7 @@ public class SumBucketIT extends ESIntegTestCase {
                 .addAggregation(
                         terms("terms")
                                 .field("tag")
-                                .order(Order.term(true))
+                                .order(BucketOrder.key(true))
                                 .subAggregation(
                                         histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
                                                 .extendedBounds(minRandomValue, maxRandomValue))

+ 4 - 0
docs/java-api/aggregations/bucket/datehistogram-aggregation.asciidoc

@@ -67,3 +67,7 @@ key [2005-01-01T00:00:00.000Z], date [2005], doc_count [1]
 key [2007-01-01T00:00:00.000Z], date [2007], doc_count [2]
 key [2008-01-01T00:00:00.000Z], date [2008], doc_count [3]
 --------------------------------------------------
+
+===== Order
+
+Supports the same order functionality as the <<java-aggs-bucket-terms,`Terms Aggregation`>>.

+ 4 - 0
docs/java-api/aggregations/bucket/histogram-aggregation.asciidoc

@@ -42,3 +42,7 @@ for (Histogram.Bucket entry : agg.getBuckets()) {
     logger.info("key [{}], doc_count [{}]", key, docCount);
 }
 --------------------------------------------------
+
+===== Order
+
+Supports the same order functionality as the <<java-aggs-bucket-terms,`Terms Aggregation`>>.

+ 26 - 4
docs/java-api/aggregations/bucket/terms-aggregation.asciidoc

@@ -39,7 +39,14 @@ for (Terms.Bucket entry : genders.getBuckets()) {
 }
 --------------------------------------------------
 
-==== Order
+===== Order
+
+Import bucket ordering strategy classes:
+
+[source,java]
+--------------------------------------------------
+import org.elasticsearch.search.aggregations.BucketOrder;
+--------------------------------------------------
 
 Ordering the buckets by their `doc_count` in an ascending manner:
 
@@ -48,7 +55,7 @@ Ordering the buckets by their `doc_count` in an ascending manner:
 AggregationBuilders
     .terms("genders")
     .field("gender")
-    .order(Terms.Order.count(true))
+    .order(BucketOrder.count(true))
 --------------------------------------------------
 
 Ordering the buckets alphabetically by their terms in an ascending manner:
@@ -58,7 +65,7 @@ Ordering the buckets alphabetically by their terms in an ascending manner:
 AggregationBuilders
     .terms("genders")
     .field("gender")
-    .order(Terms.Order.term(true))
+    .order(BucketOrder.key(true))
 --------------------------------------------------
 
 Ordering the buckets by single value metrics sub-aggregation (identified by the aggregation name):
@@ -68,7 +75,22 @@ Ordering the buckets by single value metrics sub-aggregation (identified by the
 AggregationBuilders
     .terms("genders")
     .field("gender")
-    .order(Terms.Order.aggregation("avg_height", false))
+    .order(BucketOrder.aggregation("avg_height", false))
+    .subAggregation(
+        AggregationBuilders.avg("avg_height").field("height")
+    )
+--------------------------------------------------
+
+Ordering the buckets by multiple criteria:
+
+[source,java]
+--------------------------------------------------
+AggregationBuilders
+    .terms("genders")
+    .field("gender")
+    .order(BucketOrder.compound( // in order of priority:
+        BucketOrder.aggregation("avg_height", false), // sort by sub-aggregation first
+        BucketOrder.count(true))) // then bucket count as a tie-breaker
     .subAggregation(
         AggregationBuilders.avg("avg_height").field("height")
     )

+ 7 - 0
docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc

@@ -390,3 +390,10 @@ POST /sales/_search?size=0
 // TEST[setup:sales]
 
 <1> Documents without a value in the `publish_date` field will fall into the same bucket as documents that have the value `2000-01-01`.
+
+==== Order
+
+By default the returned buckets are sorted by their `key` ascending, though the order behaviour can be controlled using
+the `order` setting. Supports the same `order` functionality as the <<search-aggregations-bucket-terms-aggregation-order,`Terms Aggregation`>>.
+
+deprecated[6.0.0, Use `_key` instead of `_time` to order buckets by their dates/keys]

+ 2 - 114
docs/reference/aggregations/bucket/histogram-aggregation.asciidoc

@@ -179,120 +179,8 @@ POST /sales/_search?size=0
 
 ==== Order
 
-By default the returned buckets are sorted by their `key` ascending, though the order behaviour can be controlled
-using the `order` setting.
-
-Ordering the buckets by their key - descending:
-
-[source,js]
---------------------------------------------------
-POST /sales/_search?size=0
-{
-    "aggs" : {
-        "prices" : {
-            "histogram" : {
-                "field" : "price",
-                "interval" : 50,
-                "order" : { "_key" : "desc" }
-            }
-        }
-    }
-}
---------------------------------------------------
-// CONSOLE
-// TEST[setup:sales]
-
-Ordering the buckets by their `doc_count` - ascending:
-
-[source,js]
---------------------------------------------------
-POST /sales/_search?size=0
-{
-    "aggs" : {
-        "prices" : {
-            "histogram" : {
-                "field" : "price",
-                "interval" : 50,
-                "order" : { "_count" : "asc" }
-            }
-        }
-    }
-}
---------------------------------------------------
-// CONSOLE
-// TEST[setup:sales]
-
-If the histogram aggregation has a direct metrics sub-aggregation, the latter can determine the order of the buckets:
-
-[source,js]
---------------------------------------------------
-POST /sales/_search?size=0
-{
-    "aggs" : {
-        "prices" : {
-            "histogram" : {
-                "field" : "price",
-                "interval" : 50,
-                "order" : { "price_stats.min" : "asc" } <1>
-            },
-            "aggs" : {
-                "price_stats" : { "stats" : {"field" : "price"} }
-            }
-        }
-    }
-}
---------------------------------------------------
-// CONSOLE
-// TEST[setup:sales]
-
-<1> The `{ "price_stats.min" : asc" }` will sort the buckets based on `min` value of their `price_stats` sub-aggregation.
-
-It is also possible to order the buckets based on a "deeper" aggregation in the hierarchy. This is supported as long
-as the aggregations path are of a single-bucket type, where the last aggregation in the path may either by a single-bucket
-one or a metrics one. If it's a single-bucket type, the order will be defined by the number of docs in the bucket (i.e. `doc_count`),
-in case it's a metrics one, the same rules as above apply (where the path must indicate the metric name to sort by in case of
-a multi-value metrics aggregation, and in case of a single-value metrics aggregation the sort will be applied on that value).
-
-The path must be defined in the following form:
-
-// https://en.wikipedia.org/wiki/Extended_Backus%E2%80%93Naur_Form
-[source,ebnf]
---------------------------------------------------
-AGG_SEPARATOR       =  '>' ;
-METRIC_SEPARATOR    =  '.' ;
-AGG_NAME            =  <the name of the aggregation> ;
-METRIC              =  <the name of the metric (in case of multi-value metrics aggregation)> ;
-PATH                =  <AGG_NAME> [ <AGG_SEPARATOR>, <AGG_NAME> ]* [ <METRIC_SEPARATOR>, <METRIC> ] ;
---------------------------------------------------
-
-[source,js]
---------------------------------------------------
-POST /sales/_search?size=0
-{
-    "aggs" : {
-        "prices" : {
-            "histogram" : {
-                "field" : "price",
-                "interval" : 50,
-                "order" : { "promoted_products>rating_stats.avg" : "desc" } <1>
-            },
-            "aggs" : {
-                "promoted_products" : {
-                    "filter" : { "term" : { "promoted" : true }},
-                    "aggs" : {
-                        "rating_stats" : { "stats" : { "field" : "rating" }}
-                    }
-                }
-            }
-        }
-    }
-}
---------------------------------------------------
-// CONSOLE
-// TEST[setup:sales]
-
-The above will sort the buckets based on the avg rating among the promoted products
-
+By default the returned buckets are sorted by their `key` ascending, though the order behaviour can be controlled using
+the `order` setting. Supports the same `order` functionality as the <<search-aggregations-bucket-terms-aggregation-order,`Terms Aggregation`>>.
 
 ==== Offset
 

+ 2 - 1
docs/reference/aggregations/bucket/terms-aggregation.asciidoc

@@ -280,13 +280,14 @@ Ordering the buckets alphabetically by their terms in an ascending manner:
         "genres" : {
             "terms" : {
                 "field" : "genre",
-                "order" : { "_term" : "asc" }
+                "order" : { "_key" : "asc" }
             }
         }
     }
 }
 --------------------------------------------------
 
+deprecated[6.0.0, Use `_key` instead of `_term` to order buckets by their term]
 
 Ordering the buckets by single value metrics sub-aggregation (identified by the aggregation name):
 

+ 9 - 1
docs/reference/migration/migrate_6_0/java.asciidoc

@@ -26,4 +26,12 @@ When sending a request through the request builders e.g. client.prepareSearch().
 be possible to call `addListener` against the returned `ListenableActionFuture`. With this change an
 `ActionFuture` is returned instead, which is consistent with what the `Client` methods return, hence
 it is not possible to associate the future with listeners. The `execute` method that accept a listener
-as an argument can be used instead.
+as an argument can be used instead.
+
+==== `Terms.Order` and `Histogram.Order` classes replace by `BucketOrder`
+
+The `terms`, `histogram`, and `date_histogram` aggregation code has been refactored to use common
+code for ordering buckets. The `BucketOrder` class must be used instead of `Terms.Order` and
+`Histogram.Order`. The `static` methods in the `BucketOrder` class must be called instead of directly
+accessing internal order instances, e.g. `BucketOrder.count(boolean)` and `BucketOrder.aggregation(String, boolean)`.
+Use `BucketOrder.key(boolean)` to order the `terms` aggregation buckets by `_term`.

+ 63 - 0
rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/10_histogram.yaml

@@ -10,6 +10,8 @@ setup:
                 "properties":
                   "number":
                      "type" : "integer"
+                  "date":
+                     "type" : "date"
   - do:
       cluster.health:
         wait_for_status: green
@@ -143,3 +145,64 @@ setup:
   - match: { aggregations.histo.buckets.3.key_as_string: "Value is 150.0" }
   
   - match: { aggregations.histo.buckets.3.doc_count: 1 }
+
+---
+"Deprecated _time order":
+
+  - skip:
+      version: " - 5.99.99"
+      reason:  _time order deprecated in 6.0, replaced by _key
+      features: "warnings"
+
+  - do:
+      index:
+        index: test_1
+        type: test
+        id: 1
+        body: { "date" : "2016-01-01" }
+
+  - do:
+      index:
+        index: test_1
+        type: test
+        id: 2
+        body: { "date" : "2016-01-02" }
+
+  - do:
+      index:
+        index: test_1
+        type: test
+        id: 3
+        body: { "date" : "2016-02-01" }
+
+  - do:
+      index:
+        index: test_1
+        type: test
+        id: 4
+        body: { "date" : "2016-03-01" }
+
+  - do:
+      indices.refresh: {}
+
+  - do:
+      search:
+        body: { "aggs" : { "histo" : { "date_histogram" : { "field" : "date", "interval" : "month", "order" : { "_time" : "desc" } } } } }
+      warnings:
+        - "Deprecated aggregation order key [_time] used, replaced by [_key]"
+
+  - match: { hits.total: 4 }
+
+  - length: { aggregations.histo.buckets: 3 }
+
+  - match: { aggregations.histo.buckets.0.key_as_string: "2016-03-01T00:00:00.000Z" }
+
+  - match: { aggregations.histo.buckets.0.doc_count: 1 }
+
+  - match: { aggregations.histo.buckets.1.key_as_string: "2016-02-01T00:00:00.000Z" }
+
+  - match: { aggregations.histo.buckets.1.doc_count: 1 }
+
+  - match: { aggregations.histo.buckets.2.key_as_string: "2016-01-01T00:00:00.000Z" }
+
+  - match: { aggregations.histo.buckets.2.doc_count: 2 }

+ 54 - 0
rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/20_terms.yaml

@@ -747,3 +747,57 @@ setup:
   - match: { aggregations.number_terms.buckets.2.key: 14.6 }
 
   - match: { aggregations.number_terms.buckets.2.doc_count: 1 }
+
+---
+"Deprecated _term order":
+
+  - skip:
+      version: " - 5.99.99"
+      reason:  _term order deprecated in 6.0, replaced by _key
+      features: "warnings"
+
+  - do:
+      index:
+        index: test_1
+        type: test
+        id: 1
+        body: { "str": "abc" }
+
+  - do:
+      index:
+        index: test_1
+        type: test
+        id: 2
+        body: { "str": "abc" }
+
+  - do:
+      index:
+        index: test_1
+        type: test
+        id: 3
+        body: { "str": "bcd" }
+
+  - do:
+      indices.refresh: {}
+
+  - do:
+      search:
+        body: { "size" : 0, "aggs" : { "str_terms" : { "terms" : { "field" : "str", "order" :  { "_term" : "desc" } } } } }
+      warnings:
+        - "Deprecated aggregation order key [_term] used, replaced by [_key]"
+
+  - match: { hits.total: 3 }
+
+  - length: { aggregations.str_terms.buckets: 2 }
+
+  - match: { aggregations.str_terms.buckets.0.key: "bcd" }
+
+  - is_false: aggregations.str_terms.buckets.0.key_as_string
+
+  - match: { aggregations.str_terms.buckets.0.doc_count: 1 }
+
+  - match: { aggregations.str_terms.buckets.1.key: "abc" }
+
+  - is_false: aggregations.str_terms.buckets.1.key_as_string
+
+  - match: { aggregations.str_terms.buckets.1.doc_count: 2 }

+ 2 - 2
test/framework/src/main/java/org/elasticsearch/test/AbstractSerializingTestCase.java

@@ -51,7 +51,7 @@ public abstract class AbstractSerializingTestCase<T extends ToXContent & Writeab
         }
     }
 
-    private void assertParsedInstance(XContentType xContentType, BytesReference instanceAsBytes, T expectedInstance)
+    protected void assertParsedInstance(XContentType xContentType, BytesReference instanceAsBytes, T expectedInstance)
             throws IOException {
 
         XContentParser parser = createParser(XContentFactory.xContent(xContentType), instanceAsBytes);
@@ -61,7 +61,7 @@ public abstract class AbstractSerializingTestCase<T extends ToXContent & Writeab
         assertEquals(expectedInstance.hashCode(), newInstance.hashCode());
     }
 
-    private T parseInstance(XContentParser parser) throws IOException {
+    protected T parseInstance(XContentParser parser) throws IOException {
         T parsedInstance = doParseInstance(parser);
         assertNull(parser.nextToken());
         return parsedInstance;

+ 3 - 3
test/framework/src/main/java/org/elasticsearch/test/AbstractWireSerializingTestCase.java

@@ -100,7 +100,7 @@ public abstract class AbstractWireSerializingTestCase<T extends Writeable> exten
         return deserializedInstance;
     }
 
-    private T copyInstance(T instance) throws IOException {
+    protected T copyInstance(T instance) throws IOException {
         try (BytesStreamOutput output = new BytesStreamOutput()) {
             instance.writeTo(output);
             try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(),
@@ -112,9 +112,9 @@ public abstract class AbstractWireSerializingTestCase<T extends Writeable> exten
 
     /**
      * Get the {@link NamedWriteableRegistry} to use when de-serializing the object.
-     * 
+     *
      * Override this method if you need to register {@link NamedWriteable}s for the test object to de-serialize.
-     * 
+     *
      * By default this will return a {@link NamedWriteableRegistry} with no registered {@link NamedWriteable}s
      */
     protected NamedWriteableRegistry getNamedWriteableRegistry() {

+ 29 - 5
test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java

@@ -72,6 +72,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentHelper;
 import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.common.xcontent.XContentParser.Token;
 import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.env.Environment;
@@ -924,18 +925,39 @@ public abstract class ESTestCase extends LuceneTestCase {
      * recursive shuffling behavior can be made by passing in the names of fields which
      * internally should stay untouched.
      */
-    protected static XContentBuilder shuffleXContent(XContentParser parser, boolean prettyPrint, String... exceptFieldNames)
-            throws IOException {
-        //we need a sorted map for reproducibility, as we are going to shuffle its keys and write XContent back
-        Map<String, Object> shuffledMap = shuffleMap((LinkedHashMap<String, Object>)parser.mapOrdered(),
-                new HashSet<>(Arrays.asList(exceptFieldNames)));
+    public XContentBuilder shuffleXContent(XContentParser parser, boolean prettyPrint, String... exceptFieldNames) throws IOException {
         XContentBuilder xContentBuilder = XContentFactory.contentBuilder(parser.contentType());
         if (prettyPrint) {
             xContentBuilder.prettyPrint();
         }
+        Token token = parser.currentToken() == null ? parser.nextToken() : parser.currentToken();
+        if (token == Token.START_ARRAY) {
+            List<Object> shuffledList = shuffleList(parser.listOrderedMap(), new HashSet<>(Arrays.asList(exceptFieldNames)));
+            return xContentBuilder.value(shuffledList);
+        }
+        //we need a sorted map for reproducibility, as we are going to shuffle its keys and write XContent back
+        Map<String, Object> shuffledMap = shuffleMap((LinkedHashMap<String, Object>)parser.mapOrdered(),
+            new HashSet<>(Arrays.asList(exceptFieldNames)));
         return xContentBuilder.map(shuffledMap);
     }
 
+    // shuffle fields of objects in the list, but not the list itself
+    private static List<Object> shuffleList(List<Object> list, Set<String> exceptFields) {
+        List<Object> targetList = new ArrayList<>();
+        for(Object value : list) {
+            if (value instanceof Map) {
+                @SuppressWarnings("unchecked")
+                LinkedHashMap<String, Object> valueMap = (LinkedHashMap<String, Object>) value;
+                targetList.add(shuffleMap(valueMap, exceptFields));
+            } else if(value instanceof List) {
+                targetList.add(shuffleList((List) value, exceptFields));
+            }  else {
+                targetList.add(value);
+            }
+        }
+        return targetList;
+    }
+
     public static LinkedHashMap<String, Object> shuffleMap(LinkedHashMap<String, Object> map, Set<String> exceptFields) {
         List<String> keys = new ArrayList<>(map.keySet());
         LinkedHashMap<String, Object> targetMap = new LinkedHashMap<>();
@@ -946,6 +968,8 @@ public abstract class ESTestCase extends LuceneTestCase {
                 @SuppressWarnings("unchecked")
                 LinkedHashMap<String, Object> valueMap = (LinkedHashMap<String, Object>) value;
                 targetMap.put(key, shuffleMap(valueMap, exceptFields));
+            } else if(value instanceof List && exceptFields.contains(key) == false) {
+                targetMap.put(key, shuffleList((List) value, exceptFields));
             } else {
                 targetMap.put(key, value);
             }