Browse Source

Aggregations Refactor: Refactor Sampler Aggregation

Colin Goodheart-Smithe 9 years ago
parent
commit
1aea0faa86

+ 2 - 0
core/src/main/java/org/elasticsearch/search/SearchModule.java

@@ -55,6 +55,7 @@ import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanc
 import org.elasticsearch.search.aggregations.bucket.range.geodistance.InternalGeoDistance;
 import org.elasticsearch.search.aggregations.bucket.range.ipv4.InternalIPv4Range;
 import org.elasticsearch.search.aggregations.bucket.range.ipv4.IpRangeParser;
+import org.elasticsearch.search.aggregations.bucket.sampler.DiversifiedSamplerParser;
 import org.elasticsearch.search.aggregations.bucket.sampler.InternalSampler;
 import org.elasticsearch.search.aggregations.bucket.sampler.SamplerParser;
 import org.elasticsearch.search.aggregations.bucket.sampler.UnmappedSampler;
@@ -264,6 +265,7 @@ public class SearchModule extends AbstractModule {
         multibinderAggParser.addBinding().to(FilterParser.class);
         multibinderAggParser.addBinding().to(FiltersParser.class);
         multibinderAggParser.addBinding().to(SamplerParser.class);
+        multibinderAggParser.addBinding().to(DiversifiedSamplerParser.class);
         multibinderAggParser.addBinding().to(TermsParser.class);
         multibinderAggParser.addBinding().to(SignificantTermsParser.class);
         multibinderAggParser.addBinding().to(RangeParser.class);

+ 79 - 0
core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedSamplerAggregationBuilder.java

@@ -0,0 +1,79 @@
+/*
+ * 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.sampler;
+
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.search.aggregations.ValuesSourceAggregationBuilder;
+
+import java.io.IOException;
+
+/**
+ * Builder for the {@link Sampler} aggregation.
+ */
+public class DiversifiedSamplerAggregationBuilder extends ValuesSourceAggregationBuilder<DiversifiedSamplerAggregationBuilder> {
+
+    private int shardSize = SamplerAggregator.Factory.DEFAULT_SHARD_SAMPLE_SIZE;
+
+    int maxDocsPerValue = SamplerAggregator.DiversifiedFactory.MAX_DOCS_PER_VALUE_DEFAULT;
+    String executionHint = null;
+
+    /**
+     * Sole constructor.
+     */
+    public DiversifiedSamplerAggregationBuilder(String name) {
+        super(name, SamplerAggregator.DiversifiedFactory.TYPE.name());
+    }
+
+    /**
+     * Set the max num docs to be returned from each shard.
+     */
+    public DiversifiedSamplerAggregationBuilder shardSize(int shardSize) {
+        this.shardSize = shardSize;
+        return this;
+    }
+
+    public DiversifiedSamplerAggregationBuilder maxDocsPerValue(int maxDocsPerValue) {
+        this.maxDocsPerValue = maxDocsPerValue;
+        return this;
+    }
+
+    public DiversifiedSamplerAggregationBuilder executionHint(String executionHint) {
+        this.executionHint = executionHint;
+        return this;
+    }
+
+    @Override
+    protected XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException {
+        if (shardSize != SamplerAggregator.Factory.DEFAULT_SHARD_SAMPLE_SIZE) {
+            builder.field(SamplerAggregator.SHARD_SIZE_FIELD.getPreferredName(), shardSize);
+        }
+
+        if (maxDocsPerValue != SamplerAggregator.DiversifiedFactory.MAX_DOCS_PER_VALUE_DEFAULT) {
+            builder.field(SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD.getPreferredName(), maxDocsPerValue);
+        }
+        if (executionHint != null) {
+            builder.field(SamplerAggregator.EXECUTION_HINT_FIELD.getPreferredName(), executionHint);
+        }
+
+        return builder;
+    }
+
+
+}

+ 97 - 0
core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedSamplerParser.java

@@ -0,0 +1,97 @@
+/*
+ * 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.sampler;
+
+
+import org.elasticsearch.common.ParseField;
+import org.elasticsearch.common.ParseFieldMatcher;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
+import org.elasticsearch.search.aggregations.support.ValueType;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
+import org.elasticsearch.search.aggregations.support.ValuesSourceType;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ *
+ */
+public class DiversifiedSamplerParser extends AnyValuesSourceParser {
+
+    public DiversifiedSamplerParser() {
+        super(true, false);
+    }
+
+    @Override
+    public String type() {
+        return SamplerAggregator.DiversifiedFactory.TYPE.name();
+    }
+
+    @Override
+    protected ValuesSourceAggregatorFactory<ValuesSource> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
+            ValueType targetValueType, Map<ParseField, Object> otherOptions) {
+        SamplerAggregator.DiversifiedFactory factory = new SamplerAggregator.DiversifiedFactory(aggregationName, valuesSourceType,
+                targetValueType);
+        Integer shardSize = (Integer) otherOptions.get(SamplerAggregator.SHARD_SIZE_FIELD);
+        if (shardSize != null) {
+            factory.shardSize(shardSize);
+        }
+        Integer maxDocsPerValue = (Integer) otherOptions.get(SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD);
+        if (maxDocsPerValue != null) {
+            factory.maxDocsPerValue(maxDocsPerValue);
+        }
+        String executionHint = (String) otherOptions.get(SamplerAggregator.EXECUTION_HINT_FIELD);
+        if (executionHint != null) {
+            factory.executionHint(executionHint);
+        }
+        return factory;
+    }
+
+    @Override
+    protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
+            ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
+        if (token == XContentParser.Token.VALUE_NUMBER) {
+            if (parseFieldMatcher.match(currentFieldName, SamplerAggregator.SHARD_SIZE_FIELD)) {
+                int shardSize = parser.intValue();
+                otherOptions.put(SamplerAggregator.SHARD_SIZE_FIELD, shardSize);
+                return true;
+            } else if (parseFieldMatcher.match(currentFieldName, SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD)) {
+                int maxDocsPerValue = parser.intValue();
+                otherOptions.put(SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD, maxDocsPerValue);
+                return true;
+            }
+        } else if (token == XContentParser.Token.VALUE_STRING) {
+            if (parseFieldMatcher.match(currentFieldName, SamplerAggregator.EXECUTION_HINT_FIELD)) {
+                String executionHint = parser.text();
+                otherOptions.put(SamplerAggregator.EXECUTION_HINT_FIELD, executionHint);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public AggregatorFactory[] getFactoryPrototypes() {
+        return new AggregatorFactory[] { new SamplerAggregator.DiversifiedFactory(null, null, null) };
+    }
+
+}

+ 3 - 24
core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java

@@ -29,10 +29,7 @@ import java.io.IOException;
  */
 public class SamplerAggregationBuilder extends ValuesSourceAggregationBuilder<SamplerAggregationBuilder> {
 
-    private int shardSize = SamplerParser.DEFAULT_SHARD_SAMPLE_SIZE;
-
-    int maxDocsPerValue = SamplerParser.MAX_DOCS_PER_VALUE_DEFAULT;
-    String executionHint = null;
+    private int shardSize = SamplerAggregator.Factory.DEFAULT_SHARD_SAMPLE_SIZE;
 
     /**
      * Sole constructor.
@@ -49,28 +46,10 @@ public class SamplerAggregationBuilder extends ValuesSourceAggregationBuilder<Sa
         return this;
     }
 
-    public SamplerAggregationBuilder maxDocsPerValue(int maxDocsPerValue) {
-        this.maxDocsPerValue = maxDocsPerValue;
-        return this;
-    }
-
-    public SamplerAggregationBuilder executionHint(String executionHint) {
-        this.executionHint = executionHint;
-        return this;
-    }
-
     @Override
     protected XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException {
-        // builder.startObject();
-        if (shardSize != SamplerParser.DEFAULT_SHARD_SAMPLE_SIZE) {
-            builder.field(SamplerParser.SHARD_SIZE_FIELD.getPreferredName(), shardSize);
-        }
-
-        if (maxDocsPerValue != SamplerParser.MAX_DOCS_PER_VALUE_DEFAULT) {
-            builder.field(SamplerParser.MAX_DOCS_PER_VALUE_FIELD.getPreferredName(), maxDocsPerValue);
-        }
-        if (executionHint != null) {
-            builder.field(SamplerParser.EXECUTION_HINT_FIELD.getPreferredName(), executionHint);
+        if (shardSize != SamplerAggregator.Factory.DEFAULT_SHARD_SAMPLE_SIZE) {
+            builder.field(SamplerAggregator.SHARD_SIZE_FIELD.getPreferredName(), shardSize);
         }
 
         return builder;

+ 146 - 8
core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java

@@ -21,12 +21,16 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
 import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.ParseFieldMatcher;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.InternalAggregation.Type;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.NonCollectingAggregator;
 import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
@@ -34,14 +38,16 @@ import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
 import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
+import org.elasticsearch.search.aggregations.support.ValueType;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
-import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
+import org.elasticsearch.search.aggregations.support.ValuesSourceType;
 
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * Aggregate on only the top-scoring docs on a shard.
@@ -55,6 +61,10 @@ import java.util.Map;
  */
 public class SamplerAggregator extends SingleBucketAggregator {
 
+    public static final ParseField SHARD_SIZE_FIELD = new ParseField("shard_size");
+    public static final ParseField MAX_DOCS_PER_VALUE_FIELD = new ParseField("max_docs_per_value");
+    public static final ParseField EXECUTION_HINT_FIELD = new ParseField("execution_hint");
+
 
     public enum ExecutionMode {
 
@@ -182,34 +192,123 @@ public class SamplerAggregator extends SingleBucketAggregator {
 
     public static class Factory extends AggregatorFactory {
 
-        private int shardSize;
+        public static final int DEFAULT_SHARD_SAMPLE_SIZE = 100;
+
+        private int shardSize = DEFAULT_SHARD_SAMPLE_SIZE;
 
-        public Factory(String name, int shardSize) {
+        public Factory(String name) {
             super(name, InternalSampler.TYPE);
+        }
+
+        /**
+         * Set the max num docs to be returned from each shard.
+         */
+        public void shardSize(int shardSize) {
             this.shardSize = shardSize;
         }
 
+        /**
+         * Get the max num docs to be returned from each shard.
+         */
+        public int shardSize() {
+            return shardSize;
+        }
+
         @Override
         public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
                 List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
             return new SamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
         }
 
+        @Override
+        protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
+            builder.startObject();
+            builder.field(SHARD_SIZE_FIELD.getPreferredName(), shardSize);
+            builder.endObject();
+            return builder;
+        }
+
+        @Override
+        protected AggregatorFactory doReadFrom(String name, StreamInput in) throws IOException {
+            Factory factory = new Factory(name);
+            factory.shardSize = in.readVInt();
+            return factory;
+        }
+
+        @Override
+        protected void doWriteTo(StreamOutput out) throws IOException {
+            out.writeVInt(shardSize);
+        }
+
+        @Override
+        protected int doHashCode() {
+            return Objects.hash(shardSize);
+        }
+
+        @Override
+        protected boolean doEquals(Object obj) {
+            Factory other = (Factory) obj;
+            return Objects.equals(shardSize, other.shardSize);
+        }
+
     }
 
     public static class DiversifiedFactory extends ValuesSourceAggregatorFactory<ValuesSource> {
 
-        private int shardSize;
-        private int maxDocsPerValue;
-        private String executionHint;
+        public static final Type TYPE = new Type("diversified_sampler");
+
+        public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1;
+
+        private int shardSize = Factory.DEFAULT_SHARD_SAMPLE_SIZE;
+        private int maxDocsPerValue = MAX_DOCS_PER_VALUE_DEFAULT;
+        private String executionHint = null;
+
+        public DiversifiedFactory(String name, ValuesSourceType valueSourceType, ValueType valueType) {
+            super(name, TYPE, valueSourceType, valueType);
+        }
 
-        public DiversifiedFactory(String name, int shardSize, String executionHint, ValuesSourceParser.Input vsInput, int maxDocsPerValue) {
-            super(name, InternalSampler.TYPE, vsInput);
+        /**
+         * Set the max num docs to be returned from each shard.
+         */
+        public void shardSize(int shardSize) {
             this.shardSize = shardSize;
+        }
+
+        /**
+         * Get the max num docs to be returned from each shard.
+         */
+        public int shardSize() {
+            return shardSize;
+        }
+
+        /**
+         * Set the max num docs to be returned per value.
+         */
+        public void maxDocsPerValue(int maxDocsPerValue) {
             this.maxDocsPerValue = maxDocsPerValue;
+        }
+
+        /**
+         * Get the max num docs to be returned per value.
+         */
+        public int maxDocsPerValue() {
+            return maxDocsPerValue;
+        }
+
+        /**
+         * Set the execution hint.
+         */
+        public void executionHint(String executionHint) {
             this.executionHint = executionHint;
         }
 
+        /**
+         * Get the execution hint.
+         */
+        public String executionHint() {
+            return executionHint;
+        }
+
         @Override
         protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent,
                 boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
@@ -256,6 +355,45 @@ public class SamplerAggregator extends SingleBucketAggregator {
             };
         }
 
+        @Override
+        protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
+            builder.field(SHARD_SIZE_FIELD.getPreferredName(), shardSize);
+            builder.field(MAX_DOCS_PER_VALUE_FIELD.getPreferredName(), maxDocsPerValue);
+            if (executionHint != null) {
+                builder.field(EXECUTION_HINT_FIELD.getPreferredName(), executionHint);
+            }
+            return builder;
+        }
+
+        @Override
+        protected ValuesSourceAggregatorFactory<ValuesSource> innerReadFrom(String name, ValuesSourceType valuesSourceType,
+                ValueType targetValueType, StreamInput in) throws IOException {
+            DiversifiedFactory factory = new DiversifiedFactory(name, valuesSourceType, targetValueType);
+            factory.shardSize = in.readVInt();
+            factory.maxDocsPerValue = in.readVInt();
+            factory.executionHint = in.readOptionalString();
+            return factory;
+        }
+
+        @Override
+        protected void innerWriteTo(StreamOutput out) throws IOException {
+            out.writeVInt(shardSize);
+            out.writeVInt(maxDocsPerValue);
+            out.writeOptionalString(executionHint);
+        }
+
+        @Override
+        protected int innerHashCode() {
+            return Objects.hash(shardSize, maxDocsPerValue, executionHint);
+        }
+
+        @Override
+        protected boolean innerEquals(Object obj) {
+            DiversifiedFactory other = (DiversifiedFactory) obj;
+            return Objects.equals(shardSize, other.shardSize)
+                    && Objects.equals(maxDocsPerValue, other.maxDocsPerValue)
+                    && Objects.equals(executionHint, other.executionHint);
+        }
     }
 
     @Override

+ 13 - 48
core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java

@@ -18,12 +18,11 @@
  */
 package org.elasticsearch.search.aggregations.bucket.sampler;
 
-import org.elasticsearch.common.ParseField;
+
+import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.search.SearchParseException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
-import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
 import org.elasticsearch.search.internal.SearchContext;
 
 import java.io.IOException;
@@ -33,14 +32,6 @@ import java.io.IOException;
  */
 public class SamplerParser implements Aggregator.Parser {
 
-    public static final int DEFAULT_SHARD_SAMPLE_SIZE = 100;
-    public static final ParseField SHARD_SIZE_FIELD = new ParseField("shard_size");
-    public static final ParseField MAX_DOCS_PER_VALUE_FIELD = new ParseField("max_docs_per_value");
-    public static final ParseField EXECUTION_HINT_FIELD = new ParseField("execution_hint");
-    public static final boolean DEFAULT_USE_GLOBAL_ORDINALS = false;
-    public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1;
-
-
     @Override
     public String type() {
         return InternalSampler.TYPE.name();
@@ -51,60 +42,34 @@ public class SamplerParser implements Aggregator.Parser {
 
         XContentParser.Token token;
         String currentFieldName = null;
-        String executionHint = null;
-        int shardSize = DEFAULT_SHARD_SAMPLE_SIZE;
-        int maxDocsPerValue = MAX_DOCS_PER_VALUE_DEFAULT;
-        ValuesSourceParser vsParser = null;
-        boolean diversityChoiceMade = false;
-
-        vsParser = ValuesSourceParser.any(aggregationName, InternalSampler.TYPE, context).scriptable(true).formattable(false).build();
+        Integer shardSize = null;
 
         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
             if (token == XContentParser.Token.FIELD_NAME) {
                 currentFieldName = parser.currentName();
-            } else if (vsParser.token(currentFieldName, token, parser)) {
-                continue;
             } else if (token == XContentParser.Token.VALUE_NUMBER) {
-                if (context.parseFieldMatcher().match(currentFieldName, SHARD_SIZE_FIELD)) {
+                if (context.parseFieldMatcher().match(currentFieldName, SamplerAggregator.SHARD_SIZE_FIELD)) {
                     shardSize = parser.intValue();
-                } else if (context.parseFieldMatcher().match(currentFieldName, MAX_DOCS_PER_VALUE_FIELD)) {
-                    diversityChoiceMade = true;
-                    maxDocsPerValue = parser.intValue();
                 } else {
-                    throw new SearchParseException(context, "Unsupported property \"" + currentFieldName + "\" for aggregation \""
-                            + aggregationName, parser.getTokenLocation());
-                }
-            } else if (!vsParser.token(currentFieldName, token, parser)) {
-                if (context.parseFieldMatcher().match(currentFieldName, EXECUTION_HINT_FIELD)) {
-                    executionHint = parser.text();
-                } else {
-                    throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
-                            parser.getTokenLocation());
+                    throw new ParsingException(parser.getTokenLocation(),
+                            "Unsupported property \"" + currentFieldName + "\" for aggregation \"" + aggregationName);
                 }
             } else {
-                throw new SearchParseException(context, "Unsupported property \"" + currentFieldName + "\" for aggregation \""
-                        + aggregationName, parser.getTokenLocation());
+                throw new ParsingException(parser.getTokenLocation(),
+                        "Unsupported property \"" + currentFieldName + "\" for aggregation \"" + aggregationName);
             }
         }
 
-        ValuesSourceParser.Input vsInput = vsParser.input();
-        if (vsInput.valid()) {
-            return new SamplerAggregator.DiversifiedFactory(aggregationName, shardSize, executionHint, vsInput, maxDocsPerValue);
-        } else {
-            if (diversityChoiceMade) {
-                throw new SearchParseException(context, "Sampler aggregation has " + MAX_DOCS_PER_VALUE_FIELD.getPreferredName()
-                        + " setting but no \"field\" or \"script\" setting to provide values for aggregation \"" + aggregationName + "\"",
-                        parser.getTokenLocation());
-
-            }
-            return new SamplerAggregator.Factory(aggregationName, shardSize);
+        SamplerAggregator.Factory factory = new SamplerAggregator.Factory(aggregationName);
+        if (shardSize != null) {
+            factory.shardSize(shardSize);
         }
+        return factory;
     }
 
-    // NORELEASE implement this method when refactoring this aggregation
     @Override
     public AggregatorFactory[] getFactoryPrototypes() {
-        return null;
+        return new AggregatorFactory[] { new SamplerAggregator.Factory(null) };
     }
 
 }

+ 2 - 2
core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java

@@ -433,7 +433,7 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
     }
 
     @Override
-    public final int doHashCode() {
+    protected final int doHashCode() {
         return Objects.hash(field, format, missing, script, targetValueType, timeZone, valueType, valuesSourceType,
                 innerHashCode());
     }
@@ -446,7 +446,7 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
     }
 
     @Override
-    public final boolean doEquals(Object obj) {
+    protected final boolean doEquals(Object obj) {
         ValuesSourceAggregatorFactory<?> other = (ValuesSourceAggregatorFactory<?>) obj;
         if (!Objects.equals(field, other.field))
             return false;

+ 238 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/DiversifiedSamplerIT.java

@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.action.search.SearchType;
+import org.elasticsearch.index.query.TermQueryBuilder;
+import org.elasticsearch.search.aggregations.bucket.sampler.DiversifiedSamplerAggregationBuilder;
+import org.elasticsearch.search.aggregations.bucket.sampler.Sampler;
+import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator;
+import org.elasticsearch.search.aggregations.bucket.terms.Terms;
+import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
+import org.elasticsearch.search.aggregations.bucket.terms.TermsBuilder;
+import org.elasticsearch.search.aggregations.metrics.max.Max;
+import org.elasticsearch.test.ESIntegTestCase;
+
+import java.util.Collection;
+
+import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
+import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.sampler;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+
+/**
+ * Tests the Sampler aggregation
+ */
+@ESIntegTestCase.SuiteScopeTestCase
+public class DiversifiedSamplerIT extends ESIntegTestCase {
+
+    public static final int NUM_SHARDS = 2;
+
+    public String randomExecutionHint() {
+        return randomBoolean() ? null : randomFrom(SamplerAggregator.ExecutionMode.values()).toString();
+    }
+
+
+    @Override
+    public void setupSuiteScopeCluster() throws Exception {
+        assertAcked(prepareCreate("test").setSettings(SETTING_NUMBER_OF_SHARDS, NUM_SHARDS, SETTING_NUMBER_OF_REPLICAS, 0).addMapping(
+                "book", "author", "type=string,index=not_analyzed", "name", "type=string,index=analyzed", "genre",
+                "type=string,index=not_analyzed", "price", "type=float"));
+        createIndex("idx_unmapped");
+        // idx_unmapped_author is same as main index but missing author field
+        assertAcked(prepareCreate("idx_unmapped_author").setSettings(SETTING_NUMBER_OF_SHARDS, NUM_SHARDS, SETTING_NUMBER_OF_REPLICAS, 0)
+                .addMapping("book", "name", "type=string,index=analyzed", "genre", "type=string,index=not_analyzed", "price", "type=float"));
+
+        ensureGreen();
+        String data[] = {
+                // "id,cat,name,price,inStock,author_t,series_t,sequence_i,genre_s",
+                "0553573403,book,A Game of Thrones,7.99,true,George R.R. Martin,A Song of Ice and Fire,1,fantasy",
+                "0553579908,book,A Clash of Kings,7.99,true,George R.R. Martin,A Song of Ice and Fire,2,fantasy",
+                "055357342X,book,A Storm of Swords,7.99,true,George R.R. Martin,A Song of Ice and Fire,3,fantasy",
+                "0553293354,book,Foundation,17.99,true,Isaac Asimov,Foundation Novels,1,scifi",
+                "0812521390,book,The Black Company,6.99,false,Glen Cook,The Chronicles of The Black Company,1,fantasy",
+                "0812550706,book,Ender's Game,6.99,true,Orson Scott Card,Ender,1,scifi",
+                "0441385532,book,Jhereg,7.95,false,Steven Brust,Vlad Taltos,1,fantasy",
+                "0380014300,book,Nine Princes In Amber,6.99,true,Roger Zelazny,the Chronicles of Amber,1,fantasy",
+                "0805080481,book,The Book of Three,5.99,true,Lloyd Alexander,The Chronicles of Prydain,1,fantasy",
+                "080508049X,book,The Black Cauldron,5.99,true,Lloyd Alexander,The Chronicles of Prydain,2,fantasy"
+
+            };
+
+        for (int i = 0; i < data.length; i++) {
+            String[] parts = data[i].split(",");
+            client().prepareIndex("test", "book", "" + i).setSource("author", parts[5], "name", parts[2], "genre", parts[8], "price",Float.parseFloat(parts[3])).get();
+            client().prepareIndex("idx_unmapped_author", "book", "" + i).setSource("name", parts[2], "genre", parts[8],"price",Float.parseFloat(parts[3])).get();
+        }
+        client().admin().indices().refresh(new RefreshRequest("test")).get();
+    }
+
+    public void testIssue10719() throws Exception {
+        // Tests that we can refer to nested elements under a sample in a path
+        // statement
+        boolean asc = randomBoolean();
+        SearchResponse response = client().prepareSearch("test").setTypes("book").setSearchType(SearchType.QUERY_AND_FETCH)
+                .addAggregation(terms("genres")
+                        .field("genre")
+                        .order(Terms.Order.aggregation("sample>max_price.value", asc))
+                        .subAggregation(sampler("sample").shardSize(100)
+                                .subAggregation(max("max_price").field("price")))
+                ).execute().actionGet();
+        assertSearchResponse(response);
+        Terms genres = response.getAggregations().get("genres");
+        Collection<Bucket> genreBuckets = genres.getBuckets();
+        // For this test to be useful we need >1 genre bucket to compare
+        assertThat(genreBuckets.size(), greaterThan(1));
+        double lastMaxPrice = asc ? Double.MIN_VALUE : Double.MAX_VALUE;
+        for (Terms.Bucket genreBucket : genres.getBuckets()) {
+            Sampler sample = genreBucket.getAggregations().get("sample");
+            Max maxPriceInGenre = sample.getAggregations().get("max_price");
+            double price = maxPriceInGenre.getValue();
+            if (asc) {
+                assertThat(price, greaterThanOrEqualTo(lastMaxPrice));
+            } else {
+                assertThat(price, lessThanOrEqualTo(lastMaxPrice));
+            }
+            lastMaxPrice = price;
+        }
+
+    }
+
+    public void testSimpleDiversity() throws Exception {
+        int MAX_DOCS_PER_AUTHOR = 1;
+        DiversifiedSamplerAggregationBuilder sampleAgg = new DiversifiedSamplerAggregationBuilder("sample").shardSize(100);
+        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
+        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
+        SearchResponse response = client().prepareSearch("test")
+                .setSearchType(SearchType.QUERY_AND_FETCH)
+                .setQuery(new TermQueryBuilder("genre", "fantasy"))
+                .setFrom(0).setSize(60)
+                .addAggregation(sampleAgg)
+                .execute()
+                .actionGet();
+        assertSearchResponse(response);
+        Sampler sample = response.getAggregations().get("sample");
+        Terms authors = sample.getAggregations().get("authors");
+        Collection<Bucket> testBuckets = authors.getBuckets();
+
+        for (Terms.Bucket testBucket : testBuckets) {
+            assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_AUTHOR));
+        }
+    }
+
+    public void testNestedDiversity() throws Exception {
+        // Test multiple samples gathered under buckets made by a parent agg
+        int MAX_DOCS_PER_AUTHOR = 1;
+        TermsBuilder rootTerms = new TermsBuilder("genres").field("genre");
+
+        DiversifiedSamplerAggregationBuilder sampleAgg = new DiversifiedSamplerAggregationBuilder("sample").shardSize(100);
+        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
+        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
+
+        rootTerms.subAggregation(sampleAgg);
+        SearchResponse response = client().prepareSearch("test").setSearchType(SearchType.QUERY_AND_FETCH)
+                .addAggregation(rootTerms).execute().actionGet();
+        assertSearchResponse(response);
+        Terms genres = response.getAggregations().get("genres");
+        Collection<Bucket> genreBuckets = genres.getBuckets();
+        for (Terms.Bucket genreBucket : genreBuckets) {
+            Sampler sample = genreBucket.getAggregations().get("sample");
+            Terms authors = sample.getAggregations().get("authors");
+            Collection<Bucket> testBuckets = authors.getBuckets();
+
+            for (Terms.Bucket testBucket : testBuckets) {
+                assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_AUTHOR));
+            }
+        }
+    }
+
+    public void testNestedSamples() throws Exception {
+        // Test samples nested under samples
+        int MAX_DOCS_PER_AUTHOR = 1;
+        int MAX_DOCS_PER_GENRE = 2;
+        DiversifiedSamplerAggregationBuilder rootSample = new DiversifiedSamplerAggregationBuilder("genreSample").shardSize(100)
+                .field("genre")
+                .maxDocsPerValue(MAX_DOCS_PER_GENRE);
+
+        DiversifiedSamplerAggregationBuilder sampleAgg = new DiversifiedSamplerAggregationBuilder("sample").shardSize(100);
+        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
+        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
+        sampleAgg.subAggregation(new TermsBuilder("genres").field("genre"));
+
+        rootSample.subAggregation(sampleAgg);
+        SearchResponse response = client().prepareSearch("test").setSearchType(SearchType.QUERY_AND_FETCH).addAggregation(rootSample)
+                .execute().actionGet();
+        assertSearchResponse(response);
+        Sampler genreSample = response.getAggregations().get("genreSample");
+        Sampler sample = genreSample.getAggregations().get("sample");
+
+        Terms genres = sample.getAggregations().get("genres");
+        Collection<Bucket> testBuckets = genres.getBuckets();
+        for (Terms.Bucket testBucket : testBuckets) {
+            assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_GENRE));
+        }
+
+        Terms authors = sample.getAggregations().get("authors");
+        testBuckets = authors.getBuckets();
+        for (Terms.Bucket testBucket : testBuckets) {
+            assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_AUTHOR));
+        }
+    }
+
+    public void testPartiallyUnmappedDiversifyField() throws Exception {
+        // One of the indexes is missing the "author" field used for
+        // diversifying results
+        DiversifiedSamplerAggregationBuilder sampleAgg = new DiversifiedSamplerAggregationBuilder("sample").shardSize(100).field("author")
+                .maxDocsPerValue(1);
+        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
+        SearchResponse response = client().prepareSearch("idx_unmapped_author", "test").setSearchType(SearchType.QUERY_AND_FETCH)
+                .setQuery(new TermQueryBuilder("genre", "fantasy")).setFrom(0).setSize(60).addAggregation(sampleAgg)
+                .execute().actionGet();
+        assertSearchResponse(response);
+        Sampler sample = response.getAggregations().get("sample");
+        assertThat(sample.getDocCount(), greaterThan(0l));
+        Terms authors = sample.getAggregations().get("authors");
+        assertThat(authors.getBuckets().size(), greaterThan(0));
+    }
+
+    public void testWhollyUnmappedDiversifyField() throws Exception {
+        //All of the indices are missing the "author" field used for diversifying results
+        int MAX_DOCS_PER_AUTHOR = 1;
+        DiversifiedSamplerAggregationBuilder sampleAgg = new DiversifiedSamplerAggregationBuilder("sample").shardSize(100);
+        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
+        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
+        SearchResponse response = client().prepareSearch("idx_unmapped", "idx_unmapped_author").setSearchType(SearchType.QUERY_AND_FETCH)
+                .setQuery(new TermQueryBuilder("genre", "fantasy")).setFrom(0).setSize(60).addAggregation(sampleAgg).execute().actionGet();
+        assertSearchResponse(response);
+        Sampler sample = response.getAggregations().get("sample");
+        assertThat(sample.getDocCount(), equalTo(0l));
+        Terms authors = sample.getAggregations().get("authors");
+        assertNull(authors);
+    }
+
+}

