Browse Source

Made the `nested`, `reverse_nested` and `children` aggs ignore unmapped nested fields or unmapped child / parent types.

Closes #8760
Martijn van Groningen 11 years ago
parent
commit
ca68136628

+ 21 - 18
src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenParser.java

@@ -67,27 +67,30 @@ public class ChildrenParser implements Aggregator.Parser {
             throw new SearchParseException(context, "Missing [child_type] field for children aggregation [" + aggregationName + "]");
         }
 
+        ValuesSourceConfig<ValuesSource.Bytes.WithOrdinals.ParentChild> config = new ValuesSourceConfig<>(ValuesSource.Bytes.WithOrdinals.ParentChild.class);
         DocumentMapper childDocMapper = context.mapperService().documentMapper(childType);
-        if (childDocMapper == null) {
-            throw new SearchParseException(context, "[children] No mapping for for type [" + childType + "]");
-        }
-        ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
-        if (!parentFieldMapper.active()) {
-            throw new SearchParseException(context, "[children] _parent field not configured");
-        }
 
-        String parentType = parentFieldMapper.type();
-        DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType);
-        if (parentDocMapper == null) {
-            throw new SearchParseException(context, "[children]  Type [" + childType + "] points to a non existent parent type [" + parentType + "]");
+        String parentType = null;
+        Filter parentFilter = null;
+        Filter childFilter = null;
+        if (childDocMapper != null) {
+            ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
+            if (!parentFieldMapper.active()) {
+                throw new SearchParseException(context, "[children] _parent field not configured");
+            }
+            parentType = parentFieldMapper.type();
+            DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType);
+            if (parentDocMapper != null) {
+                parentFilter = context.filterCache().cache(parentDocMapper.typeFilter(), null, context.queryParserService().autoFilterCachePolicy());
+                childFilter = context.filterCache().cache(childDocMapper.typeFilter(), null, context.queryParserService().autoFilterCachePolicy());
+                ParentChildIndexFieldData parentChildIndexFieldData = context.fieldData().getForField(parentFieldMapper);
+                config.fieldContext(new FieldContext(parentFieldMapper.names().indexName(), parentChildIndexFieldData, parentFieldMapper));
+            } else {
+                config.unmapped(true);
+            }
+        } else {
+            config.unmapped(true);
         }
-
-        Filter parentFilter = context.filterCache().cache(parentDocMapper.typeFilter(), null, context.queryParserService().autoFilterCachePolicy());
-        Filter childFilter = context.filterCache().cache(childDocMapper.typeFilter(), null, context.queryParserService().autoFilterCachePolicy());
-
-        ParentChildIndexFieldData parentChildIndexFieldData = context.fieldData().getForField(parentFieldMapper);
-        ValuesSourceConfig<ValuesSource.Bytes.WithOrdinals.ParentChild> config = new ValuesSourceConfig<>(ValuesSource.Bytes.WithOrdinals.ParentChild.class);
-        config.fieldContext(new FieldContext(parentFieldMapper.names().indexName(), parentChildIndexFieldData, parentFieldMapper));
         return new ParentToChildrenAggregator.Factory(aggregationName, config, parentType, parentFilter, childFilter);
     }
 }

+ 9 - 2
src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java

@@ -24,7 +24,6 @@ import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.util.Bits;
-import org.elasticsearch.ElasticsearchIllegalStateException;
 import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.lucene.ReaderContextAware;
@@ -35,6 +34,7 @@ import org.elasticsearch.index.search.child.ConstantScorer;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.NonCollectingAggregator;
 import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
@@ -200,7 +200,14 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
 
         @Override
         protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
-            throw new ElasticsearchIllegalStateException("[children] aggregation doesn't support unmapped");
+            return new NonCollectingAggregator(name, aggregationContext, parent, metaData) {
+
+                @Override
+                public InternalAggregation buildEmptyAggregation() {
+                    return new InternalChildren(name, 0, buildEmptySubAggregations(), getMetaData());
+                }
+
+            };
         }
 
         @Override

+ 28 - 25
src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java

