Browse Source

Add parent-aggregation to parent-join module (#34210)

Add `parent` aggregation, a special single bucket aggregation that joins children documents to their parent.
Dominik Stadler 7 years ago
parent
commit
d351422215
20 changed files with 1488 additions and 98 deletions
  1. 3 3
      client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java
  2. 2 0
      docs/reference/aggregations/bucket.asciidoc
  3. 213 0
      docs/reference/aggregations/bucket/parent-aggregation.asciidoc
  4. 7 3
      modules/parent-join/src/main/java/org/elasticsearch/join/ParentJoinPlugin.java
  5. 60 0
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java
  6. 56 0
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/InternalParent.java
  7. 7 0
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/JoinAggregationBuilders.java
  8. 28 0
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/Parent.java
  9. 176 0
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregationBuilder.java
  10. 83 0
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java
  11. 0 1
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java
  12. 36 0
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParsedParent.java
  13. 11 5
      modules/parent-join/src/main/java/org/elasticsearch/join/spi/ParentJoinNamedXContentProvider.java
  14. 132 0
      modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/AbstractParentChildTestCase.java
  15. 1 84
      modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenIT.java
  16. 1 2
      modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenTests.java
  17. 327 0
      modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java
  18. 65 0
      modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/InternalParentTests.java
  19. 238 0
      modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentIT.java
  20. 42 0
      modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentTests.java

+ 3 - 3
client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java

@@ -628,7 +628,7 @@ public class RestHighLevelClientTests extends ESTestCase {
 
     public void testProvidedNamedXContents() {
         List<NamedXContentRegistry.Entry> namedXContents = RestHighLevelClient.getProvidedNamedXContents();
-        assertEquals(16, namedXContents.size());
+        assertEquals(17, namedXContents.size());
         Map<Class<?>, Integer> categories = new HashMap<>();
         List<String> names = new ArrayList<>();
         for (NamedXContentRegistry.Entry namedXContent : namedXContents) {
@@ -638,8 +638,8 @@ public class RestHighLevelClientTests extends ESTestCase {
                 categories.put(namedXContent.categoryClass, counter + 1);
             }
         }
-        assertEquals(4, categories.size());
-        assertEquals(Integer.valueOf(2), categories.get(Aggregation.class));
+        assertEquals("Had: " + categories, 4, categories.size());
+        assertEquals(Integer.valueOf(3), categories.get(Aggregation.class));
         assertTrue(names.contains(ChildrenAggregationBuilder.NAME));
         assertTrue(names.contains(MatrixStatsAggregationBuilder.NAME));
         assertEquals(Integer.valueOf(4), categories.get(EvaluationMetric.class));

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

@@ -49,6 +49,8 @@ include::bucket/missing-aggregation.asciidoc[]
 
 include::bucket/nested-aggregation.asciidoc[]
 
+include::bucket/parent-aggregation.asciidoc[]
+
 include::bucket/range-aggregation.asciidoc[]
 
 include::bucket/reverse-nested-aggregation.asciidoc[]

+ 213 - 0
docs/reference/aggregations/bucket/parent-aggregation.asciidoc

@@ -0,0 +1,213 @@
+[[search-aggregations-bucket-parent-aggregation]]
+=== Parent Aggregation
+
+A special single bucket aggregation that selects parent documents that have the specified type, as defined in a <<parent-join,`join` field>>.
+
+This aggregation has a single option:
+
+* `type` - The child type that should be selected.
+
+For example, let's say we have an index of questions and answers. The answer type has the following `join` field in the mapping:
+
+[source,js]
+--------------------------------------------------
+PUT parent_example
+{
+  "mappings": {
+    "_doc": {
+      "properties": {
+        "join": {
+          "type": "join",
+          "relations": {
+            "question": "answer"
+          }
+        }
+      }
+    }
+  }
+}
+--------------------------------------------------
+// CONSOLE
+
+The `question` document contain a tag field and the `answer` documents contain an owner field. With the `parent`
+aggregation the owner buckets can be mapped to the tag buckets in a single request even though the two fields exist in
+two different kinds of documents.
+
+An example of a question document:
+
+[source,js]
+--------------------------------------------------
+PUT parent_example/_doc/1
+{
+  "join": {
+    "name": "question"
+  },
+  "body": "<p>I have Windows 2003 server and i bought a new Windows 2008 server...",
+  "title": "Whats the best way to file transfer my site from server to a newer one?",
+  "tags": [
+    "windows-server-2003",
+    "windows-server-2008",
+    "file-transfer"
+  ]
+}
+--------------------------------------------------
+// CONSOLE
+// TEST[continued]
+
+Examples of `answer` documents:
+
+[source,js]
+--------------------------------------------------
+PUT parent_example/_doc/2?routing=1
+{
+  "join": {
+    "name": "answer",
+    "parent": "1"
+  },
+  "owner": {
+    "location": "Norfolk, United Kingdom",
+    "display_name": "Sam",
+    "id": 48
+  },
+  "body": "<p>Unfortunately you're pretty much limited to FTP...",
+  "creation_date": "2009-05-04T13:45:37.030"
+}
+
+PUT parent_example/_doc/3?routing=1&refresh
+{
+  "join": {
+    "name": "answer",
+    "parent": "1"
+  },
+  "owner": {
+    "location": "Norfolk, United Kingdom",
+    "display_name": "Troll",
+    "id": 49
+  },
+  "body": "<p>Use Linux...",
+  "creation_date": "2009-05-05T13:45:37.030"
+}
+--------------------------------------------------
+// CONSOLE
+// TEST[continued]
+
+The following request can be built that connects the two together:
+
+[source,js]
+--------------------------------------------------
+POST parent_example/_search?size=0
+{
+  "aggs": {
+    "top-names": {
+      "terms": {
+        "field": "owner.display_name.keyword",
+        "size": 10
+      },
+      "aggs": {
+        "to-questions": {
+          "parent": {
+            "type" : "answer"
+          },
+          "aggs": {
+            "top-tags": {
+              "terms": {
+                "field": "tags.keyword",
+                "size": 10
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+}
+--------------------------------------------------
+// CONSOLE
+// TEST[continued]
+
+<1> The `type` points to type / mapping with the name `answer`.
+
+The above example returns the top answer owners and per owner the top question tags.
+
+Possible response:
+
+[source,js]
+--------------------------------------------------
+{
+  "took": 9,
+  "timed_out": false,
+  "_shards": {
+    "total": 1,
+    "successful": 1,
+    "skipped": 0,
+    "failed": 0
+  },
+  "hits": {
+    "total": 3,
+    "max_score": null,
+    "hits": []
+  },
+  "aggregations": {
+    "top-names": {
+      "doc_count_error_upper_bound": 0,
+      "sum_other_doc_count": 0,
+      "buckets": [
+        {
+          "key": "Sam",
+          "doc_count": 1, <1>
+          "to-questions": {
+            "doc_count": 1, <2>
+            "top-tags": {
+              "doc_count_error_upper_bound": 0,
+              "sum_other_doc_count": 0,
+              "buckets": [
+                {
+                  "key": "file-transfer",
+                  "doc_count": 1
+                },
+                {
+                  "key": "windows-server-2003",
+                  "doc_count": 1
+                },
+                {
+                  "key": "windows-server-2008",
+                  "doc_count": 1
+                }
+              ]
+            }
+          }
+        },
+        {
+          "key": "Troll",
+          "doc_count": 1, <1>
+          "to-questions": {
+            "doc_count": 1, <2>
+            "top-tags": {
+              "doc_count_error_upper_bound": 0,
+              "sum_other_doc_count": 0,
+              "buckets": [
+                {
+                  "key": "file-transfer",
+                  "doc_count": 1
+                },
+                {
+                  "key": "windows-server-2003",
+                  "doc_count": 1
+                },
+                {
+                  "key": "windows-server-2008",
+                  "doc_count": 1
+                }
+              ]
+            }
+          }
+        }
+      ]
+    }
+  }
+}
+--------------------------------------------------
+// TESTRESPONSE[s/"took": 9/"took": $body.took/]
+
+<1> The number of answer documents with the tag `Sam`, `Troll`, etc.
+<2> The number of question documents that are related to answer documents with the tag `Sam`, `Troll`, etc.

+ 7 - 3
modules/parent-join/src/main/java/org/elasticsearch/join/ParentJoinPlugin.java

@@ -22,6 +22,8 @@ package org.elasticsearch.join;
 import org.elasticsearch.index.mapper.Mapper;
 import org.elasticsearch.join.aggregations.ChildrenAggregationBuilder;
 import org.elasticsearch.join.aggregations.InternalChildren;
+import org.elasticsearch.join.aggregations.InternalParent;
+import org.elasticsearch.join.aggregations.ParentAggregationBuilder;
 import org.elasticsearch.join.mapper.ParentJoinFieldMapper;
 import org.elasticsearch.join.query.HasChildQueryBuilder;
 import org.elasticsearch.join.query.HasParentQueryBuilder;
@@ -51,9 +53,11 @@ public class ParentJoinPlugin extends Plugin implements SearchPlugin, MapperPlug
 
     @Override
     public List<AggregationSpec> getAggregations() {
-        return Collections.singletonList(
-          new AggregationSpec(ChildrenAggregationBuilder.NAME, ChildrenAggregationBuilder::new, ChildrenAggregationBuilder::parse)
-              .addResultReader(InternalChildren::new)
+        return Arrays.asList(
+            new AggregationSpec(ChildrenAggregationBuilder.NAME, ChildrenAggregationBuilder::new, ChildrenAggregationBuilder::parse)
+                .addResultReader(InternalChildren::new),
+            new AggregationSpec(ParentAggregationBuilder.NAME, ParentAggregationBuilder::new, ParentAggregationBuilder::parse)
+                .addResultReader(InternalParent::new)
         );
     }
 

+ 60 - 0
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java

@@ -0,0 +1,60 @@
+/*
+ * 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.join.aggregations;
+
+import org.apache.lucene.search.Query;
+import org.elasticsearch.common.ParseField;
+import org.elasticsearch.search.aggregations.Aggregator;
+import org.elasticsearch.search.aggregations.AggregatorFactories;
+import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link BucketsAggregator} which resolves to the matching parent documents.
+ */
+public class ChildrenToParentAggregator extends ParentJoinAggregator {
+
+    static final ParseField TYPE_FIELD = new ParseField("type");
+
+    public ChildrenToParentAggregator(String name, AggregatorFactories factories,
+            SearchContext context, Aggregator parent, Query childFilter,
+            Query parentFilter, ValuesSource.Bytes.WithOrdinals valuesSource,
+            long maxOrd, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
+        super(name, factories, context, parent, childFilter, parentFilter, valuesSource, maxOrd, pipelineAggregators, metaData);
+    }
+
+    @Override
+    public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
+        return new InternalParent(name, bucketDocCount(owningBucketOrdinal),
+                bucketAggregations(owningBucketOrdinal), pipelineAggregators(), metaData());
+    }
+
+    @Override
+    public InternalAggregation buildEmptyAggregation() {
+        return new InternalParent(name, 0, buildEmptySubAggregations(), pipelineAggregators(),
+                metaData());
+    }
+}

+ 56 - 0
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/InternalParent.java

@@ -0,0 +1,56 @@
+/*
+ * 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.join.aggregations;
+
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Results of the {@link ChildrenToParentAggregator}.
+ */
+public class InternalParent extends InternalSingleBucketAggregation implements Parent {
+    public InternalParent(String name, long docCount, InternalAggregations aggregations, List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        super(name, docCount, aggregations, pipelineAggregators, metaData);
+    }
+
+    /**
+     * Read from a stream.
+     */
+    public InternalParent(StreamInput in) throws IOException {
+        super(in);
+    }
+
+    @Override
+    public String getWriteableName() {
+        return ParentAggregationBuilder.NAME;
+    }
+
+    @Override
+    protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
+        return new InternalParent(name, docCount, subAggregations, pipelineAggregators(), getMetaData());
+    }
+}

+ 7 - 0
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/JoinAggregationBuilders.java

@@ -26,4 +26,11 @@ public abstract class JoinAggregationBuilders {
     public static ChildrenAggregationBuilder children(String name, String childType) {
         return new ChildrenAggregationBuilder(name, childType);
     }
+
+    /**
+     * Create a new {@link Parent} aggregation with the given name.
+     */
+    public static ParentAggregationBuilder parent(String name, String childType) {
+        return new ParentAggregationBuilder(name, childType);
+    }
 }

+ 28 - 0
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/Parent.java

@@ -0,0 +1,28 @@
+/*
+ * 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.join.aggregations;
+
+import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
+
+/**
+ * An single bucket aggregation that translates child documents to their parent documents.
+ */
+public interface Parent extends SingleBucketAggregation {
+}

+ 176 - 0
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregationBuilder.java

@@ -0,0 +1,176 @@
+/*
+ * 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.join.aggregations;
+
+import org.apache.lucene.search.Query;
+import org.elasticsearch.common.ParsingException;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.join.mapper.ParentIdFieldMapper;
+import org.elasticsearch.join.mapper.ParentJoinFieldMapper;
+import org.elasticsearch.search.aggregations.AggregationBuilder;
+import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
+import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.support.FieldContext;
+import org.elasticsearch.search.aggregations.support.ValueType;
+import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.WithOrdinals;
+import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
+import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
+import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+import org.elasticsearch.search.aggregations.support.ValuesSourceType;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+public class ParentAggregationBuilder
+        extends ValuesSourceAggregationBuilder<WithOrdinals, ParentAggregationBuilder> {
+
+    public static final String NAME = "parent";
+
+    private final String childType;
+    private Query parentFilter;
+    private Query childFilter;
+
+    /**
+     * @param name
+     *            the name of this aggregation
+     * @param childType
+     *            the type of children documents
+     */
+    public ParentAggregationBuilder(String name, String childType) {
+        super(name, ValuesSourceType.BYTES, ValueType.STRING);
+        if (childType == null) {
+            throw new IllegalArgumentException("[childType] must not be null: [" + name + "]");
+        }
+        this.childType = childType;
+    }
+
+    protected ParentAggregationBuilder(ParentAggregationBuilder clone,
+                                         Builder factoriesBuilder, Map<String, Object> metaData) {
+        super(clone, factoriesBuilder, metaData);
+        this.childType = clone.childType;
+        this.childFilter = clone.childFilter;
+        this.parentFilter = clone.parentFilter;
+    }
+
+    @Override
+    protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map<String, Object> metaData) {
+        return new ParentAggregationBuilder(this, factoriesBuilder, metaData);
+    }
+
+    /**
+     * Read from a stream.
+     */
+    public ParentAggregationBuilder(StreamInput in) throws IOException {
+        super(in, ValuesSourceType.BYTES, ValueType.STRING);
+        childType = in.readString();
+    }
+
+    @Override
+    protected void innerWriteTo(StreamOutput out) throws IOException {
+        out.writeString(childType);
+    }
+
+    @Override
+    protected ValuesSourceAggregatorFactory<WithOrdinals, ?> innerBuild(SearchContext context,
+                                                                        ValuesSourceConfig<WithOrdinals> config,
+                                                                        AggregatorFactory<?> parent,
+                                                                        Builder subFactoriesBuilder) throws IOException {
+        return new ParentAggregatorFactory(name, config, childFilter, parentFilter, context, parent,
+                subFactoriesBuilder, metaData);
+    }
+
+    @Override
+    protected ValuesSourceConfig<WithOrdinals> resolveConfig(SearchContext context) {
+        ValuesSourceConfig<WithOrdinals> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
+        joinFieldResolveConfig(context, config);
+        return config;
+    }
+
+    private void joinFieldResolveConfig(SearchContext context, ValuesSourceConfig<WithOrdinals> config) {
+        ParentJoinFieldMapper parentJoinFieldMapper = ParentJoinFieldMapper.getMapper(context.mapperService());
+        ParentIdFieldMapper parentIdFieldMapper = parentJoinFieldMapper.getParentIdFieldMapper(childType, false);
+        if (parentIdFieldMapper != null) {
+            parentFilter = parentIdFieldMapper.getParentFilter();
+            childFilter = parentIdFieldMapper.getChildFilter(childType);
+            MappedFieldType fieldType = parentIdFieldMapper.fieldType();
+            final SortedSetDVOrdinalsIndexFieldData fieldData = context.getForField(fieldType);
+            config.fieldContext(new FieldContext(fieldType.name(), fieldData, fieldType));
+        } else {
+            config.unmapped(true);
+        }
+    }
+
+    @Override
+    protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
+        builder.field(ChildrenToParentAggregator.TYPE_FIELD.getPreferredName(), childType);
+        return builder;
+    }
+
+    public static ParentAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        String childType = null;
+
+        XContentParser.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) {
+                if ("type".equals(currentFieldName)) {
+                    childType = parser.text();
+                } else {
+                    throw new ParsingException(parser.getTokenLocation(),
+                            "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
+                }
+            } else {
+                throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + aggregationName + "].");
+            }
+        }
+
+        if (childType == null) {
+            throw new ParsingException(parser.getTokenLocation(),
+                    "Missing [child_type] field for parent aggregation [" + aggregationName + "]");
+        }
+
+        return new ParentAggregationBuilder(aggregationName, childType);
+    }
+
+    @Override
+    protected int innerHashCode() {
+        return Objects.hash(childType);
+    }
+
+    @Override
+    protected boolean innerEquals(Object obj) {
+        ParentAggregationBuilder other = (ParentAggregationBuilder) obj;
+        return Objects.equals(childType, other.childType);
+    }
+
+    @Override
+    public String getType() {
+        return NAME;
+    }
+}

+ 83 - 0
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java

@@ -0,0 +1,83 @@
+/*
+ * 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.join.aggregations;
+
+import org.apache.lucene.search.Query;
+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.NonCollectingAggregator;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.WithOrdinals;
+import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
+import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+public class ParentAggregatorFactory extends ValuesSourceAggregatorFactory<WithOrdinals, ParentAggregatorFactory> {
+
+    private final Query parentFilter;
+    private final Query childFilter;
+
+    public ParentAggregatorFactory(String name,
+                                   ValuesSourceConfig<WithOrdinals> config,
+                                   Query childFilter,
+                                   Query parentFilter,
+                                   SearchContext context,
+                                   AggregatorFactory<?> parent,
+                                   AggregatorFactories.Builder subFactoriesBuilder,
+                                   Map<String, Object> metaData) throws IOException {
+        super(name, config, context, parent, subFactoriesBuilder, metaData);
+
+        this.childFilter = childFilter;
+        this.parentFilter = parentFilter;
+    }
+
+    @Override
+    protected Aggregator createUnmapped(Aggregator parent,
+                                        List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
+        return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) {
+            @Override
+            public InternalAggregation buildEmptyAggregation() {
+                return new InternalParent(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
+            }
+        };
+    }
+
+    @Override
+    protected Aggregator doCreateInternal(WithOrdinals valuesSource,
+                                          Aggregator children,
+                                          boolean collectsFromSingleBucket,
+                                          List<PipelineAggregator> pipelineAggregators,
+                                          Map<String, Object> metaData) throws IOException {
+
+        long maxOrd = valuesSource.globalMaxOrd(context.searcher());
+        if (collectsFromSingleBucket) {
+            return new ChildrenToParentAggregator(name, factories, context, children, childFilter,
+                parentFilter, valuesSource, maxOrd, pipelineAggregators, metaData);
+        } else {
+            return asMultiBucketAggregator(this, context, children);
+        }
+    }
+}

+ 0 - 1
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java

@@ -53,5 +53,4 @@ public class ParentToChildrenAggregator extends ParentJoinAggregator {
         return new InternalChildren(name, 0, buildEmptySubAggregations(), pipelineAggregators(),
                 metaData());
     }
-
 }

+ 36 - 0
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParsedParent.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.join.aggregations;
+
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
+
+import java.io.IOException;
+
+public class ParsedParent extends ParsedSingleBucketAggregation implements Parent {
+
+    @Override
+    public String getType() {
+        return ParentAggregationBuilder.NAME;
+    }
+
+    public static ParsedParent fromXContent(XContentParser parser, final String name) throws IOException {
+        return parseXContent(parser, new ParsedParent(), name);
+    }
+}

+ 11 - 5
modules/parent-join/src/main/java/org/elasticsearch/join/spi/ParentJoinNamedXContentProvider.java

@@ -23,20 +23,26 @@ import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.xcontent.ContextParser;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.join.aggregations.ChildrenAggregationBuilder;
+import org.elasticsearch.join.aggregations.ParentAggregationBuilder;
 import org.elasticsearch.join.aggregations.ParsedChildren;
+import org.elasticsearch.join.aggregations.ParsedParent;
 import org.elasticsearch.plugins.spi.NamedXContentProvider;
 import org.elasticsearch.search.aggregations.Aggregation;
 
+import java.util.Arrays;
 import java.util.List;
 
-import static java.util.Collections.singletonList;
-
 public class ParentJoinNamedXContentProvider implements NamedXContentProvider {
 
     @Override
     public List<NamedXContentRegistry.Entry> getNamedXContentParsers() {
-        ParseField parseField = new ParseField(ChildrenAggregationBuilder.NAME);
-        ContextParser<Object, Aggregation> contextParser = (p, name) -> ParsedChildren.fromXContent(p, (String) name);
-        return singletonList(new NamedXContentRegistry.Entry(Aggregation.class, parseField, contextParser));
+        ParseField parseFieldChildren = new ParseField(ChildrenAggregationBuilder.NAME);
+        ParseField parseFieldParent = new ParseField(ParentAggregationBuilder.NAME);
+        ContextParser<Object, Aggregation> contextParserChildren = (p, name) -> ParsedChildren.fromXContent(p, (String) name);
+        ContextParser<Object, Aggregation> contextParserParent = (p, name) -> ParsedParent.fromXContent(p, (String) name);
+        return Arrays.asList(
+            new NamedXContentRegistry.Entry(Aggregation.class, parseFieldChildren, contextParserChildren),
+            new NamedXContentRegistry.Entry(Aggregation.class, parseFieldParent, contextParserParent)
+        );
     }
 }

+ 132 - 0
modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/AbstractParentChildTestCase.java

@@ -0,0 +1,132 @@
+/*
+ * 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.join.aggregations;
+
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.join.query.ParentChildTestCase;
+import org.junit.Before;
+
+/**
+ * Small base test-class which combines stuff used for Children and Parent aggregation tests
+ */
+public abstract class AbstractParentChildTestCase extends ParentChildTestCase {
+    protected final Map<String, Control> categoryToControl = new HashMap<>();
+    protected final Map<String, ParentControl> articleToControl = new HashMap<>();
+
+    @Before
+    public void setupCluster() throws Exception {
+        assertAcked(
+            prepareCreate("test")
+                .addMapping("doc",
+                    addFieldMappings(buildParentJoinFieldMappingFromSimplifiedDef("join_field", true, "article", "comment"),
+                        "commenter", "keyword", "category", "keyword"))
+        );
+
+        List<IndexRequestBuilder> requests = new ArrayList<>();
+        String[] uniqueCategories = new String[randomIntBetween(1, 25)];
+        for (int i = 0; i < uniqueCategories.length; i++) {
+            uniqueCategories[i] = Integer.toString(i);
+        }
+        int catIndex = 0;
+
+        int numParentDocs = randomIntBetween(uniqueCategories.length, uniqueCategories.length * 5);
+        for (int i = 0; i < numParentDocs; i++) {
+            String id = "article-" + i;
+
+            // TODO: this array is always of length 1, and testChildrenAggs fails if this is changed
+            String[] categories = new String[randomIntBetween(1,1)];
+            for (int j = 0; j < categories.length; j++) {
+                String category = categories[j] = uniqueCategories[catIndex++ % uniqueCategories.length];
+                Control control = categoryToControl.computeIfAbsent(category, Control::new);
+                control.articleIds.add(id);
+                articleToControl.put(id, new ParentControl(category));
+            }
+
+            IndexRequestBuilder indexRequest = createIndexRequest("test", "article", id, null, "category", categories, "randomized", true);
+            requests.add(indexRequest);
+        }
+
+        String[] commenters = new String[randomIntBetween(5, 50)];
+        for (int i = 0; i < commenters.length; i++) {
+            commenters[i] = Integer.toString(i);
+        }
+
+        int id = 0;
+        for (Control control : categoryToControl.values()) {
+            for (String articleId : control.articleIds) {
+                int numChildDocsPerParent = randomIntBetween(0, 5);
+                for (int i = 0; i < numChildDocsPerParent; i++) {
+                    String commenter = commenters[id % commenters.length];
+                    String idValue = "comment-" + id++;
+                    control.commentIds.add(idValue);
+                    Set<String> ids = control.commenterToCommentId.computeIfAbsent(commenter, k -> new HashSet<>());
+                    ids.add(idValue);
+
+                    articleToControl.get(articleId).commentIds.add(idValue);
+
+                    IndexRequestBuilder indexRequest = createIndexRequest("test", "comment", idValue,
+                        articleId, "commenter", commenter, "randomized", true);
+                    requests.add(indexRequest);
+                }
+            }
+        }
+
+        requests.add(createIndexRequest("test", "article", "a", null, "category", new String[]{"a"}, "randomized", false));
+        requests.add(createIndexRequest("test", "article", "b", null, "category", new String[]{"a", "b"}, "randomized", false));
+        requests.add(createIndexRequest("test", "article", "c", null, "category", new String[]{"a", "b", "c"}, "randomized", false));
+        requests.add(createIndexRequest("test", "article", "d", null, "category", new String[]{"c"}, "randomized", false));
+        requests.add(createIndexRequest("test", "comment", "e", "a"));
+        requests.add(createIndexRequest("test", "comment", "f", "c"));
+
+        indexRandom(true, requests);
+        ensureSearchable("test");
+    }
+
+
+    protected static final class Control {
+
+        final String category;
+        final Set<String> articleIds = new HashSet<>();
+        final Set<String> commentIds = new HashSet<>();
+        final Map<String, Set<String>> commenterToCommentId = new HashMap<>();
+
+        private Control(String category) {
+            this.category = category;
+        }
+    }
+
+    protected static final class ParentControl {
+        final String category;
+        final Set<String> commentIds = new HashSet<>();
+
+        private ParentControl(String category) {
+            this.category = category;
+        }
+    }
+}

+ 1 - 84
modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenIT.java

@@ -25,7 +25,6 @@ import org.elasticsearch.action.update.UpdateResponse;
 import org.elasticsearch.client.Requests;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.join.query.ParentChildTestCase;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
 import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -33,11 +32,8 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.Sum;
 import org.elasticsearch.search.aggregations.metrics.TopHits;
 import org.elasticsearch.search.sort.SortOrder;
-import org.junit.Before;
 
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -58,80 +54,7 @@ import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.sameInstance;
 
-public class ChildrenIT extends ParentChildTestCase {
-
-
-    private static final Map<String, Control> categoryToControl = new HashMap<>();
-
-
-    @Before
-    public void setupCluster() throws Exception {
-        categoryToControl.clear();
-        assertAcked(
-            prepareCreate("test")
-                .addMapping("doc",
-                    addFieldMappings(buildParentJoinFieldMappingFromSimplifiedDef("join_field", true, "article", "comment"),
-                        "commenter", "keyword", "category", "keyword"))
-        );
-
-        List<IndexRequestBuilder> requests = new ArrayList<>();
-        String[] uniqueCategories = new String[randomIntBetween(1, 25)];
-        for (int i = 0; i < uniqueCategories.length; i++) {
-            uniqueCategories[i] = Integer.toString(i);
-        }
-        int catIndex = 0;
-
-        int numParentDocs = randomIntBetween(uniqueCategories.length, uniqueCategories.length * 5);
-        for (int i = 0; i < numParentDocs; i++) {
-            String id = "article-" + i;
-
-            // TODO: this array is always of length 1, and testChildrenAggs fails if this is changed
-            String[] categories = new String[randomIntBetween(1,1)];
-            for (int j = 0; j < categories.length; j++) {
-                String category = categories[j] = uniqueCategories[catIndex++ % uniqueCategories.length];
-                Control control = categoryToControl.get(category);
-                if (control == null) {
-                    categoryToControl.put(category, control = new Control());
-                }
-                control.articleIds.add(id);
-            }
-
-            requests.add(createIndexRequest("test", "article", id, null, "category", categories, "randomized", true));
-        }
-
-        String[] commenters = new String[randomIntBetween(5, 50)];
-        for (int i = 0; i < commenters.length; i++) {
-            commenters[i] = Integer.toString(i);
-        }
-
-        int id = 0;
-        for (Control control : categoryToControl.values()) {
-            for (String articleId : control.articleIds) {
-                int numChildDocsPerParent = randomIntBetween(0, 5);
-                for (int i = 0; i < numChildDocsPerParent; i++) {
-                    String commenter = commenters[id % commenters.length];
-                    String idValue = "comment-" + id++;
-                    control.commentIds.add(idValue);
-                    Set<String> ids = control.commenterToCommentId.get(commenter);
-                    if (ids == null) {
-                        control.commenterToCommentId.put(commenter, ids = new HashSet<>());
-                    }
-                    ids.add(idValue);
-                    requests.add(createIndexRequest("test", "comment", idValue, articleId, "commenter", commenter));
-                }
-            }
-        }
-
-        requests.add(createIndexRequest("test", "article", "a", null, "category", new String[]{"a"}, "randomized", false));
-        requests.add(createIndexRequest("test", "article", "b", null, "category", new String[]{"a", "b"}, "randomized", false));
-        requests.add(createIndexRequest("test", "article", "c", null, "category", new String[]{"a", "b", "c"}, "randomized", false));
-        requests.add(createIndexRequest("test", "article", "d", null, "category", new String[]{"c"}, "randomized", false));
-        requests.add(createIndexRequest("test", "comment", "e", "a"));
-        requests.add(createIndexRequest("test", "comment", "f", "c"));
-
-        indexRandom(true, requests);
-        ensureSearchable("test");
-    }
+public class ChildrenIT extends AbstractParentChildTestCase {
 
     public void testChildrenAggs() throws Exception {
         SearchResponse searchResponse = client().prepareSearch("test")
@@ -455,10 +378,4 @@ public class ChildrenIT extends ParentChildTestCase {
         children = parents.getBuckets().get(0).getAggregations().get("child_docs");
         assertThat(children.getDocCount(), equalTo(2L));
     }
-
-    private static final class Control {
-        final Set<String> articleIds = new HashSet<>();
-        final Set<String> commentIds = new HashSet<>();
-        final Map<String, Set<String>> commenterToCommentId = new HashMap<>();
-    }
 }

+ 1 - 2
modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenTests.java

@@ -37,8 +37,7 @@ public class ChildrenTests extends BaseAggregationTestCase<ChildrenAggregationBu
     protected ChildrenAggregationBuilder createTestAggregatorBuilder() {
         String name = randomAlphaOfLengthBetween(3, 20);
         String childType = randomAlphaOfLengthBetween(5, 40);
-        ChildrenAggregationBuilder factory = new ChildrenAggregationBuilder(name, childType);
-        return factory;
+        return new ChildrenAggregationBuilder(name, childType);
     }
 
 }

+ 327 - 0
modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java

@@ -0,0 +1,327 @@
+/*
+ * 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.join.aggregations;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.Version;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.index.Index;
+import org.elasticsearch.index.mapper.ContentPath;
+import org.elasticsearch.index.mapper.IdFieldMapper;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.mapper.Mapper;
+import org.elasticsearch.index.mapper.MapperService;
+import org.elasticsearch.index.mapper.NumberFieldMapper;
+import org.elasticsearch.index.mapper.Uid;
+import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.join.mapper.MetaJoinFieldMapper;
+import org.elasticsearch.join.mapper.ParentJoinFieldMapper;
+import org.elasticsearch.search.aggregations.Aggregation;
+import org.elasticsearch.search.aggregations.AggregationBuilder;
+import org.elasticsearch.search.aggregations.AggregatorTestCase;
+import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
+import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
+import org.elasticsearch.search.aggregations.metrics.InternalMin;
+import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder;
+import org.elasticsearch.search.aggregations.support.ValueType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.function.Consumer;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class ChildrenToParentAggregatorTests extends AggregatorTestCase {
+
+    private static final String CHILD_TYPE = "child_type";
+    private static final String PARENT_TYPE = "parent_type";
+
+    public void testNoDocs() throws IOException {
+        Directory directory = newDirectory();
+
+        RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
+        // intentionally not writing any docs
+        indexWriter.close();
+        IndexReader indexReader = DirectoryReader.open(directory);
+
+        testCase(new MatchAllDocsQuery(), newSearcher(indexReader, false, true), childrenToParent -> {
+            assertEquals(0, childrenToParent.getDocCount());
+            Aggregation parentAggregation = childrenToParent.getAggregations().get("in_parent");
+            assertEquals(0, childrenToParent.getDocCount());
+            assertNotNull("Aggregations: " + childrenToParent.getAggregations().asMap(), parentAggregation);
+            assertEquals(Double.POSITIVE_INFINITY, ((InternalMin) parentAggregation).getValue(), Double.MIN_VALUE);
+        });
+        indexReader.close();
+        directory.close();
+    }
+
+    public void testParentChild() throws IOException {
+        Directory directory = newDirectory();
+        RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
+
+        final Map<String, Tuple<Integer, Integer>> expectedParentChildRelations = setupIndex(indexWriter);
+        indexWriter.close();
+
+        IndexReader indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(directory),
+                new ShardId(new Index("foo", "_na_"), 1));
+        // TODO set "maybeWrap" to true for IndexSearcher once #23338 is resolved
+        IndexSearcher indexSearcher = newSearcher(indexReader, false, true);
+
+        // verify with all documents
+        testCase(new MatchAllDocsQuery(), indexSearcher, parent -> {
+            int expectedTotalParents = 0;
+            int expectedMinValue = Integer.MAX_VALUE;
+            for (Tuple<Integer, Integer> expectedValues : expectedParentChildRelations.values()) {
+                expectedTotalParents++;
+                expectedMinValue = Math.min(expectedMinValue, expectedValues.v2());
+            }
+            assertEquals("Having " + parent.getDocCount() + " docs and aggregation results: " +
+                    parent.getAggregations().asMap(),
+                expectedTotalParents, parent.getDocCount());
+            assertEquals(expectedMinValue, ((InternalMin) parent.getAggregations().get("in_parent")).getValue(), Double.MIN_VALUE);
+        });
+
+        // verify for each children
+        for (String parent : expectedParentChildRelations.keySet()) {
+            testCase(new TermInSetQuery(IdFieldMapper.NAME, Uid.encodeId("child0_" + parent)),
+                indexSearcher, aggregation -> {
+                assertEquals("Expected one result for min-aggregation for parent: " + parent +
+                        ", but had aggregation-results: " + aggregation,
+                    1, aggregation.getDocCount());
+                assertEquals(expectedParentChildRelations.get(parent).v2(),
+                        ((InternalMin) aggregation.getAggregations().get("in_parent")).getValue(), Double.MIN_VALUE);
+            });
+        }
+
+        indexReader.close();
+        directory.close();
+    }
+
+
+    public void testParentChildTerms() throws IOException {
+        Directory directory = newDirectory();
+        RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
+
+        final Map<String, Tuple<Integer, Integer>> expectedParentChildRelations = setupIndex(indexWriter);
+        indexWriter.close();
+
+        SortedMap<Integer, Long> entries = new TreeMap<>();
+        for (Tuple<Integer, Integer> value : expectedParentChildRelations.values()) {
+            Long l = entries.computeIfAbsent(value.v2(), integer -> 0L);
+            entries.put(value.v2(), l+1);
+        }
+        List<Map.Entry<Integer, Long>> sortedValues = new ArrayList<>(entries.entrySet());
+        sortedValues.sort((o1, o2) -> {
+            // sort larger values first
+            int ret = o2.getValue().compareTo(o1.getValue());
+            if(ret != 0) {
+                return ret;
+            }
+
+            // on equal value, sort by key
+            return o1.getKey().compareTo(o2.getKey());
+        });
+
+        IndexReader indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(directory),
+            new ShardId(new Index("foo", "_na_"), 1));
+        // TODO set "maybeWrap" to true for IndexSearcher once #23338 is resolved
+        IndexSearcher indexSearcher = newSearcher(indexReader, false, true);
+
+        // verify a terms-aggregation inside the parent-aggregation
+        testCaseTerms(new MatchAllDocsQuery(), indexSearcher, parent -> {
+            assertNotNull(parent);
+            LongTerms valueTerms = parent.getAggregations().get("value_terms");
+            assertNotNull(valueTerms);
+
+            List<LongTerms.Bucket> valueTermsBuckets = valueTerms.getBuckets();
+            assertNotNull(valueTermsBuckets);
+            assertEquals("Had: " + parent, sortedValues.size(), valueTermsBuckets.size());
+            int i = 0;
+            for (Map.Entry<Integer, Long> entry : sortedValues) {
+                LongTerms.Bucket bucket = valueTermsBuckets.get(i);
+                assertEquals(entry.getKey().longValue(), bucket.getKeyAsNumber());
+                assertEquals(entry.getValue(), (Long)bucket.getDocCount());
+
+                i++;
+            }
+        });
+
+        indexReader.close();
+        directory.close();
+    }
+
+    public void testTermsParentChildTerms() throws IOException {
+        Directory directory = newDirectory();
+        RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
+
+        final Map<String, Tuple<Integer, Integer>> expectedParentChildRelations = setupIndex(indexWriter);
+        indexWriter.close();
+
+        SortedMap<Integer, Long> sortedValues = new TreeMap<>();
+        for (Tuple<Integer, Integer> value : expectedParentChildRelations.values()) {
+            Long l = sortedValues.computeIfAbsent(value.v2(), integer -> 0L);
+            sortedValues.put(value.v2(), l+1);
+        }
+
+        IndexReader indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(directory),
+            new ShardId(new Index("foo", "_na_"), 1));
+        // TODO set "maybeWrap" to true for IndexSearcher once #23338 is resolved
+        IndexSearcher indexSearcher = newSearcher(indexReader, false, true);
+
+        // verify a terms-aggregation inside the parent-aggregation which itself is inside a
+        // terms-aggregation on the child-documents
+        testCaseTermsParentTerms(new MatchAllDocsQuery(), indexSearcher, longTerms -> {
+            assertNotNull(longTerms);
+
+            for (LongTerms.Bucket bucket : longTerms.getBuckets()) {
+                assertNotNull(bucket);
+                assertNotNull(bucket.getKeyAsString());
+            }
+        });
+
+        indexReader.close();
+        directory.close();
+    }
+
+    private static Map<String, Tuple<Integer, Integer>> setupIndex(RandomIndexWriter iw) throws IOException {
+        Map<String, Tuple<Integer, Integer>> expectedValues = new HashMap<>();
+        int numParents = randomIntBetween(1, 10);
+        for (int i = 0; i < numParents; i++) {
+            String parent = "parent" + i;
+            int randomValue = randomIntBetween(0, 100);
+            List<Field> parentDocument = createParentDocument(parent, randomValue);
+            /*long parentDocId =*/ iw.addDocument(parentDocument);
+            //System.out.println("Parent: " + parent + ": " + randomValue + ", id: " + parentDocId);
+            int numChildren = randomIntBetween(1, 10);
+            int minValue = Integer.MAX_VALUE;
+            for (int c = 0; c < numChildren; c++) {
+                minValue = Math.min(minValue, randomValue);
+                int randomSubValue = randomIntBetween(0, 100);
+                List<Field> childDocument = createChildDocument("child" + c + "_" + parent, parent, randomSubValue);
+                /*long childDocId =*/ iw.addDocument(childDocument);
+                //System.out.println("Child: " + "child" + c + "_" + parent + ": " + randomSubValue + ", id: " + childDocId);
+            }
+            expectedValues.put(parent, new Tuple<>(numChildren, minValue));
+        }
+        return expectedValues;
+    }
+
+    private static List<Field> createParentDocument(String id, int value) {
+        return Arrays.asList(
+                new StringField(IdFieldMapper.NAME, Uid.encodeId(id), Field.Store.NO),
+                new StringField("join_field", PARENT_TYPE, Field.Store.NO),
+                createJoinField(PARENT_TYPE, id),
+                new SortedNumericDocValuesField("number", value)
+        );
+    }
+
+    private static List<Field> createChildDocument(String childId, String parentId, int value) {
+        return Arrays.asList(
+                new StringField(IdFieldMapper.NAME, Uid.encodeId(childId), Field.Store.NO),
+                new StringField("join_field", CHILD_TYPE, Field.Store.NO),
+                createJoinField(PARENT_TYPE, parentId),
+            new SortedNumericDocValuesField("subNumber", value)
+        );
+    }
+
+    private static SortedDocValuesField createJoinField(String parentType, String id) {
+        return new SortedDocValuesField("join_field#" + parentType, new BytesRef(id));
+    }
+
+    @Override
+    protected MapperService mapperServiceMock() {
+        ParentJoinFieldMapper joinFieldMapper = createJoinFieldMapper();
+        MapperService mapperService = mock(MapperService.class);
+        MetaJoinFieldMapper.MetaJoinFieldType metaJoinFieldType = mock(MetaJoinFieldMapper.MetaJoinFieldType.class);
+        when(metaJoinFieldType.getMapper()).thenReturn(joinFieldMapper);
+        when(mapperService.fullName("_parent_join")).thenReturn(metaJoinFieldType);
+        return mapperService;
+    }
+
+    private static ParentJoinFieldMapper createJoinFieldMapper() {
+        Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
+        return new ParentJoinFieldMapper.Builder("join_field")
+                .addParent(PARENT_TYPE, Collections.singleton(CHILD_TYPE))
+                .build(new Mapper.BuilderContext(settings, new ContentPath(0)));
+    }
+
+    private void testCase(Query query, IndexSearcher indexSearcher, Consumer<InternalParent> verify)
+            throws IOException {
+
+        ParentAggregationBuilder aggregationBuilder = new ParentAggregationBuilder("_name", CHILD_TYPE);
+        aggregationBuilder.subAggregation(new MinAggregationBuilder("in_parent").field("number"));
+
+        MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
+        fieldType.setName("number");
+        InternalParent result = search(indexSearcher, query, aggregationBuilder, fieldType);
+        verify.accept(result);
+    }
+
+    private void testCaseTerms(Query query, IndexSearcher indexSearcher, Consumer<InternalParent> verify)
+            throws IOException {
+
+        ParentAggregationBuilder aggregationBuilder = new ParentAggregationBuilder("_name", CHILD_TYPE);
+        aggregationBuilder.subAggregation(new TermsAggregationBuilder("value_terms", ValueType.LONG).field("number"));
+
+        MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
+        fieldType.setName("number");
+        InternalParent result = search(indexSearcher, query, aggregationBuilder, fieldType);
+        verify.accept(result);
+    }
+
+    // run a terms aggregation on the number in child-documents, then a parent aggregation and then terms on the parent-number
+    private void testCaseTermsParentTerms(Query query, IndexSearcher indexSearcher, Consumer<LongTerms> verify)
+            throws IOException {
+        AggregationBuilder aggregationBuilder =
+            new TermsAggregationBuilder("subvalue_terms", ValueType.LONG).field("subNumber").
+                subAggregation(new ParentAggregationBuilder("to_parent", CHILD_TYPE).
+                    subAggregation(new TermsAggregationBuilder("value_terms", ValueType.LONG).field("number")));
+
+        MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
+        fieldType.setName("number");
+        MappedFieldType subFieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
+        subFieldType.setName("subNumber");
+        LongTerms result = search(indexSearcher, query, aggregationBuilder, fieldType, subFieldType);
+        verify.accept(result);
+    }
+}

+ 65 - 0
modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/InternalParentTests.java

@@ -0,0 +1,65 @@
+/*
+ * 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.join.aggregations;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.elasticsearch.common.ParseField;
+import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry.Entry;
+import org.elasticsearch.search.aggregations.Aggregation;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.InternalSingleBucketAggregationTestCase;
+import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+public class InternalParentTests extends InternalSingleBucketAggregationTestCase<InternalParent> {
+
+    @Override
+    protected List<Entry> getNamedXContents() {
+        List<Entry> extendedNamedXContents = new ArrayList<>(super.getNamedXContents());
+        extendedNamedXContents.add(new Entry(Aggregation.class, new ParseField(ParentAggregationBuilder.NAME),
+                (p, c) -> ParsedParent.fromXContent(p, (String) c)));
+        return extendedNamedXContents ;
+    }
+
+    @Override
+    protected InternalParent createTestInstance(String name, long docCount, InternalAggregations aggregations,
+            List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
+        return new InternalParent(name, docCount, aggregations, pipelineAggregators, metaData);
+    }
+
+    @Override
+    protected void extraAssertReduced(InternalParent reduced, List<InternalParent> inputs) {
+        // Nothing extra to assert
+    }
+
+    @Override
+    protected Reader<InternalParent> instanceReader() {
+        return InternalParent::new;
+    }
+
+    @Override
+    protected Class<? extends ParsedSingleBucketAggregation> implementationClass() {
+        return ParsedParent.class;
+    }
+}

+ 238 - 0
modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentIT.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.join.aggregations;
+
+import org.elasticsearch.action.search.SearchRequestBuilder;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.search.aggregations.Aggregation;
+import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
+import org.elasticsearch.search.aggregations.bucket.terms.Terms;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
+import static org.elasticsearch.join.aggregations.JoinAggregationBuilders.parent;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.topHits;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
+import static org.hamcrest.Matchers.equalTo;
+
+public class ParentIT extends AbstractParentChildTestCase {
+
+    public void testSimpleParentAgg() throws Exception {
+        final SearchRequestBuilder searchRequest = client().prepareSearch("test")
+            .setSize(10000)
+            .setQuery(matchQuery("randomized", true))
+            .addAggregation(
+                parent("to_article", "comment")
+                    .subAggregation(
+                        terms("category").field("category").size(10000)));
+        SearchResponse searchResponse = searchRequest.get();
+        assertSearchResponse(searchResponse);
+
+        long articlesWithComment = articleToControl.values().stream().filter(
+            parentControl -> !parentControl.commentIds.isEmpty()
+        ).count();
+
+        Parent parentAgg = searchResponse.getAggregations().get("to_article");
+        assertThat("Request: " + searchRequest + "\nResponse: " + searchResponse + "\n",
+            parentAgg.getDocCount(), equalTo(articlesWithComment));
+        Terms categoryTerms = parentAgg.getAggregations().get("category");
+        long categoriesWithComments = categoryToControl.values().stream().filter(
+            control -> !control.commentIds.isEmpty()).count();
+        assertThat("Buckets: " + categoryTerms.getBuckets().stream().map(
+            (Function<Terms.Bucket, String>) MultiBucketsAggregation.Bucket::getKeyAsString).collect(Collectors.toList()) +
+                "\nCategories: " + categoryToControl.keySet(),
+            (long)categoryTerms.getBuckets().size(), equalTo(categoriesWithComments));
+        for (Map.Entry<String, Control> entry : categoryToControl.entrySet()) {
+            // no children for this category -> no entry in the child to parent-aggregation
+            if(entry.getValue().commentIds.isEmpty()) {
+                assertNull(categoryTerms.getBucketByKey(entry.getKey()));
+                continue;
+            }
+
+            final Terms.Bucket categoryBucket = categoryTerms.getBucketByKey(entry.getKey());
+            assertNotNull("Failed for category " + entry.getKey(),
+                categoryBucket);
+            assertThat("Failed for category " + entry.getKey(),
+                categoryBucket.getKeyAsString(), equalTo(entry.getKey()));
+
+            // count all articles in this category which have at least one comment
+            long articlesForCategory = articleToControl.values().stream().
+                // only articles with this category
+                filter(parentControl -> parentControl.category.equals(entry.getKey())).
+                // only articles which have comments
+                filter(parentControl -> !parentControl.commentIds.isEmpty()).
+                count();
+            assertThat("Failed for category " + entry.getKey(),
+                categoryBucket.getDocCount(), equalTo(articlesForCategory));
+        }
+    }
+
+    public void testParentAggs() throws Exception {
+        final SearchRequestBuilder searchRequest = client().prepareSearch("test")
+            .setSize(10000)
+            .setQuery(matchQuery("randomized", true))
+            .addAggregation(
+                terms("to_commenter").field("commenter").size(10000).subAggregation(
+                    parent("to_article", "comment").subAggregation(
+                        terms("to_category").field("category").size(10000).subAggregation(
+                            topHits("top_category")
+                        ))
+                )
+            );
+        SearchResponse searchResponse = searchRequest.get();
+        assertSearchResponse(searchResponse);
+
+        final Set<String> commenters = getCommenters();
+        final Map<String, Set<String>> commenterToComments = getCommenterToComments();
+
+        Terms categoryTerms = searchResponse.getAggregations().get("to_commenter");
+        assertThat("Request: " + searchRequest + "\nResponse: " + searchResponse + "\n",
+            categoryTerms.getBuckets().size(), equalTo(commenters.size()));
+        for (Terms.Bucket commenterBucket : categoryTerms.getBuckets()) {
+            Set<String> comments = commenterToComments.get(commenterBucket.getKeyAsString());
+            assertNotNull(comments);
+            assertThat("Failed for commenter " + commenterBucket.getKeyAsString(),
+                commenterBucket.getDocCount(), equalTo((long)comments.size()));
+
+            Parent articleAgg = commenterBucket.getAggregations().get("to_article");
+            assertThat(articleAgg.getName(), equalTo("to_article"));
+            // find all articles for the comments for the current commenter
+            Set<String> articles = articleToControl.values().stream().flatMap(
+                (Function<ParentControl, Stream<String>>) parentControl -> parentControl.commentIds.stream().
+                    filter(comments::contains)
+            ).collect(Collectors.toSet());
+
+            assertThat(articleAgg.getDocCount(), equalTo((long)articles.size()));
+
+            Terms categoryAgg = articleAgg.getAggregations().get("to_category");
+            assertNotNull(categoryAgg);
+
+            List<String> categories = categoryToControl.entrySet().
+                stream().
+                filter(entry -> entry.getValue().commenterToCommentId.containsKey(commenterBucket.getKeyAsString())).
+                map(Map.Entry::getKey).
+                collect(Collectors.toList());
+
+            for (String category : categories) {
+                Terms.Bucket categoryBucket = categoryAgg.getBucketByKey(category);
+                assertNotNull(categoryBucket);
+
+                Aggregation topCategory = categoryBucket.getAggregations().get("top_category");
+                assertNotNull(topCategory);
+            }
+        }
+
+        for (String commenter : commenters) {
+            Terms.Bucket categoryBucket = categoryTerms.getBucketByKey(commenter);
+            assertThat(categoryBucket.getKeyAsString(), equalTo(commenter));
+            assertThat(categoryBucket.getDocCount(), equalTo((long) commenterToComments.get(commenter).size()));
+
+            Parent childrenBucket = categoryBucket.getAggregations().get("to_article");
+            assertThat(childrenBucket.getName(), equalTo("to_article"));
+        }
+    }
+
+    private Set<String> getCommenters() {
+        return categoryToControl.values().stream().flatMap(
+                (Function<Control, Stream<String>>) control -> control.commenterToCommentId.keySet().stream()).
+                collect(Collectors.toSet());
+    }
+
+    private Map<String, Set<String>> getCommenterToComments() {
+        final Map<String, Set<String>> commenterToComments = new HashMap<>();
+        for (Control control : categoryToControl.values()) {
+            for (Map.Entry<String, Set<String>> entry : control.commenterToCommentId.entrySet()) {
+                final Set<String> comments = commenterToComments.computeIfAbsent(entry.getKey(), s -> new HashSet<>());
+                comments.addAll(entry.getValue());
+            }
+        }
+        return commenterToComments;
+    }
+
+    public void testNonExistingParentType() throws Exception {
+        SearchResponse searchResponse = client().prepareSearch("test")
+            .addAggregation(
+                parent("non-existing", "xyz")
+            ).get();
+        assertSearchResponse(searchResponse);
+
+        Parent parent = searchResponse.getAggregations().get("non-existing");
+        assertThat(parent.getName(), equalTo("non-existing"));
+        assertThat(parent.getDocCount(), equalTo(0L));
+    }
+
+    public void testTermsParentAggTerms() throws Exception {
+        final SearchRequestBuilder searchRequest = client().prepareSearch("test")
+            .setSize(10000)
+            .setQuery(matchQuery("randomized", true))
+            .addAggregation(
+                terms("to_commenter").field("commenter").size(10000).subAggregation(
+                    parent("to_article", "comment").subAggregation(
+                        terms("to_category").field("category").size(10000))));
+        SearchResponse searchResponse = searchRequest.get();
+        assertSearchResponse(searchResponse);
+
+        final Set<String> commenters = getCommenters();
+        final Map<String, Set<String>> commenterToComments = getCommenterToComments();
+
+        Terms commentersAgg = searchResponse.getAggregations().get("to_commenter");
+        assertThat("Request: " + searchRequest + "\nResponse: " + searchResponse + "\n",
+            commentersAgg.getBuckets().size(), equalTo(commenters.size()));
+        for (Terms.Bucket commenterBucket : commentersAgg.getBuckets()) {
+            Set<String> comments = commenterToComments.get(commenterBucket.getKeyAsString());
+            assertNotNull(comments);
+            assertThat("Failed for commenter " + commenterBucket.getKeyAsString(),
+                commenterBucket.getDocCount(), equalTo((long)comments.size()));
+
+            Parent articleAgg = commenterBucket.getAggregations().get("to_article");
+            assertThat(articleAgg.getName(), equalTo("to_article"));
+            // find all articles for the comments for the current commenter
+            Set<String> articles = articleToControl.values().stream().flatMap(
+                (Function<ParentControl, Stream<String>>) parentControl -> parentControl.commentIds.stream().
+                    filter(comments::contains)
+            ).collect(Collectors.toSet());
+
+            assertThat(articleAgg.getDocCount(), equalTo((long)articles.size()));
+
+            Terms categoryAgg = articleAgg.getAggregations().get("to_category");
+            assertNotNull(categoryAgg);
+
+            List<String> categories = categoryToControl.entrySet().
+                stream().
+                    filter(entry -> entry.getValue().commenterToCommentId.containsKey(commenterBucket.getKeyAsString())).
+                    map(Map.Entry::getKey).
+                collect(Collectors.toList());
+
+            for (String category : categories) {
+                Terms.Bucket categoryBucket = categoryAgg.getBucketByKey(category);
+                assertNotNull(categoryBucket);
+            }
+        }
+    }
+}

+ 42 - 0
modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentTests.java

@@ -0,0 +1,42 @@
+/*
+ * 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.join.aggregations;
+
+import java.util.Collection;
+import java.util.Collections;
+
+import org.elasticsearch.join.ParentJoinPlugin;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
+
+public class ParentTests extends BaseAggregationTestCase<ParentAggregationBuilder> {
+
+    @Override
+    protected Collection<Class<? extends Plugin>> getPlugins() {
+        return Collections.singleton(ParentJoinPlugin.class);
+    }
+
+    @Override
+    protected ParentAggregationBuilder createTestAggregatorBuilder() {
+        String name = randomAlphaOfLengthBetween(3, 20);
+        String parentType = randomAlphaOfLengthBetween(5, 40);
+        return new ParentAggregationBuilder(name, parentType);
+    }
+}