+ 63 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/DiversifiedSamplerTests.java

@@ -0,0 +1,63 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.bucket;
+
+import org.elasticsearch.script.Script;
+import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
+import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator;
+import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.ExecutionMode;
+import org.elasticsearch.search.aggregations.support.ValuesSourceType;
+
+public class DiversifiedSamplerTests extends BaseAggregationTestCase<SamplerAggregator.DiversifiedFactory> {
+
+    @Override
+    protected final SamplerAggregator.DiversifiedFactory createTestAggregatorFactory() {
+        SamplerAggregator.DiversifiedFactory factory = new SamplerAggregator.DiversifiedFactory("foo", ValuesSourceType.ANY,
+                null);
+        String field = randomNumericField();
+        int randomFieldBranch = randomInt(3);
+        switch (randomFieldBranch) {
+        case 0:
+            factory.field(field);
+            break;
+        case 1:
+            factory.field(field);
+            factory.script(new Script("_value + 1"));
+            break;
+        case 2:
+            factory.script(new Script("doc[" + field + "] + 1"));
+            break;
+        }
+        if (randomBoolean()) {
+            factory.missing("MISSING");
+        }
+        if (randomBoolean()) {
+            factory.maxDocsPerValue(randomIntBetween(1, 1000));
+        }
+        if (randomBoolean()) {
+            factory.shardSize(randomIntBetween(1, 1000));
+        }
+        if (randomBoolean()) {
+            factory.executionHint(randomFrom(ExecutionMode.values()).toString());
+        }
+        return factory;
+    }
+
+}

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