@@ -31,11 +31,7 @@ import org.elasticsearch.common.lucene.docset.DocIdSets;
 import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.mapper.object.ObjectMapper;
 import org.elasticsearch.index.search.nested.NonNestedDocsFilter;
-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.*;
 import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
 import org.elasticsearch.search.internal.SearchContext;
@@ -48,7 +44,6 @@ import java.util.Map;
  */
 public class NestedAggregator extends SingleBucketAggregator implements ReaderContextAware {
 
-    private final String nestedPath;
     private final Aggregator parentAggregator;
     private BitDocIdSetFilter parentFilter;
     private final Filter childFilter;
@@ -56,22 +51,9 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
     private DocIdSetIterator childDocs;
     private BitSet parentDocs;
 
-    public NestedAggregator(String name, AggregatorFactories factories, String nestedPath, AggregationContext aggregationContext, Aggregator parentAggregator, Map<String, Object> metaData, FilterCachingPolicy filterCachingPolicy) {
+    public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map<String, Object> metaData, FilterCachingPolicy filterCachingPolicy) {
         super(name, factories, aggregationContext, parentAggregator, metaData);
-        this.nestedPath = nestedPath;
         this.parentAggregator = parentAggregator;
-        MapperService.SmartNameObjectMapper mapper = aggregationContext.searchContext().smartNameObjectMapper(nestedPath);
-        if (mapper == null) {
-            throw new AggregationExecutionException("[nested] nested path [" + nestedPath + "] not found");
-        }
-        ObjectMapper objectMapper = mapper.mapper();
-        if (objectMapper == null) {
-            throw new AggregationExecutionException("[nested] nested path [" + nestedPath + "] not found");
-        }
-        if (!objectMapper.nested().isNested()) {
-            throw new AggregationExecutionException("[nested] nested path [" + nestedPath + "] is not nested");
-        }
-
         childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy);
         // The childDocs need to be consumed in docId order, this ensures that:
         aggregationContext.ensureScoreDocsInOrder();
@@ -144,10 +126,6 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
         return new InternalNested(name, 0, buildEmptySubAggregations(), getMetaData());
     }
 
-    public String getNestedPath() {
-        return nestedPath;
-    }
-
     private static Filter findClosestNestedPath(Aggregator parent) {
         for (; parent != null; parent = parent.parent()) {
             if (parent instanceof NestedAggregator) {
@@ -172,7 +150,32 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
 
         @Override
         public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
-            return new NestedAggregator(name, factories, path, context, parent, metaData, filterCachingPolicy);
+            MapperService.SmartNameObjectMapper mapper = context.searchContext().smartNameObjectMapper(path);
+            if (mapper == null) {
+                return new Unmapped(name, context, parent, metaData);
+            }
+            ObjectMapper objectMapper = mapper.mapper();
+            if (objectMapper == null) {
+                return new Unmapped(name, context, parent, metaData);
+            }
+            if (!objectMapper.nested().isNested()) {
+                throw new AggregationExecutionException("[nested] nested path [" + path + "] is not nested");
+            }
+            return new NestedAggregator(name, factories, objectMapper, context, parent, metaData, filterCachingPolicy);
+        }
+
+        private final static class Unmapped extends NonCollectingAggregator {
+
+            public Unmapped(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
+                super(name, context, parent, metaData);
+            }
+
+            @Override
+            public InternalAggregation buildEmptyAggregation() {
+                return new InternalNested(name, 0, buildEmptySubAggregations(), getMetaData());
+            }
         }
     }
+
+
 }

+ 37 - 20
src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java

@@ -49,28 +49,11 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
     // TODO: Add LongIntPagedHashMap?
     private final LongIntOpenHashMap bucketOrdToLastCollectedParentDoc;
 
