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

Closes #8760
This commit is contained in:
Martijn van Groningen 2014-12-07 22:16:37 +01:00
parent 4cb23a0520
commit ca68136628
7 changed files with 167 additions and 119 deletions

View File

@ -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 + "]");
}
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");
}
String parentType = parentFieldMapper.type();
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 + "]");
}
Filter parentFilter = context.filterCache().cache(parentDocMapper.typeFilter(), null, context.queryParserService().autoFilterCachePolicy());
Filter childFilter = context.filterCache().cache(childDocMapper.typeFilter(), null, context.queryParserService().autoFilterCachePolicy());
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);
ValuesSourceConfig<ValuesSource.Bytes.WithOrdinals.ParentChild> config = new ValuesSourceConfig<>(ValuesSource.Bytes.WithOrdinals.ParentChild.class);
config.fieldContext(new FieldContext(parentFieldMapper.names().indexName(), parentChildIndexFieldData, parentFieldMapper));
} else {
config.unmapped(true);
}
} else {
config.unmapped(true);
}
return new ParentToChildrenAggregator.Factory(aggregationName, config, parentType, parentFilter, childFilter);
}
}

View File

@ -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

View File

@ -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());
}
}
}
}

View File

@ -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());
}
}
}
}

View File

@ -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;

View File

@ -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")
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();
fail("expected execution to fail - an attempt to nested facet on non-nested field/path");
} 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"));
}
}
}

View File

@ -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));
}
}