@@ -123,7 +123,7 @@ public class SamplerIT extends ESIntegTestCase {
 
     }
 
-    public void testNoDiversity() throws Exception {
+    public void testSimpleSampler() throws Exception {
         SamplerAggregationBuilder sampleAgg = new SamplerAggregationBuilder("sample").shardSize(100);
         sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
         SearchResponse response = client().prepareSearch("test").setSearchType(SearchType.QUERY_AND_FETCH)
@@ -140,86 +140,6 @@ public class SamplerIT extends ESIntegTestCase {
         assertThat(maxBooksPerAuthor, equalTo(3l));
     }
 
-    public void testSimpleDiversity() throws Exception {
-        int MAX_DOCS_PER_AUTHOR = 1;
-        SamplerAggregationBuilder sampleAgg = new SamplerAggregationBuilder("sample").shardSize(100);
-        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
-        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
-        SearchResponse response = client().prepareSearch("test")
-                .setSearchType(SearchType.QUERY_AND_FETCH)
-                .setQuery(new TermQueryBuilder("genre", "fantasy"))
-                .setFrom(0).setSize(60)
-                .addAggregation(sampleAgg)
-                .execute()
-                .actionGet();
-        assertSearchResponse(response);
-        Sampler sample = response.getAggregations().get("sample");
-        Terms authors = sample.getAggregations().get("authors");
-        Collection<Bucket> testBuckets = authors.getBuckets();
-
-        for (Terms.Bucket testBucket : testBuckets) {
-            assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_AUTHOR));
-        }
-    }
-
-    public void testNestedDiversity() throws Exception {
-        // Test multiple samples gathered under buckets made by a parent agg
-        int MAX_DOCS_PER_AUTHOR = 1;
-        TermsBuilder rootTerms = new TermsBuilder("genres").field("genre");
-
-        SamplerAggregationBuilder sampleAgg = new SamplerAggregationBuilder("sample").shardSize(100);
-        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
-        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
-
-        rootTerms.subAggregation(sampleAgg);
-        SearchResponse response = client().prepareSearch("test").setSearchType(SearchType.QUERY_AND_FETCH)
-                .addAggregation(rootTerms).execute().actionGet();
-        assertSearchResponse(response);
-        Terms genres = response.getAggregations().get("genres");
-        Collection<Bucket> genreBuckets = genres.getBuckets();
-        for (Terms.Bucket genreBucket : genreBuckets) {
-            Sampler sample = genreBucket.getAggregations().get("sample");
-            Terms authors = sample.getAggregations().get("authors");
-            Collection<Bucket> testBuckets = authors.getBuckets();
-
-            for (Terms.Bucket testBucket : testBuckets) {
-                assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_AUTHOR));
-            }
-        }
-    }
-
-    public void testNestedSamples() throws Exception {
-        // Test samples nested under samples
-        int MAX_DOCS_PER_AUTHOR = 1;
-        int MAX_DOCS_PER_GENRE = 2;
-        SamplerAggregationBuilder rootSample = new SamplerAggregationBuilder("genreSample").shardSize(100).field("genre")
-                .maxDocsPerValue(MAX_DOCS_PER_GENRE);
-
-        SamplerAggregationBuilder sampleAgg = new SamplerAggregationBuilder("sample").shardSize(100);
-        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
-        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
-        sampleAgg.subAggregation(new TermsBuilder("genres").field("genre"));
-
-        rootSample.subAggregation(sampleAgg);
-        SearchResponse response = client().prepareSearch("test").setSearchType(SearchType.QUERY_AND_FETCH).addAggregation(rootSample)
-                .execute().actionGet();
-        assertSearchResponse(response);
-        Sampler genreSample = response.getAggregations().get("genreSample");
-        Sampler sample = genreSample.getAggregations().get("sample");
-
-        Terms genres = sample.getAggregations().get("genres");
-        Collection<Bucket> testBuckets = genres.getBuckets();
-        for (Terms.Bucket testBucket : testBuckets) {
-            assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_GENRE));
-        }
-
-        Terms authors = sample.getAggregations().get("authors");
-        testBuckets = authors.getBuckets();
-        for (Terms.Bucket testBucket : testBuckets) {
-            assertThat(testBucket.getDocCount(), lessThanOrEqualTo((long) NUM_SHARDS * MAX_DOCS_PER_AUTHOR));
-        }
-    }
-
     public void testUnmappedChildAggNoDiversity() throws Exception {
         SamplerAggregationBuilder sampleAgg = new SamplerAggregationBuilder("sample").shardSize(100);
         sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
@@ -254,34 +174,4 @@ public class SamplerIT extends ESIntegTestCase {
         assertThat(authors.getBuckets().size(), greaterThan(0));
     }
 
-    public void testPartiallyUnmappedDiversifyField() throws Exception {
-        // One of the indexes is missing the "author" field used for
-        // diversifying results
-        SamplerAggregationBuilder sampleAgg = new SamplerAggregationBuilder("sample").shardSize(100).field("author").maxDocsPerValue(1);
-        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
-        SearchResponse response = client().prepareSearch("idx_unmapped_author", "test").setSearchType(SearchType.QUERY_AND_FETCH)
-                .setQuery(new TermQueryBuilder("genre", "fantasy")).setFrom(0).setSize(60).addAggregation(sampleAgg)
-                .execute().actionGet();
-        assertSearchResponse(response);
-        Sampler sample = response.getAggregations().get("sample");
-        assertThat(sample.getDocCount(), greaterThan(0l));
-        Terms authors = sample.getAggregations().get("authors");
-        assertThat(authors.getBuckets().size(), greaterThan(0));
-    }
-
-    public void testWhollyUnmappedDiversifyField() throws Exception {
-        //All of the indices are missing the "author" field used for diversifying results
-        int MAX_DOCS_PER_AUTHOR = 1;
-        SamplerAggregationBuilder sampleAgg = new SamplerAggregationBuilder("sample").shardSize(100);
-        sampleAgg.field("author").maxDocsPerValue(MAX_DOCS_PER_AUTHOR).executionHint(randomExecutionHint());
-        sampleAgg.subAggregation(new TermsBuilder("authors").field("author"));
-        SearchResponse response = client().prepareSearch("idx_unmapped", "idx_unmapped_author").setSearchType(SearchType.QUERY_AND_FETCH)
-                .setQuery(new TermQueryBuilder("genre", "fantasy")).setFrom(0).setSize(60).addAggregation(sampleAgg).execute().actionGet();
-        assertSearchResponse(response);
-        Sampler sample = response.getAggregations().get("sample");
-        assertThat(sample.getDocCount(), equalTo(0l));
-        Terms authors = sample.getAggregations().get("authors");
-        assertNull(authors);
-    }
-
 }