-    public ReverseNestedAggregator(String name, AggregatorFactories factories, String nestedPath, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
+    public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
         super(name, factories, aggregationContext, parent, metaData);
-
-        // Early validation
-        NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent);
-        if (closestNestedAggregator == null) {
-            throw new SearchParseException(context.searchContext(), "Reverse nested aggregation [" + name + "] can only be used inside a [nested] aggregation");
-        }
-        if (nestedPath == null) {
+        if (objectMapper == null) {
             parentFilter = SearchContext.current().bitsetFilterCache().getBitDocIdSetFilter(NonNestedDocsFilter.INSTANCE);
         } else {
-            MapperService.SmartNameObjectMapper mapper = SearchContext.current().smartNameObjectMapper(nestedPath);
-            if (mapper == null) {
-                throw new AggregationExecutionException("[reverse_nested] nested path [" + nestedPath + "] not found");
-            }
-            ObjectMapper objectMapper = mapper.mapper();
-            if (objectMapper == null) {
-                throw new AggregationExecutionException("[reverse_nested] nested path [" + nestedPath + "] not found");
-            }
-            if (!objectMapper.nested().isNested()) {
-                throw new AggregationExecutionException("[reverse_nested] nested path [" + nestedPath + "] is not nested");
-            }
             parentFilter = SearchContext.current().bitsetFilterCache().getBitDocIdSetFilter(objectMapper.nestedTypeFilter());
         }
         bucketOrdToLastCollectedParentDoc = new LongIntOpenHashMap(32);
@@ -158,7 +141,41 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
 
         @Override
         public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
-            return new ReverseNestedAggregator(name, factories, path, context, parent, metaData);
+            // Early validation
+            NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent);
+            if (closestNestedAggregator == null) {
+                throw new SearchParseException(context.searchContext(), "Reverse nested aggregation [" + name + "] can only be used inside a [nested] aggregation");
+            }
+
+            final ObjectMapper objectMapper;
+            if (path != null) {
+                MapperService.SmartNameObjectMapper mapper = SearchContext.current().smartNameObjectMapper(path);
+                if (mapper == null) {
+                    return new Unmapped(name, context, parent, metaData);
+                }
+                objectMapper = mapper.mapper();
+                if (objectMapper == null) {
+                    return new Unmapped(name, context, parent, metaData);
+                }
+                if (!objectMapper.nested().isNested()) {
+                    throw new AggregationExecutionException("[reverse_nested] nested path [" + path + "] is not nested");
+                }
+            } else {
+                objectMapper = null;
+            }
+            return new ReverseNestedAggregator(name, factories, objectMapper, context, parent, metaData);
+        }
+
+        private final static class Unmapped extends NonCollectingAggregator {
+
+            public Unmapped(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
+                super(name, context, parent, metaData);
+            }
+
+            @Override
+            public InternalAggregation buildEmptyAggregation() {
+                return new InternalReverseNested(name, 0, buildEmptySubAggregations(), getMetaData());
+            }
         }
     }
 }

+ 18 - 18
src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenTests.java

@@ -18,13 +18,6 @@
  */
 package org.elasticsearch.search.aggregations.bucket;
 
-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.action.search.SearchResponse;
 import org.elasticsearch.action.update.UpdateResponse;
@@ -37,18 +30,12 @@ import org.elasticsearch.search.sort.SortOrder;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.junit.Test;
 
+import java.util.*;
+
 import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.children;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.topHits;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
-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.is;
-import static org.hamcrest.Matchers.sameInstance;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.*;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
+import static org.hamcrest.Matchers.*;
 
 /**
  */
@@ -269,6 +256,19 @@ public class ChildrenTests extends ElasticsearchIntegrationTest {
         }
     }
 