+ 36 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/SamplerTests.java

@@ -0,0 +1,36 @@
+/*
+ * 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;
+
+import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
+import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator;
+
+public class SamplerTests extends BaseAggregationTestCase<SamplerAggregator.Factory> {
+
+    @Override
+    protected final SamplerAggregator.Factory createTestAggregatorFactory() {
+        SamplerAggregator.Factory factory = new SamplerAggregator.Factory("foo");
+        if (randomBoolean()) {
+            factory.shardSize(randomIntBetween(1, 1000));
+        }
+        return factory;
+    }
+
+}

+ 2 - 0
docs/reference/aggregations/bucket.asciidoc

@@ -19,6 +19,8 @@ include::bucket/datehistogram-aggregation.asciidoc[]
 
 include::bucket/daterange-aggregation.asciidoc[]
 
+include::bucket/diversified-sampler-aggregation.asciidoc[]
+
 include::bucket/filter-aggregation.asciidoc[]
 
 include::bucket/filters-aggregation.asciidoc[]

+ 154 - 0
docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc

@@ -0,0 +1,154 @@
+[[search-aggregations-bucket-sampler-aggregation]]
+=== Sampler Aggregation
+
+experimental[]
+
+A filtering aggregation used to limit any sub aggregations' processing to a sample of the top-scoring documents. Diversity settings are 
+used to limit the number of matches that share a common value such as an "author".
+
+.Example use cases:
+* Tightening the focus of analytics to high-relevance matches rather than the potentially very long tail of low-quality matches
+* Removing bias from analytics by ensuring fair representation of content from different sources
+* Reducing the running cost of aggregations that can produce useful results using only samples e.g. `significant_terms`
+ 
+
+Example:
+
+[source,js]
+--------------------------------------------------
+{
+    "query": {
+        "match": {
+            "text": "iphone"
+        }
+    },
+    "aggs": {
+        "sample": {
+            "sampler": {
+                "shard_size": 200,
+                "field" : "user.id"   
+            },
+            "aggs": {
+                "keywords": {
+                    "significant_terms": {
+                        "field": "text"
+                    }
+                }
+            }
+        }
+    }
+}
+--------------------------------------------------
+
+Response:
+
+[source,js]
+--------------------------------------------------
+{
+    ...
+        "aggregations": {
+        "sample": {
+            "doc_count": 1000,<1>
+            "keywords": {<2>
+                "doc_count": 1000,
+                "buckets": [
+                    ...
+                    {
+                        "key": "bend",
+                        "doc_count": 58,
+                        "score": 37.982536582524276,
+                        "bg_count": 103
+                    },
+                    ....
+}
+--------------------------------------------------
+
+<1> 1000 documents were sampled in total becase we asked for a maximum of 200 from an index with 5 shards. The cost of performing the nested significant_terms aggregation was therefore limited rather than unbounded.
+<2> The results of the significant_terms aggregation are not skewed by any single over-active Twitter user because we asked for a maximum of one tweet from any one user in our sample.
+
+
+==== shard_size
+
+The `shard_size` parameter limits how many top-scoring documents are collected in the sample processed on each shard.
+The default value is 100.
+
+==== Controlling diversity
+=`field` or `script` and `max_docs_per_value` settings are used to control the maximum number of documents collected on any one shard which share a common value.
+The choice of value (e.g. `author`) is loaded from a regular `field` or derived dynamically by a `script`.
+
+The aggregation will throw an error if the choice of field or script produces multiple values for a document.
+It is currently not possible to offer this form of de-duplication using many values, primarily due to concerns over efficiency.
+
+NOTE: Any good market researcher will tell you that when working with samples of data it is important
+that the sample represents a healthy variety of opinions rather than being skewed by any single voice.
+The same is true with aggregations and sampling with these diversify settings can offer a way to remove the bias in your content (an over-populated geography, a large spike in a timeline or an over-active forum spammer).  
+
+==== Field
+
+Controlling diversity using a field:
+
+[source,js]
+--------------------------------------------------
+{
+    "aggs" : {
+        "sample" : {
+            "diverisfied_sampler" : {
+                "field" : "author",
+                "max_docs_per_value" : 3
+            }
+        }
+    }
+}
+--------------------------------------------------
+
+Note that the `max_docs_per_value` setting applies on a per-shard basis only for the purposes of shard-local sampling.
+It is not intended as a way of providing a global de-duplication feature on search results.
+
+
+
+==== Script
+
+Controlling diversity using a script:
+
+[source,js]
+--------------------------------------------------
+{
+    "aggs" : {
+        "sample" : {
+            "diverisfied_sampler" : {
+                "script" : "doc['author'].value + '/' + doc['genre'].value"
+            }
+        }
+    }
+}
+--------------------------------------------------
+Note in the above example we chose to use the default `max_docs_per_value` setting of 1 and combine author and genre fields to ensure 
+each shard sample has, at most, one match for an author/genre pair.
+
+
+==== execution_hint
+
+When using the settings to control diversity, the optional `execution_hint` setting can influence the management of the values used for de-duplication.
+Each option will hold up to `shard_size` values in memory while performing de-duplication but the type of value held can be controlled as follows:
+ 
+ - hold field values directly (`map`)
+ - hold ordinals of the field as determined by the Lucene index (`global_ordinals`)
+ - hold hashes of the field values - with potential for hash collisions (`bytes_hash`)
+ 
+The default setting is to use `global_ordinals` if this information is available from the Lucene index and reverting to `map` if not.
+The `bytes_hash` setting may prove faster in some cases but introduces the possibility of false positives in de-duplication logic due to the possibility of hash collisions.
+Please note that Elasticsearch will ignore the choice of execution hint if it is not applicable and that there is no backward compatibility guarantee on these hints.
+
+==== Limitations
+
+===== Cannot be nested under `breadth_first` aggregations
+Being a quality-based filter the sampler aggregation needs access to the relevance score produced for each document.
+It therefore cannot be nested under a `terms` aggregation which has the `collect_mode` switched from the default `depth_first` mode to `breadth_first` as this discards scores.
+In this situation an error will be thrown.
+
+===== Limited de-dup logic.
+The de-duplication logic in the diversify settings applies only at a shard level so will not apply across shards.
+
+===== No specialized syntax for geo/date fields
+Currently the syntax for defining the diversifying values is defined by a choice of `field` or `script` - there is no added syntactical sugar for expressing geo or date units such as "1w" (1 week).
+This support may be added in a later release and users will currently have to create these sorts of values using a script.

+ 3 - 81
docs/reference/aggregations/bucket/sampler-aggregation.asciidoc

@@ -4,11 +4,9 @@
 experimental[]
 
 A filtering aggregation used to limit any sub aggregations' processing to a sample of the top-scoring documents.
-Optionally, diversity settings can be used to limit the number of matches that share a common value such as an "author".
 
 .Example use cases:
 * Tightening the focus of analytics to high-relevance matches rather than the potentially very long tail of low-quality matches
-* Removing bias from analytics by ensuring fair representation of content from different sources
 * Reducing the running cost of aggregations that can produce useful results using only samples e.g. `significant_terms`
  
 
@@ -25,8 +23,7 @@ Example:
     "aggs": {
         "sample": {
             "sampler": {
-                "shard_size": 200,
-                "field" : "user.id"   
+                "shard_size": 200
             },
             "aggs": {
                 "keywords": {
@@ -63,8 +60,7 @@ Response:
 }
 --------------------------------------------------
 
-<1> 1000 documents were sampled in total becase we asked for a maximum of 200 from an index with 5 shards. The cost of performing the nested significant_terms aggregation was therefore limited rather than unbounded.
-<2> The results of the significant_terms aggregation are not skewed by any single over-active Twitter user because we asked for a maximum of one tweet from any one user in our sample.
+<1> 1000 documents were sampled in total because we asked for a maximum of 200 from an index with 5 shards. The cost of performing the nested significant_terms aggregation was therefore limited rather than unbounded.
 
 
 ==== shard_size
@@ -72,83 +68,9 @@ Response:
 The `shard_size` parameter limits how many top-scoring documents are collected in the sample processed on each shard.
 The default value is 100.
 
-==== Controlling diversity
-Optionally, you can use the `field` or `script` and `max_docs_per_value` settings to control the maximum number of documents collected on any one shard which share a common value.
-The choice of value (e.g. `author`) is loaded from a regular `field` or derived dynamically by a `script`.
-
-The aggregation will throw an error if the choice of field or script produces multiple values for a document.
-It is currently not possible to offer this form of de-duplication using many values, primarily due to concerns over efficiency.
-
-NOTE: Any good market researcher will tell you that when working with samples of data it is important
-that the sample represents a healthy variety of opinions rather than being skewed by any single voice.
-The same is true with aggregations and sampling with these diversify settings can offer a way to remove the bias in your content (an over-populated geography, a large spike in a timeline or an over-active forum spammer).  
-
-==== Field
-
-Controlling diversity using a field:
-
-[source,js]
---------------------------------------------------
-{
-    "aggs" : {
-        "sample" : {
-            "sampler" : {
-                "field" : "author",
-                "max_docs_per_value" : 3
-            }
-        }
-    }
-}
---------------------------------------------------
-
-Note that the `max_docs_per_value` setting applies on a per-shard basis only for the purposes of shard-local sampling.
-It is not intended as a way of providing a global de-duplication feature on search results.
-
-
-
-==== Script
-
-Controlling diversity using a script:
-
-[source,js]
---------------------------------------------------
-{
-    "aggs" : {
-        "sample" : {
-            "sampler" : {
-                "script" : "doc['author'].value + '/' + doc['genre'].value"
-            }
-        }
-    }
-}
---------------------------------------------------
-Note in the above example we chose to use the default `max_docs_per_value` setting of 1 and combine author and genre fields to ensure 
-each shard sample has, at most, one match for an author/genre pair.
-
-
-==== execution_hint
-
-When using the settings to control diversity, the optional `execution_hint` setting can influence the management of the values used for de-duplication.
-Each option will hold up to `shard_size` values in memory while performing de-duplication but the type of value held can be controlled as follows:
- 
- - hold field values directly (`map`)
- - hold ordinals of the field as determined by the Lucene index (`global_ordinals`)
- - hold hashes of the field values - with potential for hash collisions (`bytes_hash`)
- 
-The default setting is to use `global_ordinals` if this information is available from the Lucene index and reverting to `map` if not.
-The `bytes_hash` setting may prove faster in some cases but introduces the possibility of false positives in de-duplication logic due to the possibility of hash collisions.
-Please note that Elasticsearch will ignore the choice of execution hint if it is not applicable and that there is no backward compatibility guarantee on these hints.
-
 ==== Limitations
 
 ===== Cannot be nested under `breadth_first` aggregations
 Being a quality-based filter the sampler aggregation needs access to the relevance score produced for each document.
 It therefore cannot be nested under a `terms` aggregation which has the `collect_mode` switched from the default `depth_first` mode to `breadth_first` as this discards scores.
-In this situation an error will be thrown.
-
-===== Limited de-dup logic.
-The de-duplication logic in the diversify settings applies only at a shard level so will not apply across shards.
-
-===== No specialized syntax for geo/date fields
-Currently the syntax for defining the diversifying values is defined by a choice of `field` or `script` - there is no added syntactical sugar for expressing geo or date units such as "1w" (1 week).
-This support may be added in a later release and users will currently have to create these sorts of values using a script.
+In this situation an error will be thrown.