+    @Test
+    public void testNonExistingChildType() throws Exception {
+        SearchResponse searchResponse = client().prepareSearch("test")
+                .addAggregation(
+                        children("non-existing").childType("xyz")
+                ).get();
+        assertSearchResponse(searchResponse);
+
+        Children children = searchResponse.getAggregations().get("non-existing");
+        assertThat(children.getName(), equalTo("non-existing"));
+        assertThat(children.getDocCount(), equalTo(0l));
+    }
+
     private static final class Control {
 
         final String category;

+ 29 - 24
src/test/java/org/elasticsearch/search/aggregations/bucket/NestedTests.java

@@ -18,11 +18,8 @@
  */
 package org.elasticsearch.search.aggregations.bucket;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.action.search.SearchPhaseExecutionException;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
@@ -39,19 +36,15 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.hamcrest.Matchers;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.nested;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.*;
 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.is;
-import static org.hamcrest.Matchers.sameInstance;
+import static org.hamcrest.Matchers.*;
 import static org.hamcrest.core.IsNull.notNullValue;
 
 /**
@@ -68,7 +61,7 @@ public class NestedTests extends ElasticsearchIntegrationTest {
     public void setupSuiteScopeCluster() throws Exception {
 
         assertAcked(prepareCreate("idx")
-                .addMapping("type", "nested", "type=nested"));
+                .addMapping("type", "nested", "type=nested", "incorrect", "type=object"));
 
         List<IndexRequestBuilder> builders = new ArrayList<>();
 
@@ -194,17 +187,16 @@ public class NestedTests extends ElasticsearchIntegrationTest {
     }
 
     @Test
-    public void onNonNestedField() throws Exception {
-        try {
-            client().prepareSearch("idx")
-                    .addAggregation(nested("nested").path("value")
-                            .subAggregation(stats("nested_value_stats").field("nested.value")))
-                    .execute().actionGet();
-
-            fail("expected execution to fail - an attempt to nested facet on non-nested field/path");
+    public void nonExistingNestedField() throws Exception {
+        SearchResponse searchResponse = client().prepareSearch("idx")
+                .addAggregation(nested("nested").path("value")
+                        .subAggregation(stats("nested_value_stats").field("nested.value")))
+                .execute().actionGet();
 
-        } catch (ElasticsearchException ese) {
-        }
+        Nested nested = searchResponse.getAggregations().get("nested");
+        assertThat(nested, Matchers.notNullValue());
+        assertThat(nested.getName(), equalTo("nested"));
+        assertThat(nested.getDocCount(), is(0l));
     }
 
     @Test
@@ -344,4 +336,17 @@ public class NestedTests extends ElasticsearchIntegrationTest {
         assertThat(nested.getName(), equalTo("nested"));
         assertThat(nested.getDocCount(), is(0l));
     }
+
+    @Test
+    public void nestedOnObjectField() throws Exception {
+        try {
+            client().prepareSearch("idx")
+                    .setQuery(matchAllQuery())
+                    .addAggregation(nested("object_field").path("incorrect"))
+                    .execute().actionGet();
+            fail();
+        } catch (SearchPhaseExecutionException e) {
+            assertThat(e.getMessage(), containsString("[nested] nested path [incorrect] is not nested"));
+        }
+    }
 }

+ 23 - 10
src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedTests.java

@@ -18,10 +18,6 @@
  */
 package org.elasticsearch.search.aggregations.bucket;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 import org.elasticsearch.action.search.SearchPhaseExecutionException;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -30,17 +26,19 @@ import org.elasticsearch.search.aggregations.bucket.nested.Nested;
 import org.elasticsearch.search.aggregations.bucket.nested.ReverseNested;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
+import org.hamcrest.Matchers;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.nested;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.reverseNested;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
+import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.*;
 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.is;
-import static org.hamcrest.Matchers.sameInstance;
+import static org.hamcrest.Matchers.*;
 import static org.hamcrest.core.IsNull.notNullValue;
 
 /**
@@ -453,4 +451,19 @@ public class ReverseNestedTests extends ElasticsearchIntegrationTest {
                                 )
                 ).get();
     }
+
+    @Test
+    public void nonExistingNestedField() throws Exception {
+        SearchResponse searchResponse = client().prepareSearch("idx")
+                .setQuery(matchAllQuery())
+                .addAggregation(nested("nested2").path("nested1.nested2").subAggregation(reverseNested("incorrect").path("nested3")))
+                .execute().actionGet();
+
+        Nested nested = searchResponse.getAggregations().get("nested2");
+        assertThat(nested, Matchers.notNullValue());
+        assertThat(nested.getName(), equalTo("nested2"));
+
+        ReverseNested reverseNested = nested.getAggregations().get("incorrect");
+        assertThat(reverseNested.getDocCount(), is(0l));
+    }
 }