diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java index 617aa28a0ba92..8f092383a5b38 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java @@ -38,7 +38,6 @@ import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder; import org.elasticsearch.search.sort.FieldSortBuilder; -import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; @@ -46,7 +45,6 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.function.Function; @@ -54,7 +52,6 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.support.XContentMapValues.extractValue; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; -import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.index.query.QueryBuilders.nestedQuery; @@ -62,9 +59,9 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSuccessful; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHit; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -616,4 +613,25 @@ public void testInnerHitsWithIgnoreUnmapped() throws Exception { assertSearchHits(response, "1", "3"); } + public void testDontExplode() throws Exception { + assertAcked(prepareCreate("index2").addMapping("type", "nested", "type=nested")); + client().prepareIndex("index2", "type", "1").setSource(jsonBuilder().startObject() + .startArray("nested") + .startObject() + .field("field", "value1") + .endObject() + .endArray() + .endObject()) + .setRefreshPolicy(IMMEDIATE) + .get(); + + QueryBuilder query = nestedQuery("nested", matchQuery("nested.field", "value1"), ScoreMode.Avg) + .innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1)); + SearchResponse response = client().prepareSearch("index2") + .setQuery(query) + .get(); + assertNoFailures(response); + assertHitCount(response, 1); + } + } diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java index 6485b8f6491ff..0e0e1f1748cb6 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java @@ -316,6 +316,14 @@ protected Query doToQuery(QueryShardContext context) throws IOException { private Query joinFieldDoToQuery(QueryShardContext context) throws IOException { ParentJoinFieldMapper joinFieldMapper = ParentJoinFieldMapper.getMapper(context.getMapperService()); + if (joinFieldMapper == null) { + if (ignoreUnmapped) { + return new MatchNoDocsQuery(); + } else { + throw new QueryShardException(context, "[" + NAME + "] no join field has been configured"); + } + } + ParentIdFieldMapper parentIdFieldMapper = joinFieldMapper.getParentIdFieldMapper(type, false); if (parentIdFieldMapper != null) { Query parentFilter = parentIdFieldMapper.getParentFilter(); @@ -329,7 +337,8 @@ private Query joinFieldDoToQuery(QueryShardContext context) throws IOException { if (ignoreUnmapped) { return new MatchNoDocsQuery(); } else { - throw new QueryShardException(context, "[" + NAME + "] join field has no parent type configured"); + throw new QueryShardException(context, "[" + NAME + "] join field [" + joinFieldMapper.name() + + "] doesn't hold [" + type + "] as a child"); } } } @@ -513,7 +522,7 @@ protected void extractInnerHitBuilders(Map inner InnerHitContextBuilder.extractInnerHits(query, children); String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : type; InnerHitContextBuilder innerHitContextBuilder = - new HasParentQueryBuilder.ParentChildInnerHitContextBuilder(type, query, innerHitBuilder, children); + new ParentChildInnerHitContextBuilder(type, true, query, innerHitBuilder, children); innerHits.put(name, innerHitContextBuilder); } } diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java index 3d23fac01bcce..e7b41dddf34e2 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java @@ -18,34 +18,22 @@ */ package org.elasticsearch.join.query; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.DocValuesTermsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopDocsCollector; -import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopScoreDocCollector; -import org.apache.lucene.search.TotalHitCountCollector; -import org.apache.lucene.search.Weight; import org.apache.lucene.search.join.ScoreMode; import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParentFieldMapper; import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.InnerHitBuilder; @@ -57,10 +45,6 @@ import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.join.mapper.ParentIdFieldMapper; import org.elasticsearch.join.mapper.ParentJoinFieldMapper; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHitField; -import org.elasticsearch.search.fetch.subphase.InnerHitsContext; -import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.HashMap; @@ -69,8 +53,6 @@ import java.util.Objects; import java.util.Set; -import static org.elasticsearch.search.fetch.subphase.InnerHitsContext.intersect; - /** * Builder for the 'has_parent' query. */ @@ -198,6 +180,14 @@ protected Query doToQuery(QueryShardContext context) throws IOException { private Query joinFieldDoToQuery(QueryShardContext context) throws IOException { ParentJoinFieldMapper joinFieldMapper = ParentJoinFieldMapper.getMapper(context.getMapperService()); + if (joinFieldMapper == null) { + if (ignoreUnmapped) { + return new MatchNoDocsQuery(); + } else { + throw new QueryShardException(context, "[" + NAME + "] no join field has been configured"); + } + } + ParentIdFieldMapper parentIdFieldMapper = joinFieldMapper.getParentIdFieldMapper(type, true); if (parentIdFieldMapper != null) { Query parentFilter = parentIdFieldMapper.getParentFilter(); @@ -212,7 +202,8 @@ private Query joinFieldDoToQuery(QueryShardContext context) throws IOException { if (ignoreUnmapped) { return new MatchNoDocsQuery(); } else { - throw new QueryShardException(context, "[" + NAME + "] join field has no parent type configured"); + throw new QueryShardException(context, "[" + NAME + "] join field [" + joinFieldMapper.name() + + "] doesn't hold [" + type + "] as a parent"); } } } @@ -389,119 +380,9 @@ protected void extractInnerHitBuilders(Map inner InnerHitContextBuilder.extractInnerHits(query, children); String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : type; InnerHitContextBuilder innerHitContextBuilder = - new ParentChildInnerHitContextBuilder(type, query, innerHitBuilder, children); + new ParentChildInnerHitContextBuilder(type, false, query, innerHitBuilder, children); innerHits.put(name, innerHitContextBuilder); } } - static class ParentChildInnerHitContextBuilder extends InnerHitContextBuilder { - private final String typeName; - - ParentChildInnerHitContextBuilder(String typeName, QueryBuilder query, InnerHitBuilder innerHitBuilder, - Map children) { - super(query, innerHitBuilder, children); - this.typeName = typeName; - } - - @Override - public void build(SearchContext parentSearchContext, InnerHitsContext innerHitsContext) throws IOException { - QueryShardContext queryShardContext = parentSearchContext.getQueryShardContext(); - DocumentMapper documentMapper = queryShardContext.documentMapper(typeName); - if (documentMapper == null) { - if (innerHitBuilder.isIgnoreUnmapped() == false) { - throw new IllegalStateException("[" + query.getName() + "] no mapping found for type [" + typeName + "]"); - } else { - return; - } - } - String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : documentMapper.type(); - ParentChildInnerHitSubContext parentChildInnerHits = new ParentChildInnerHitSubContext( - name, parentSearchContext, queryShardContext.getMapperService(), documentMapper - ); - setupInnerHitsContext(queryShardContext, parentChildInnerHits); - innerHitsContext.addInnerHitDefinition(parentChildInnerHits); - } - } - - static final class ParentChildInnerHitSubContext extends InnerHitsContext.InnerHitSubContext { - private final MapperService mapperService; - private final DocumentMapper documentMapper; - - ParentChildInnerHitSubContext(String name, SearchContext context, MapperService mapperService, DocumentMapper documentMapper) { - super(name, context); - this.mapperService = mapperService; - this.documentMapper = documentMapper; - } - - @Override - public TopDocs[] topDocs(SearchHit[] hits) throws IOException { - Weight innerHitQueryWeight = createInnerHitQueryWeight(); - TopDocs[] result = new TopDocs[hits.length]; - for (int i = 0; i < hits.length; i++) { - SearchHit hit = hits[i]; - final Query hitQuery; - if (isParentHit(hit)) { - String field = ParentFieldMapper.joinField(hit.getType()); - hitQuery = new DocValuesTermsQuery(field, hit.getId()); - } else if (isChildHit(hit)) { - DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.getType()); - final String parentType = hitDocumentMapper.parentFieldMapper().type(); - SearchHitField parentField = hit.field(ParentFieldMapper.NAME); - if (parentField == null) { - throw new IllegalStateException("All children must have a _parent"); - } - Term uidTerm = context.mapperService().createUidTerm(parentType, parentField.getValue()); - if (uidTerm == null) { - hitQuery = new MatchNoDocsQuery("Missing type: " + parentType); - } else { - hitQuery = new TermQuery(uidTerm); - } - } else { - result[i] = Lucene.EMPTY_TOP_DOCS; - continue; - } - - BooleanQuery q = new BooleanQuery.Builder() - // Only include docs that have the current hit as parent - .add(hitQuery, BooleanClause.Occur.FILTER) - // Only include docs that have this inner hits type - .add(documentMapper.typeFilter(context.getQueryShardContext()), BooleanClause.Occur.FILTER) - .build(); - Weight weight = context.searcher().createNormalizedWeight(q, false); - if (size() == 0) { - TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector(); - for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) { - intersect(weight, innerHitQueryWeight, totalHitCountCollector, ctx); - } - result[i] = new TopDocs(totalHitCountCollector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0); - } else { - int topN = Math.min(from() + size(), context.searcher().getIndexReader().maxDoc()); - TopDocsCollector topDocsCollector; - if (sort() != null) { - topDocsCollector = TopFieldCollector.create(sort().sort, topN, true, trackScores(), trackScores()); - } else { - topDocsCollector = TopScoreDocCollector.create(topN); - } - try { - for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) { - intersect(weight, innerHitQueryWeight, topDocsCollector, ctx); - } - } finally { - clearReleasables(Lifetime.COLLECTION); - } - result[i] = topDocsCollector.topDocs(from(), size()); - } - } - return result; - } - - private boolean isParentHit(SearchHit hit) { - return hit.getType().equals(documentMapper.parentFieldMapper().type()); - } - - private boolean isChildHit(SearchHit hit) { - DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.getType()); - return documentMapper.type().equals(hitDocumentMapper.parentFieldMapper().type()); - } - } } diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/ParentChildInnerHitContextBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/ParentChildInnerHitContextBuilder.java new file mode 100644 index 0000000000000..61f050bff3fb8 --- /dev/null +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/ParentChildInnerHitContextBuilder.java @@ -0,0 +1,267 @@ +/* + * 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.query; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.DocValuesTermsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TopDocsCollector; +import org.apache.lucene.search.TopFieldCollector; +import org.apache.lucene.search.TopScoreDocCollector; +import org.apache.lucene.search.TotalHitCountCollector; +import org.apache.lucene.search.Weight; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.ParentFieldMapper; +import org.elasticsearch.index.query.InnerHitBuilder; +import org.elasticsearch.index.query.InnerHitContextBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.join.mapper.ParentIdFieldMapper; +import org.elasticsearch.join.mapper.ParentJoinFieldMapper; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHitField; +import org.elasticsearch.search.fetch.subphase.InnerHitsContext; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Map; + +import static org.elasticsearch.search.fetch.subphase.InnerHitsContext.intersect; + +class ParentChildInnerHitContextBuilder extends InnerHitContextBuilder { + private final String typeName; + private final boolean fetchChildInnerHits; + + ParentChildInnerHitContextBuilder(String typeName, boolean fetchChildInnerHits, QueryBuilder query, + InnerHitBuilder innerHitBuilder, Map children) { + super(query, innerHitBuilder, children); + this.typeName = typeName; + this.fetchChildInnerHits = fetchChildInnerHits; + } + + @Override + public void build(SearchContext parentSearchContext, InnerHitsContext innerHitsContext) throws IOException { + if (parentSearchContext.mapperService().getIndexSettings().isSingleType()) { + handleJoinFieldInnerHits(parentSearchContext, innerHitsContext); + } else { + handleParentFieldInnerHits(parentSearchContext, innerHitsContext); + } + } + + private void handleJoinFieldInnerHits(SearchContext context, InnerHitsContext innerHitsContext) throws IOException { + QueryShardContext queryShardContext = context.getQueryShardContext(); + ParentJoinFieldMapper joinFieldMapper = ParentJoinFieldMapper.getMapper(context.mapperService()); + if (joinFieldMapper != null) { + String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : typeName; + JoinFieldInnerHitSubContext joinFieldInnerHits = new JoinFieldInnerHitSubContext(name, context, typeName, + fetchChildInnerHits, joinFieldMapper); + setupInnerHitsContext(queryShardContext, joinFieldInnerHits); + innerHitsContext.addInnerHitDefinition(joinFieldInnerHits); + } else { + if (innerHitBuilder.isIgnoreUnmapped() == false) { + throw new IllegalStateException("no join field has been configured"); + } + } + } + + private void handleParentFieldInnerHits(SearchContext context, InnerHitsContext innerHitsContext) throws IOException { + QueryShardContext queryShardContext = context.getQueryShardContext(); + DocumentMapper documentMapper = queryShardContext.documentMapper(typeName); + if (documentMapper == null) { + if (innerHitBuilder.isIgnoreUnmapped() == false) { + throw new IllegalStateException("[" + query.getName() + "] no mapping found for type [" + typeName + "]"); + } else { + return; + } + } + String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : documentMapper.type(); + ParentChildInnerHitSubContext parentChildInnerHits = new ParentChildInnerHitSubContext( + name, context, queryShardContext.getMapperService(), documentMapper + ); + setupInnerHitsContext(queryShardContext, parentChildInnerHits); + innerHitsContext.addInnerHitDefinition(parentChildInnerHits); + } + + static final class JoinFieldInnerHitSubContext extends InnerHitsContext.InnerHitSubContext { + private final String typeName; + private final boolean fetchChildInnerHits; + private final ParentJoinFieldMapper joinFieldMapper; + + JoinFieldInnerHitSubContext(String name, SearchContext context, String typeName, boolean fetchChildInnerHits, + ParentJoinFieldMapper joinFieldMapper) { + super(name, context); + this.typeName = typeName; + this.fetchChildInnerHits = fetchChildInnerHits; + this.joinFieldMapper = joinFieldMapper; + } + + @Override + public TopDocs[] topDocs(SearchHit[] hits) throws IOException { + Weight innerHitQueryWeight = createInnerHitQueryWeight(); + TopDocs[] result = new TopDocs[hits.length]; + for (int i = 0; i < hits.length; i++) { + SearchHit hit = hits[i]; + SearchHitField joinField = hit.getFields().get(joinFieldMapper.name()); + if (joinField == null) { + result[i] = Lucene.EMPTY_TOP_DOCS; + continue; + } + + QueryShardContext qsc = context.getQueryShardContext(); + ParentIdFieldMapper parentIdFieldMapper = + joinFieldMapper.getParentIdFieldMapper(typeName, fetchChildInnerHits == false); + if (parentIdFieldMapper == null) { + result[i] = Lucene.EMPTY_TOP_DOCS; + continue; + } + + Query q; + if (fetchChildInnerHits) { + Query hitQuery = parentIdFieldMapper.fieldType().termQuery(hit.getId(), qsc); + q = new BooleanQuery.Builder() + // Only include child documents that have the current hit as parent: + .add(hitQuery, BooleanClause.Occur.FILTER) + // and only include child documents of a single relation: + .add(joinFieldMapper.fieldType().termQuery(typeName, qsc), BooleanClause.Occur.FILTER) + .build(); + } else { + SearchHitField parentIdField = hit.getFields().get(parentIdFieldMapper.name()); + q = context.mapperService().fullName(IdFieldMapper.NAME).termQuery(parentIdField.getValue(), qsc); + } + + Weight weight = context.searcher().createNormalizedWeight(q, false); + if (size() == 0) { + TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector(); + for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) { + intersect(weight, innerHitQueryWeight, totalHitCountCollector, ctx); + } + result[i] = new TopDocs(totalHitCountCollector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0); + } else { + int topN = Math.min(from() + size(), context.searcher().getIndexReader().maxDoc()); + TopDocsCollector topDocsCollector; + if (sort() != null) { + topDocsCollector = TopFieldCollector.create(sort().sort, topN, true, trackScores(), trackScores()); + } else { + topDocsCollector = TopScoreDocCollector.create(topN); + } + try { + for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) { + intersect(weight, innerHitQueryWeight, topDocsCollector, ctx); + } + } finally { + clearReleasables(Lifetime.COLLECTION); + } + result[i] = topDocsCollector.topDocs(from(), size()); + } + } + return result; + } + } + + static final class ParentChildInnerHitSubContext extends InnerHitsContext.InnerHitSubContext { + private final MapperService mapperService; + private final DocumentMapper documentMapper; + + ParentChildInnerHitSubContext(String name, SearchContext context, MapperService mapperService, DocumentMapper documentMapper) { + super(name, context); + this.mapperService = mapperService; + this.documentMapper = documentMapper; + } + + @Override + public TopDocs[] topDocs(SearchHit[] hits) throws IOException { + Weight innerHitQueryWeight = createInnerHitQueryWeight(); + TopDocs[] result = new TopDocs[hits.length]; + for (int i = 0; i < hits.length; i++) { + SearchHit hit = hits[i]; + final Query hitQuery; + if (isParentHit(hit)) { + String field = ParentFieldMapper.joinField(hit.getType()); + hitQuery = new DocValuesTermsQuery(field, hit.getId()); + } else if (isChildHit(hit)) { + DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.getType()); + final String parentType = hitDocumentMapper.parentFieldMapper().type(); + SearchHitField parentField = hit.field(ParentFieldMapper.NAME); + if (parentField == null) { + throw new IllegalStateException("All children must have a _parent"); + } + Term uidTerm = context.mapperService().createUidTerm(parentType, parentField.getValue()); + if (uidTerm == null) { + hitQuery = new MatchNoDocsQuery("Missing type: " + parentType); + } else { + hitQuery = new TermQuery(uidTerm); + } + } else { + result[i] = Lucene.EMPTY_TOP_DOCS; + continue; + } + + BooleanQuery q = new BooleanQuery.Builder() + // Only include docs that have the current hit as parent + .add(hitQuery, BooleanClause.Occur.FILTER) + // Only include docs that have this inner hits type + .add(documentMapper.typeFilter(context.getQueryShardContext()), BooleanClause.Occur.FILTER) + .build(); + Weight weight = context.searcher().createNormalizedWeight(q, false); + if (size() == 0) { + TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector(); + for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) { + intersect(weight, innerHitQueryWeight, totalHitCountCollector, ctx); + } + result[i] = new TopDocs(totalHitCountCollector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0); + } else { + int topN = Math.min(from() + size(), context.searcher().getIndexReader().maxDoc()); + TopDocsCollector topDocsCollector; + if (sort() != null) { + topDocsCollector = TopFieldCollector.create(sort().sort, topN, true, trackScores(), trackScores()); + } else { + topDocsCollector = TopScoreDocCollector.create(topN); + } + try { + for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) { + intersect(weight, innerHitQueryWeight, topDocsCollector, ctx); + } + } finally { + clearReleasables(Lifetime.COLLECTION); + } + result[i] = topDocsCollector.topDocs(from(), size()); + } + } + return result; + } + + private boolean isParentHit(SearchHit hit) { + return hit.getType().equals(documentMapper.parentFieldMapper().type()); + } + + private boolean isChildHit(SearchHit hit) { + DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.getType()); + return documentMapper.type().equals(hitDocumentMapper.parentFieldMapper().type()); + } + } +} diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenIT.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenIT.java index eb9952c150658..59467d49c82f5 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenIT.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenIT.java @@ -25,8 +25,7 @@ import org.elasticsearch.client.Requests; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.join.ParentJoinPlugin; -import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.join.query.ParentChildTestCase; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -34,14 +33,9 @@ import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.search.aggregations.metrics.tophits.TopHits; import org.elasticsearch.search.sort.SortOrder; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.ESIntegTestCase.Scope; import org.junit.Before; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -64,61 +58,9 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.sameInstance; -@ClusterScope(scope = Scope.SUITE) -public class ChildrenIT extends ESIntegTestCase { +public class ChildrenIT extends ParentChildTestCase { private static final Map categoryToControl = new HashMap<>(); - @Override - protected boolean ignoreExternalCluster() { - return true; - } - - @Override - protected Collection> nodePlugins() { - return Collections.singleton(ParentJoinPlugin.class); - } - - @Override - protected Collection> transportClientPlugins() { - return nodePlugins(); - } - - protected boolean legacy() { - return false; - } - - private IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, Object... fields) { - String name = type; - if (legacy() == false) { - type = "doc"; - } - - IndexRequestBuilder indexRequestBuilder = client().prepareIndex(index, type, id); - if (legacy()) { - if (parentId != null) { - indexRequestBuilder.setParent(parentId); - } - indexRequestBuilder.setSource(fields); - } else { - Map source = new HashMap<>(); - for (int i = 0; i < fields.length; i += 2) { - source.put((String) fields[i], fields[i + 1]); - } - Map joinField = new HashMap<>(); - if (parentId != null) { - joinField.put("name", name); - joinField.put("parent", parentId); - indexRequestBuilder.setRouting(parentId); - } else { - joinField.put("name", name); - } - source.put("join_field", joinField); - indexRequestBuilder.setSource(source); - } - indexRequestBuilder.setCreate(true); - return indexRequestBuilder; - } - @Before public void setupCluster() throws Exception { categoryToControl.clear(); diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/LegacyChildrenIT.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/LegacyChildrenIT.java index b64905d2aab75..ecf4d06f325e8 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/LegacyChildrenIT.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/LegacyChildrenIT.java @@ -18,24 +18,10 @@ */ package org.elasticsearch.join.aggregations; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.ESIntegTestCase.Scope; - -@ClusterScope(scope = Scope.SUITE) public class LegacyChildrenIT extends ChildrenIT { @Override protected boolean legacy() { return true; } - - @Override - public Settings indexSettings() { - Settings indexSettings = super.indexSettings(); - return Settings.builder() - .put(indexSettings) - .put("index.mapping.single_type", false) - .build(); - } } diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java index 8bd995e63cb28..e7a57328b6ea8 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java @@ -29,11 +29,7 @@ import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.IdsQueryBuilder; import org.elasticsearch.index.query.InnerHitBuilder; @@ -42,8 +38,6 @@ import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.functionscore.FunctionScoreQueryBuilder; -import org.elasticsearch.join.ParentJoinPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.aggregations.AggregationBuilders; @@ -55,15 +49,10 @@ import org.elasticsearch.search.fetch.subphase.highlight.HighlightField; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.ESIntegTestCase.Scope; import org.hamcrest.Matchers; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -97,77 +86,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; -@ClusterScope(scope = Scope.SUITE) -public class ChildQuerySearchIT extends ESIntegTestCase { - - @Override - protected boolean ignoreExternalCluster() { - return true; - } - - @Override - protected Collection> nodePlugins() { - return Collections.singleton(ParentJoinPlugin.class); - } - - @Override - protected Collection> transportClientPlugins() { - return nodePlugins(); - } - - @Override - public Settings indexSettings() { - return Settings.builder().put(super.indexSettings()) - // aggressive filter caching so that we can assert on the filter cache size - .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), true) - .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), true) - .build(); - } - - protected boolean legacy() { - return false; - } - - private IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, Object... fields) { - Map source = new HashMap<>(); - for (int i = 0; i < fields.length; i += 2) { - source.put((String) fields[i], fields[i + 1]); - } - return createIndexRequest(index, type, id, parentId, source); - } - - private IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, - XContentBuilder builder) throws IOException { - Map source = XContentHelper.convertToMap(JsonXContent.jsonXContent, builder.string(), false); - return createIndexRequest(index, type, id, parentId, source); - } - - private IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, Map source) { - String name = type; - if (legacy() == false) { - type = "doc"; - } - - IndexRequestBuilder indexRequestBuilder = client().prepareIndex(index, type, id); - if (legacy()) { - if (parentId != null) { - indexRequestBuilder.setParent(parentId); - } - indexRequestBuilder.setSource(source); - } else { - Map joinField = new HashMap<>(); - if (parentId != null) { - joinField.put("name", name); - joinField.put("parent", parentId); - indexRequestBuilder.setRouting(parentId); - } else { - joinField.put("name", name); - } - source.put("join_field", joinField); - indexRequestBuilder.setSource(source); - } - return indexRequestBuilder; - } +public class ChildQuerySearchIT extends ParentChildTestCase { public void testSelfReferentialIsForbidden() { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java index 9115bdb41d848..0eb890f52efa8 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.join.query; import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.ConstantScoreQuery; @@ -30,16 +31,13 @@ import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper; import org.apache.lucene.search.similarities.Similarity; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.TypeFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.query.IdsQueryBuilder; import org.elasticsearch.index.query.InnerHitBuilder; import org.elasticsearch.index.query.InnerHitContextBuilder; @@ -72,8 +70,10 @@ import static org.hamcrest.CoreMatchers.notNullValue; public class HasChildQueryBuilderTests extends AbstractQueryTestCase { - protected static final String PARENT_TYPE = "parent"; - protected static final String CHILD_TYPE = "child"; + + private static final String TYPE = "doc"; + private static final String PARENT_DOC = "parent"; + private static final String CHILD_DOC = "child"; private static String similarity; @@ -88,27 +88,17 @@ protected Collection> getPlugins() { protected Settings indexSettings() { return Settings.builder() .put(super.indexSettings()) - .put("index.mapping.single_type", false) + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build(); } @Override protected void initializeAdditionalMappings(MapperService mapperService) throws IOException { similarity = randomFrom("classic", "BM25"); - // TODO: use a single type when inner hits have been changed to work with join field, - // this test randomly generates queries with inner hits - mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE, + mapperService.merge(TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(TYPE, + "join_field", "type=join," + PARENT_DOC + "=" + CHILD_DOC, STRING_FIELD_NAME, "type=text", STRING_FIELD_NAME_2, "type=keyword", - INT_FIELD_NAME, "type=integer", - DOUBLE_FIELD_NAME, "type=double", - BOOLEAN_FIELD_NAME, "type=boolean", - DATE_FIELD_NAME, "type=date", - OBJECT_FIELD_NAME, "type=object" - ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); - mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE, - "_parent", "type=" + PARENT_TYPE, - STRING_FIELD_NAME, "type=text", "custom_string", "type=text,similarity=" + similarity, INT_FIELD_NAME, "type=integer", DOUBLE_FIELD_NAME, "type=double", @@ -132,7 +122,7 @@ protected HasChildQueryBuilder doCreateTestQueryBuilder() { innerQueryBuilder = new WrapperQueryBuilder(innerQueryBuilder.toString()); } - HasChildQueryBuilder hqb = new HasChildQueryBuilder(CHILD_TYPE, innerQueryBuilder, + HasChildQueryBuilder hqb = new HasChildQueryBuilder(CHILD_DOC, innerQueryBuilder, RandomPicks.randomFrom(random(), ScoreMode.values())); hqb.minMaxChildren(min, max); hqb.ignoreUnmapped(randomBoolean()); @@ -266,14 +256,14 @@ public void testFromJson() throws IOException { } public void testToQueryInnerQueryType() throws IOException { - String[] searchTypes = new String[]{PARENT_TYPE}; + String[] searchTypes = new String[]{TYPE}; QueryShardContext shardContext = createShardContext(); shardContext.setTypes(searchTypes); - HasChildQueryBuilder hasChildQueryBuilder = hasChildQuery(CHILD_TYPE, new IdsQueryBuilder().addIds("id"), ScoreMode.None); + HasChildQueryBuilder hasChildQueryBuilder = hasChildQuery(CHILD_DOC, new IdsQueryBuilder().addIds("id"), ScoreMode.None); Query query = hasChildQueryBuilder.toQuery(shardContext); //verify that the context types are still the same as the ones we previously set assertThat(shardContext.getTypes(), equalTo(searchTypes)); - assertLateParsingQuery(query, CHILD_TYPE, "id"); + assertLateParsingQuery(query, CHILD_DOC, "id"); } static void assertLateParsingQuery(Query query, String type, String id) throws IOException { @@ -295,14 +285,11 @@ static void assertLateParsingQuery(Query query, String type, String id) throws I assertThat(booleanTermsQuery.clauses().get(0).getOccur(), equalTo(BooleanClause.Occur.SHOULD)); assertThat(booleanTermsQuery.clauses().get(0).getQuery(), instanceOf(TermQuery.class)); TermQuery termQuery = (TermQuery) booleanTermsQuery.clauses().get(0).getQuery(); - assertThat(termQuery.getTerm().field(), equalTo(UidFieldMapper.NAME)); - //we want to make sure that the inner ids query gets executed against the child type rather - // than the main type we initially set to the context - BytesRef[] ids = Uid.createUidsForTypesAndIds(Collections.singletonList(type), Collections.singletonList(id)); - assertThat(termQuery.getTerm().bytes(), equalTo(ids[0])); + assertThat(termQuery.getTerm().field(), equalTo(IdFieldMapper.NAME)); + assertThat(termQuery.getTerm().bytes().utf8ToString(), equalTo(id)); //check the type filter assertThat(booleanQuery.clauses().get(1).getOccur(), equalTo(BooleanClause.Occur.FILTER)); - assertEquals(new TypeFieldMapper.TypesQuery(new BytesRef(type)), booleanQuery.clauses().get(1).getQuery()); + assertEquals(new TermQuery(new Term("join_field", type)), booleanQuery.clauses().get(1).getQuery()); } @Override @@ -319,7 +306,7 @@ public void testMustRewrite() throws IOException { public void testNonDefaultSimilarity() throws Exception { QueryShardContext shardContext = createShardContext(); HasChildQueryBuilder hasChildQueryBuilder = - hasChildQuery(CHILD_TYPE, new TermQueryBuilder("custom_string", "value"), ScoreMode.None); + hasChildQuery(CHILD_DOC, new TermQueryBuilder("custom_string", "value"), ScoreMode.None); HasChildQueryBuilder.LateParsingQuery query = (HasChildQueryBuilder.LateParsingQuery) hasChildQueryBuilder.toQuery(shardContext); Similarity expected = SimilarityService.BUILT_IN.get(similarity) .apply(similarity, Settings.EMPTY, Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()) @@ -337,7 +324,8 @@ public void testIgnoreUnmapped() throws IOException { final HasChildQueryBuilder failingQueryBuilder = new HasChildQueryBuilder("unmapped", new MatchAllQueryBuilder(), ScoreMode.None); failingQueryBuilder.ignoreUnmapped(false); QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext())); - assertThat(e.getMessage(), containsString("[" + HasChildQueryBuilder.NAME + "] no mapping found for type [unmapped]")); + assertThat(e.getMessage(), containsString("[" + HasChildQueryBuilder.NAME + + "] join field [join_field] doesn't hold [unmapped] as a child")); } public void testIgnoreUnmappedWithRewrite() throws IOException { diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java index 4c3194e31ab79..8be0b9374364b 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java @@ -24,6 +24,7 @@ import org.apache.lucene.search.join.ScoreMode; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; @@ -61,8 +62,9 @@ import static org.hamcrest.CoreMatchers.notNullValue; public class HasParentQueryBuilderTests extends AbstractQueryTestCase { - protected static final String PARENT_TYPE = "parent"; - protected static final String CHILD_TYPE = "child"; + private static final String TYPE = "doc"; + private static final String PARENT_DOC = "parent"; + private static final String CHILD_DOC = "child"; boolean requiresRewrite = false; @@ -75,15 +77,14 @@ protected Collection> getPlugins() { protected Settings indexSettings() { return Settings.builder() .put(super.indexSettings()) - .put("index.mapping.single_type", false) + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build(); } @Override protected void initializeAdditionalMappings(MapperService mapperService) throws IOException { - // TODO: use a single type when inner hits have been changed to work with join field, - // this test randomly generates queries with inner hits - mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE, + mapperService.merge(TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(TYPE, + "join_field", "type=join," + PARENT_DOC + "=" + CHILD_DOC, STRING_FIELD_NAME, "type=text", STRING_FIELD_NAME_2, "type=keyword", INT_FIELD_NAME, "type=integer", @@ -92,18 +93,6 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws DATE_FIELD_NAME, "type=date", OBJECT_FIELD_NAME, "type=object" ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); - mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE, - "_parent", "type=" + PARENT_TYPE, - STRING_FIELD_NAME, "type=text", - STRING_FIELD_NAME_2, "type=keyword", - INT_FIELD_NAME, "type=integer", - DOUBLE_FIELD_NAME, "type=double", - BOOLEAN_FIELD_NAME, "type=boolean", - DATE_FIELD_NAME, "type=date", - OBJECT_FIELD_NAME, "type=object" - ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); - mapperService.merge("just_a_type", new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef("just_a_type" - ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); } /** @@ -116,7 +105,7 @@ protected HasParentQueryBuilder doCreateTestQueryBuilder() { requiresRewrite = true; innerQueryBuilder = new WrapperQueryBuilder(innerQueryBuilder.toString()); } - HasParentQueryBuilder hqb = new HasParentQueryBuilder(PARENT_TYPE, innerQueryBuilder, randomBoolean()); + HasParentQueryBuilder hqb = new HasParentQueryBuilder(PARENT_DOC, innerQueryBuilder, randomBoolean()); hqb.ignoreUnmapped(randomBoolean()); if (randomBoolean()) { hqb.innerHit(new InnerHitBuilder() @@ -183,7 +172,7 @@ public void testIllegalValues() throws IOException { QueryShardContext context = createShardContext(); HasParentQueryBuilder qb = hasParentQuery("just_a_type", new MatchAllQueryBuilder(), false); QueryShardException qse = expectThrows(QueryShardException.class, () -> qb.doToQuery(context)); - assertThat(qse.getMessage(), equalTo("[has_parent] no child types found for type [just_a_type]")); + assertThat(qse.getMessage(), equalTo("[has_parent] join field [join_field] doesn't hold [just_a_type] as a parent")); } public void testDeprecatedXContent() throws IOException { @@ -201,15 +190,15 @@ public void testDeprecatedXContent() throws IOException { } public void testToQueryInnerQueryType() throws IOException { - String[] searchTypes = new String[]{CHILD_TYPE}; + String[] searchTypes = new String[]{TYPE}; QueryShardContext shardContext = createShardContext(); shardContext.setTypes(searchTypes); - HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(PARENT_TYPE, new IdsQueryBuilder().addIds("id"), + HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(PARENT_DOC, new IdsQueryBuilder().addIds("id"), false); Query query = hasParentQueryBuilder.toQuery(shardContext); //verify that the context types are still the same as the ones we previously set assertThat(shardContext.getTypes(), equalTo(searchTypes)); - HasChildQueryBuilderTests.assertLateParsingQuery(query, PARENT_TYPE, "id"); + HasChildQueryBuilderTests.assertLateParsingQuery(query, PARENT_DOC, "id"); } @Override @@ -258,7 +247,7 @@ public void testIgnoreUnmapped() throws IOException { failingQueryBuilder.ignoreUnmapped(false); QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext())); assertThat(e.getMessage(), - containsString("[" + HasParentQueryBuilder.NAME + "] query configured 'parent_type' [unmapped] is not a valid type")); + containsString("[has_parent] join field [join_field] doesn't hold [unmapped] as a parent")); } public void testIgnoreUnmappedWithRewrite() throws IOException { diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/InnerHitsIT.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/InnerHitsIT.java index 3459c9d35278f..f4e6e764acbcc 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/InnerHitsIT.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/InnerHitsIT.java @@ -23,7 +23,6 @@ import org.apache.lucene.util.ArrayUtil; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.InnerHitBuilder; import org.elasticsearch.index.query.QueryBuilder; @@ -39,9 +38,6 @@ import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.ESIntegTestCase.Scope; import java.util.ArrayList; import java.util.Arrays; @@ -52,7 +48,6 @@ import java.util.Map; import java.util.function.Function; -import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery; @@ -73,23 +68,13 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; -@ClusterScope(scope = Scope.SUITE) -public class InnerHitsIT extends ESIntegTestCase { - @Override - protected boolean ignoreExternalCluster() { - return true; - } +public class InnerHitsIT extends ParentChildTestCase { @Override protected Collection> nodePlugins() { return Arrays.asList(ParentJoinPlugin.class, CustomScriptPlugin.class); } - @Override - protected Collection> transportClientPlugins() { - return nodePlugins(); - } - public static class CustomScriptPlugin extends MockScriptPlugin { @Override protected Map, Object>> pluginScripts() { @@ -98,21 +83,27 @@ protected Map, Object>> pluginScripts() { } public void testSimpleParentChild() throws Exception { - assertAcked(prepareCreate("articles") - .setSettings("index.mapping.single_type", false) - .addMapping("article", "title", "type=text") - .addMapping("comment", "_parent", "type=article", "message", "type=text,fielddata=true") - ); + if (legacy()) { + assertAcked(prepareCreate("articles") + .addMapping("article", "title", "type=text") + .addMapping("comment", "_parent", "type=article", "message", "type=text,fielddata=true") + ); + } else { + assertAcked(prepareCreate("articles") + .addMapping("doc", "join_field", "type=join,article=comment", "title", "type=text", + "message", "type=text,fielddata=true") + ); + } List requests = new ArrayList<>(); - requests.add(client().prepareIndex("articles", "article", "1").setSource("title", "quick brown fox")); - requests.add(client().prepareIndex("articles", "comment", "1").setParent("1").setSource("message", "fox eat quick")); - requests.add(client().prepareIndex("articles", "comment", "2").setParent("1").setSource("message", "fox ate rabbit x y z")); - requests.add(client().prepareIndex("articles", "comment", "3").setParent("1").setSource("message", "rabbit got away")); - requests.add(client().prepareIndex("articles", "article", "2").setSource("title", "big gray elephant")); - requests.add(client().prepareIndex("articles", "comment", "4").setParent("2").setSource("message", "elephant captured")); - requests.add(client().prepareIndex("articles", "comment", "5").setParent("2").setSource("message", "mice squashed by elephant x")); - requests.add(client().prepareIndex("articles", "comment", "6").setParent("2").setSource("message", "elephant scared by mice x y")); + requests.add(createIndexRequest("articles", "article", "p1", null, "title", "quick brown fox")); + requests.add(createIndexRequest("articles", "comment", "c1", "p1", "message", "fox eat quick")); + requests.add(createIndexRequest("articles", "comment", "c2", "p1", "message", "fox ate rabbit x y z")); + requests.add(createIndexRequest("articles", "comment", "c3", "p1", "message", "rabbit got away")); + requests.add(createIndexRequest("articles", "article", "p2", null, "title", "big gray elephant")); + requests.add(createIndexRequest("articles", "comment", "c4", "p2", "message", "elephant captured")); + requests.add(createIndexRequest("articles", "comment", "c5", "p2", "message", "mice squashed by elephant x")); + requests.add(createIndexRequest("articles", "comment", "c6", "p2", "message", "elephant scared by mice x y")); indexRandom(true, requests); SearchResponse response = client().prepareSearch("articles") @@ -121,17 +112,17 @@ public void testSimpleParentChild() throws Exception { .get(); assertNoFailures(response); assertHitCount(response, 1); - assertSearchHit(response, 1, hasId("1")); + assertSearchHit(response, 1, hasId("p1")); assertThat(response.getHits().getAt(0).getShard(), notNullValue()); assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); assertThat(innerHits.getTotalHits(), equalTo(2L)); - assertThat(innerHits.getAt(0).getId(), equalTo("1")); - assertThat(innerHits.getAt(0).getType(), equalTo("comment")); - assertThat(innerHits.getAt(1).getId(), equalTo("2")); - assertThat(innerHits.getAt(1).getType(), equalTo("comment")); + assertThat(innerHits.getAt(0).getId(), equalTo("c1")); + assertThat(innerHits.getAt(0).getType(), equalTo(legacy() ? "comment" : "doc")); + assertThat(innerHits.getAt(1).getId(), equalTo("c2")); + assertThat(innerHits.getAt(1).getType(), equalTo(legacy() ? "comment" : "doc")); response = client().prepareSearch("articles") .setQuery(hasChildQuery("comment", matchQuery("message", "elephant"), ScoreMode.None) @@ -139,18 +130,18 @@ public void testSimpleParentChild() throws Exception { .get(); assertNoFailures(response); assertHitCount(response, 1); - assertSearchHit(response, 1, hasId("2")); + assertSearchHit(response, 1, hasId("p2")); assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); assertThat(innerHits.getTotalHits(), equalTo(3L)); - assertThat(innerHits.getAt(0).getId(), equalTo("4")); - assertThat(innerHits.getAt(0).getType(), equalTo("comment")); - assertThat(innerHits.getAt(1).getId(), equalTo("5")); - assertThat(innerHits.getAt(1).getType(), equalTo("comment")); - assertThat(innerHits.getAt(2).getId(), equalTo("6")); - assertThat(innerHits.getAt(2).getType(), equalTo("comment")); + assertThat(innerHits.getAt(0).getId(), equalTo("c4")); + assertThat(innerHits.getAt(0).getType(), equalTo(legacy() ? "comment" : "doc")); + assertThat(innerHits.getAt(1).getId(), equalTo("c5")); + assertThat(innerHits.getAt(1).getType(), equalTo(legacy() ? "comment" : "doc")); + assertThat(innerHits.getAt(2).getId(), equalTo("c6")); + assertThat(innerHits.getAt(2).getType(), equalTo(legacy() ? "comment" : "doc")); response = client().prepareSearch("articles") .setQuery( @@ -172,12 +163,22 @@ public void testSimpleParentChild() throws Exception { } public void testRandomParentChild() throws Exception { - assertAcked(prepareCreate("idx") - .setSettings("index.mapping.single_type", false) - .addMapping("parent") - .addMapping("child1", "_parent", "type=parent") - .addMapping("child2", "_parent", "type=parent") - ); + if (legacy()) { + assertAcked(prepareCreate("idx") + .addMapping("parent") + .addMapping("child1", "_parent", "type=parent") + .addMapping("child2", "_parent", "type=parent") + ); + } else { + assertAcked(prepareCreate("idx") + .addMapping("doc", jsonBuilder().startObject().startObject("doc").startObject("properties") + .startObject("join_field") + .field("type", "join") + .field("parent", new String[] {"child1", "child2"}) + .endObject() + .endObject().endObject().endObject() + )); + } int numDocs = scaledRandomIntBetween(5, 50); List requestBuilders = new ArrayList<>(); @@ -186,20 +187,18 @@ public void testRandomParentChild() throws Exception { int[] child1InnerObjects = new int[numDocs]; int[] child2InnerObjects = new int[numDocs]; for (int parent = 0; parent < numDocs; parent++) { - String parentId = String.format(Locale.ENGLISH, "%03d", parent); - requestBuilders.add(client().prepareIndex("idx", "parent", parentId).setSource("{}", XContentType.JSON)); + String parentId = String.format(Locale.ENGLISH, "p_%03d", parent); + requestBuilders.add(createIndexRequest("idx", "parent", parentId, null)); int numChildDocs = child1InnerObjects[parent] = scaledRandomIntBetween(1, numDocs); int limit = child1 + numChildDocs; for (; child1 < limit; child1++) { - requestBuilders.add(client().prepareIndex("idx", "child1", - String.format(Locale.ENGLISH, "%04d", child1)).setParent(parentId).setSource("{}", XContentType.JSON)); + requestBuilders.add(createIndexRequest("idx", "child1", String.format(Locale.ENGLISH, "c1_%04d", child1), parentId)); } numChildDocs = child2InnerObjects[parent] = scaledRandomIntBetween(1, numDocs); limit = child2 + numChildDocs; for (; child2 < limit; child2++) { - requestBuilders.add(client().prepareIndex("idx", "child2", - String.format(Locale.ENGLISH, "%04d", child2)).setParent(parentId).setSource("{}", XContentType.JSON)); + requestBuilders.add(createIndexRequest("idx", "child2", String.format(Locale.ENGLISH, "c2_%04d", child2), parentId)); } } indexRandom(true, requestBuilders); @@ -214,7 +213,6 @@ public void testRandomParentChild() throws Exception { .addSort(new FieldSortBuilder("_uid").order(SortOrder.ASC)).setSize(size)))); SearchResponse searchResponse = client().prepareSearch("idx") .setSize(numDocs) - .setTypes("parent") .addSort("_uid", SortOrder.ASC) .setQuery(boolQuery) .get(); @@ -227,16 +225,16 @@ public void testRandomParentChild() throws Exception { int offset2 = 0; for (int parent = 0; parent < numDocs; parent++) { SearchHit searchHit = searchResponse.getHits().getAt(parent); - assertThat(searchHit.getType(), equalTo("parent")); - assertThat(searchHit.getId(), equalTo(String.format(Locale.ENGLISH, "%03d", parent))); + assertThat(searchHit.getType(), equalTo(legacy() ? "parent" : "doc")); + assertThat(searchHit.getId(), equalTo(String.format(Locale.ENGLISH, "p_%03d", parent))); assertThat(searchHit.getShard(), notNullValue()); SearchHits inner = searchHit.getInnerHits().get("a"); assertThat(inner.getTotalHits(), equalTo((long) child1InnerObjects[parent])); for (int child = 0; child < child1InnerObjects[parent] && child < size; child++) { SearchHit innerHit = inner.getAt(child); - assertThat(innerHit.getType(), equalTo("child1")); - String childId = String.format(Locale.ENGLISH, "%04d", offset1 + child); + assertThat(innerHit.getType(), equalTo(legacy() ? "child1" : "doc")); + String childId = String.format(Locale.ENGLISH, "c1_%04d", offset1 + child); assertThat(innerHit.getId(), equalTo(childId)); assertThat(innerHit.getNestedIdentity(), nullValue()); } @@ -246,8 +244,8 @@ public void testRandomParentChild() throws Exception { assertThat(inner.getTotalHits(), equalTo((long) child2InnerObjects[parent])); for (int child = 0; child < child2InnerObjects[parent] && child < size; child++) { SearchHit innerHit = inner.getAt(child); - assertThat(innerHit.getType(), equalTo("child2")); - String childId = String.format(Locale.ENGLISH, "%04d", offset2 + child); + assertThat(innerHit.getType(), equalTo(legacy() ? "child2" : "doc")); + String childId = String.format(Locale.ENGLISH, "c2_%04d", offset2 + child); assertThat(innerHit.getId(), equalTo(childId)); assertThat(innerHit.getNestedIdentity(), nullValue()); } @@ -256,24 +254,27 @@ public void testRandomParentChild() throws Exception { } public void testInnerHitsOnHasParent() throws Exception { - assertAcked(prepareCreate("stack") - .setSettings("index.mapping.single_type", false) - .addMapping("question", "body", "type=text") - .addMapping("answer", "_parent", "type=question", "body", "type=text") - ); + if (legacy()) { + assertAcked(prepareCreate("stack") + .addMapping("question", "body", "type=text") + .addMapping("answer", "_parent", "type=question", "body", "type=text") + ); + } else { + assertAcked(prepareCreate("stack") + .addMapping("doc", "join_field", "type=join,question=answer", "body", "type=text") + ); + } List requests = new ArrayList<>(); - requests.add(client().prepareIndex("stack", "question", "1").setSource("body", "I'm using HTTPS + Basic authentication " + requests.add(createIndexRequest("stack", "question", "1", null, "body", "I'm using HTTPS + Basic authentication " + "to protect a resource. How can I throttle authentication attempts to protect against brute force attacks?")); - requests.add(client().prepareIndex("stack", "answer", "1").setParent("1").setSource("body", - "install fail2ban and enable rules for apache")); - requests.add(client().prepareIndex("stack", "question", "2").setSource("body", + requests.add(createIndexRequest("stack", "answer", "3", "1", "body", "install fail2ban and enable rules for apache")); + requests.add(createIndexRequest("stack", "question", "2", null, "body", "I have firewall rules set up and also denyhosts installed.\\ndo I also need to install fail2ban?")); - requests.add(client().prepareIndex("stack", "answer", "2").setParent("2").setSource("body", + requests.add(createIndexRequest("stack", "answer", "4", "2", "body", "Denyhosts protects only ssh; Fail2Ban protects all daemons.")); indexRandom(true, requests); SearchResponse response = client().prepareSearch("stack") - .setTypes("answer") .addSort("_uid", SortOrder.ASC) .setQuery( boolQuery() @@ -284,35 +285,41 @@ public void testInnerHitsOnHasParent() throws Exception { assertHitCount(response, 2); SearchHit searchHit = response.getHits().getAt(0); - assertThat(searchHit.getId(), equalTo("1")); - assertThat(searchHit.getType(), equalTo("answer")); + assertThat(searchHit.getId(), equalTo("3")); + assertThat(searchHit.getType(), equalTo(legacy() ? "answer" : "doc")); assertThat(searchHit.getInnerHits().get("question").getTotalHits(), equalTo(1L)); - assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo("question")); + assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo(legacy() ? "question" : "doc")); assertThat(searchHit.getInnerHits().get("question").getAt(0).getId(), equalTo("1")); searchHit = response.getHits().getAt(1); - assertThat(searchHit.getId(), equalTo("2")); - assertThat(searchHit.getType(), equalTo("answer")); + assertThat(searchHit.getId(), equalTo("4")); + assertThat(searchHit.getType(), equalTo(legacy() ? "answer" : "doc")); assertThat(searchHit.getInnerHits().get("question").getTotalHits(), equalTo(1L)); - assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo("question")); + assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo(legacy() ? "question" : "doc")); assertThat(searchHit.getInnerHits().get("question").getAt(0).getId(), equalTo("2")); } public void testParentChildMultipleLayers() throws Exception { - assertAcked(prepareCreate("articles") - .setSettings("index.mapping.single_type", false) - .addMapping("article", "title", "type=text") - .addMapping("comment", "_parent", "type=article", "message", "type=text") - .addMapping("remark", "_parent", "type=comment", "message", "type=text") - ); + if (legacy()) { + assertAcked(prepareCreate("articles") + .addMapping("article", "title", "type=text") + .addMapping("comment", "_parent", "type=article", "message", "type=text") + .addMapping("remark", "_parent", "type=comment", "message", "type=text") + ); + } else { + assertAcked(prepareCreate("articles") + .addMapping("doc", "join_field", "type=join,article=comment,comment=remark", + "title", "type=text", "message", "type=text") + ); + } List requests = new ArrayList<>(); - requests.add(client().prepareIndex("articles", "article", "1").setSource("title", "quick brown fox")); - requests.add(client().prepareIndex("articles", "comment", "1").setParent("1").setSource("message", "fox eat quick")); - requests.add(client().prepareIndex("articles", "remark", "1").setParent("1").setRouting("1").setSource("message", "good")); - requests.add(client().prepareIndex("articles", "article", "2").setSource("title", "big gray elephant")); - requests.add(client().prepareIndex("articles", "comment", "2").setParent("2").setSource("message", "elephant captured")); - requests.add(client().prepareIndex("articles", "remark", "2").setParent("2").setRouting("2").setSource("message", "bad")); + requests.add(createIndexRequest("articles", "article", "1", null, "title", "quick brown fox")); + requests.add(createIndexRequest("articles", "comment", "3", "1", "message", "fox eat quick")); + requests.add(createIndexRequest("articles", "remark", "5", "3", "message", "good").setRouting("1")); + requests.add(createIndexRequest("articles", "article", "2", null, "title", "big gray elephant")); + requests.add(createIndexRequest("articles", "comment", "4", "2", "message", "elephant captured")); + requests.add(createIndexRequest("articles", "remark", "6", "4", "message", "bad").setRouting("2")); indexRandom(true, requests); SearchResponse response = client().prepareSearch("articles") @@ -328,13 +335,13 @@ public void testParentChildMultipleLayers() throws Exception { assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); assertThat(innerHits.getTotalHits(), equalTo(1L)); - assertThat(innerHits.getAt(0).getId(), equalTo("1")); - assertThat(innerHits.getAt(0).getType(), equalTo("comment")); + assertThat(innerHits.getAt(0).getId(), equalTo("3")); + assertThat(innerHits.getAt(0).getType(), equalTo(legacy() ? "comment" : "doc")); innerHits = innerHits.getAt(0).getInnerHits().get("remark"); assertThat(innerHits.getTotalHits(), equalTo(1L)); - assertThat(innerHits.getAt(0).getId(), equalTo("1")); - assertThat(innerHits.getAt(0).getType(), equalTo("remark")); + assertThat(innerHits.getAt(0).getId(), equalTo("5")); + assertThat(innerHits.getAt(0).getType(), equalTo(legacy() ? "remark" : "doc")); response = client().prepareSearch("articles") .setQuery(hasChildQuery("comment", @@ -349,51 +356,47 @@ public void testParentChildMultipleLayers() throws Exception { assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); assertThat(innerHits.getTotalHits(), equalTo(1L)); - assertThat(innerHits.getAt(0).getId(), equalTo("2")); - assertThat(innerHits.getAt(0).getType(), equalTo("comment")); + assertThat(innerHits.getAt(0).getId(), equalTo("4")); + assertThat(innerHits.getAt(0).getType(), equalTo(legacy() ? "comment" : "doc")); innerHits = innerHits.getAt(0).getInnerHits().get("remark"); assertThat(innerHits.getTotalHits(), equalTo(1L)); - assertThat(innerHits.getAt(0).getId(), equalTo("2")); - assertThat(innerHits.getAt(0).getType(), equalTo("remark")); + assertThat(innerHits.getAt(0).getId(), equalTo("6")); + assertThat(innerHits.getAt(0).getType(), equalTo(legacy() ? "remark" : "doc")); } public void testRoyals() throws Exception { - assertAcked( - prepareCreate("royals") - .setSettings("index.mapping.single_type", false) - .addMapping("king") - .addMapping("prince", "_parent", "type=king") - .addMapping("duke", "_parent", "type=prince") - .addMapping("earl", "_parent", "type=duke") - .addMapping("baron", "_parent", "type=earl") - ); + if (legacy()) { + assertAcked( + prepareCreate("royals") + .addMapping("king") + .addMapping("prince", "_parent", "type=king") + .addMapping("duke", "_parent", "type=prince") + .addMapping("earl", "_parent", "type=duke") + .addMapping("baron", "_parent", "type=earl") + ); + } else { + assertAcked( + prepareCreate("royals") + .addMapping("doc", "join_field", "type=join,king=prince,prince=duke,duke=earl,earl=baron") + ); + } List requests = new ArrayList<>(); - requests.add(client().prepareIndex("royals", "king", "king").setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "prince", "prince").setParent("king").setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "duke", "duke").setParent("prince").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "earl", "earl1").setParent("duke").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "earl", "earl2").setParent("duke").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "earl", "earl3").setParent("duke").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "earl", "earl4").setParent("duke").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "baron", "baron1").setParent("earl1").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "baron", "baron2").setParent("earl2").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "baron", "baron3").setParent("earl3").setRouting("king") - .setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("royals", "baron", "baron4").setParent("earl4").setRouting("king") - .setSource("{}", XContentType.JSON)); + requests.add(createIndexRequest("royals", "king", "king", null)); + requests.add(createIndexRequest("royals", "prince", "prince", "king")); + requests.add(createIndexRequest("royals", "duke", "duke", "prince").setRouting("king")); + requests.add(createIndexRequest("royals", "earl", "earl1", "duke").setRouting("king")); + requests.add(createIndexRequest("royals", "earl", "earl2", "duke").setRouting("king")); + requests.add(createIndexRequest("royals", "earl", "earl3", "duke").setRouting("king")); + requests.add(createIndexRequest("royals", "earl", "earl4", "duke").setRouting("king")); + requests.add(createIndexRequest("royals", "baron", "baron1", "earl1").setRouting("king")); + requests.add(createIndexRequest("royals", "baron", "baron2", "earl2").setRouting("king")); + requests.add(createIndexRequest("royals", "baron", "baron3", "earl3").setRouting("king")); + requests.add(createIndexRequest("royals", "baron", "baron4", "earl4").setRouting("king")); indexRandom(true, requests); SearchResponse response = client().prepareSearch("royals") - .setTypes("duke") .setQuery(boolQuery() .filter(hasParentQuery("prince", hasParentQuery("king", matchAllQuery(), false).innerHit(new InnerHitBuilder().setName("kings")), @@ -407,8 +410,7 @@ public void testRoyals() throws Exception { .setName("earls") .setSize(4)) ) - ) - .get(); + ).get(); assertHitCount(response, 1); assertThat(response.getHits().getAt(0).getId(), equalTo("duke")); @@ -445,15 +447,19 @@ public void testRoyals() throws Exception { } public void testMatchesQueriesParentChildInnerHits() throws Exception { - assertAcked(prepareCreate("index") - .setSettings("index.mapping.single_type", false) - .addMapping("child", "_parent", "type=parent")); + if (legacy()) { + assertAcked(prepareCreate("index") + .addMapping("child", "_parent", "type=parent")); + } else { + assertAcked(prepareCreate("index") + .addMapping("doc", "join_field", "type=join,parent=child")); + } List requests = new ArrayList<>(); - requests.add(client().prepareIndex("index", "parent", "1").setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("index", "child", "1").setParent("1").setSource("field", "value1")); - requests.add(client().prepareIndex("index", "child", "2").setParent("1").setSource("field", "value2")); - requests.add(client().prepareIndex("index", "parent", "2").setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("index", "child", "3").setParent("2").setSource("field", "value1")); + requests.add(createIndexRequest("index", "parent", "1", null)); + requests.add(createIndexRequest("index", "child", "3", "1", "field", "value1")); + requests.add(createIndexRequest("index", "child", "4", "1", "field", "value2")); + requests.add(createIndexRequest("index", "parent", "2", null)); + requests.add(createIndexRequest("index", "child", "5", "2", "field", "value1")); indexRandom(true, requests); SearchResponse response = client().prepareSearch("index") @@ -486,12 +492,14 @@ public void testMatchesQueriesParentChildInnerHits() throws Exception { } public void testDontExplode() throws Exception { - assertAcked(prepareCreate("index1") - .setSettings("index.mapping.single_type", false) - .addMapping("child", "_parent", "type=parent")); + if (legacy()) { + assertAcked(prepareCreate("index1").addMapping("child", "_parent", "type=parent")); + } else { + assertAcked(prepareCreate("index1").addMapping("doc", "join_field", "type=join,parent=child")); + } List requests = new ArrayList<>(); - requests.add(client().prepareIndex("index1", "parent", "1").setSource("{}", XContentType.JSON)); - requests.add(client().prepareIndex("index1", "child", "1").setParent("1").setSource("field", "value1")); + requests.add(createIndexRequest("index1", "parent", "1", null)); + requests.add(createIndexRequest("index1", "child", "2", "1", "field", "value1")); indexRandom(true, requests); QueryBuilder query = hasChildQuery("child", matchQuery("field", "value1"), ScoreMode.None) @@ -501,34 +509,18 @@ public void testDontExplode() throws Exception { .get(); assertNoFailures(response); assertHitCount(response, 1); - - assertAcked(prepareCreate("index2").addMapping("type", "nested", "type=nested")); - client().prepareIndex("index2", "type", "1").setSource(jsonBuilder().startObject() - .startArray("nested") - .startObject() - .field("field", "value1") - .endObject() - .endArray() - .endObject()) - .setRefreshPolicy(IMMEDIATE) - .get(); - - query = nestedQuery("nested", matchQuery("nested.field", "value1"), ScoreMode.Avg) - .innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1)); - response = client().prepareSearch("index2") - .setQuery(query) - .get(); - assertNoFailures(response); - assertHitCount(response, 1); } public void testNestedInnerHitWrappedInParentChildInnerhit() throws Exception { - assertAcked(prepareCreate("test") - .setSettings("index.mapping.single_type", false) - .addMapping("child_type", "_parent", "type=parent_type", "nested_type", "type=nested")); - client().prepareIndex("test", "parent_type", "1").setSource("key", "value").get(); - client().prepareIndex("test", "child_type", "2").setParent("1").setSource("nested_type", Collections.singletonMap("key", "value")) - .get(); + if (legacy()) { + assertAcked(prepareCreate("test") + .addMapping("child_type", "_parent", "type=parent_type", "nested_type", "type=nested")); + } else { + assertAcked(prepareCreate("test") + .addMapping("doc", "join_field", "type=join,parent_type=child_type", "nested_type", "type=nested")); + } + createIndexRequest("test", "parent_type", "1", null, "key", "value").get(); + createIndexRequest("test", "child_type", "2", "1", "nested_type", Collections.singletonMap("key", "value")).get(); refresh(); SearchResponse response = client().prepareSearch("test") .setQuery(boolQuery().must(matchQuery("key", "value")) @@ -537,21 +529,28 @@ public void testNestedInnerHitWrappedInParentChildInnerhit() throws Exception { .get(); assertHitCount(response, 1); SearchHit hit = response.getHits().getAt(0); - assertThat(hit.getInnerHits().get("child_type").getAt(0).field("_parent").getValue(), equalTo("1")); + if (legacy()) { + assertThat(hit.getInnerHits().get("child_type").getAt(0).field("_parent").getValue(), equalTo("1")); + } else { + assertThat(hit.getInnerHits().get("child_type").getAt(0).field("join_field#parent_type").getValue(), equalTo("1")); + } assertThat(hit.getInnerHits().get("child_type").getAt(0).getInnerHits().get("nested_type").getAt(0).field("_parent"), nullValue()); } public void testInnerHitsWithIgnoreUnmapped() throws Exception { - assertAcked(prepareCreate("index1") - .setSettings("index.mapping.single_type", false) - .addMapping("parent_type", "nested_type", "type=nested") - .addMapping("child_type", "_parent", "type=parent_type") - ); - assertAcked(prepareCreate("index2") - .setSettings("index.mapping.single_type", false) - ); - client().prepareIndex("index1", "parent_type", "1").setSource("nested_type", Collections.singletonMap("key", "value")).get(); - client().prepareIndex("index1", "child_type", "2").setParent("1").setSource("{}", XContentType.JSON).get(); + if (legacy()) { + assertAcked(prepareCreate("index1") + .addMapping("parent_type", "nested_type", "type=nested") + .addMapping("child_type", "_parent", "type=parent_type") + ); + } else { + assertAcked(prepareCreate("index1") + .addMapping("doc", "join_field", "type=join,parent_type=child_type", "nested_type", "type=nested") + ); + } + assertAcked(prepareCreate("index2")); + createIndexRequest("index1", "parent_type", "1", null, "nested_type", Collections.singletonMap("key", "value")).get(); + createIndexRequest("index1", "child_type", "2", "1").get(); client().prepareIndex("index2", "type", "3").setSource("key", "value").get(); refresh(); diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyChildQuerySearchIT.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyChildQuerySearchIT.java index 445cb4be14970..a227d9e44be73 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyChildQuerySearchIT.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyChildQuerySearchIT.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -55,15 +54,6 @@ protected boolean legacy() { return true; } - @Override - public Settings indexSettings() { - Settings indexSettings = super.indexSettings(); - return Settings.builder() - .put(indexSettings) - .put("index.mapping.single_type", false) - .build(); - } - public void testIndexChildDocWithNoParentMapping() throws IOException { assertAcked(prepareCreate("test") .addMapping("parent") diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyHasChildQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyHasChildQueryBuilderTests.java new file mode 100644 index 0000000000000..2bf6a0f2d3b25 --- /dev/null +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyHasChildQueryBuilderTests.java @@ -0,0 +1,353 @@ +/* + * 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.query; + +import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermInSetQuery; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.join.ScoreMode; +import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper; +import org.apache.lucene.search.similarities.Similarity; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.TypeFieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.index.query.IdsQueryBuilder; +import org.elasticsearch.index.query.InnerHitBuilder; +import org.elasticsearch.index.query.InnerHitContextBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.index.query.QueryShardException; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.query.WrapperQueryBuilder; +import org.elasticsearch.index.similarity.SimilarityService; +import org.elasticsearch.join.ParentJoinPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.fetch.subphase.InnerHitsContext; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.test.AbstractQueryTestCase; +import org.elasticsearch.test.VersionUtils; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.elasticsearch.join.query.JoinQueryBuilders.hasChildQuery; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.notNullValue; + +public class LegacyHasChildQueryBuilderTests extends AbstractQueryTestCase { + protected static final String PARENT_TYPE = "parent"; + protected static final String CHILD_TYPE = "child"; + + private static String similarity; + + boolean requiresRewrite = false; + + @Override + protected Collection> getPlugins() { + return Collections.singletonList(ParentJoinPlugin.class); + } + + @Override + protected void initializeAdditionalMappings(MapperService mapperService) throws IOException { + similarity = randomFrom("classic", "BM25"); + // TODO: use a single type when inner hits have been changed to work with join field, + // this test randomly generates queries with inner hits + mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE, + STRING_FIELD_NAME, "type=text", + STRING_FIELD_NAME_2, "type=keyword", + INT_FIELD_NAME, "type=integer", + DOUBLE_FIELD_NAME, "type=double", + BOOLEAN_FIELD_NAME, "type=boolean", + DATE_FIELD_NAME, "type=date", + OBJECT_FIELD_NAME, "type=object" + ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); + mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE, + "_parent", "type=" + PARENT_TYPE, + STRING_FIELD_NAME, "type=text", + "custom_string", "type=text,similarity=" + similarity, + INT_FIELD_NAME, "type=integer", + DOUBLE_FIELD_NAME, "type=double", + BOOLEAN_FIELD_NAME, "type=boolean", + DATE_FIELD_NAME, "type=date", + OBJECT_FIELD_NAME, "type=object" + ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); + } + + @Override + protected Settings indexSettings() { + return Settings.builder() + .put(super.indexSettings()) + .put("index.mapping.single_type", false) + .build(); + } + + /** + * @return a {@link HasChildQueryBuilder} with random values all over the place + */ + @Override + protected HasChildQueryBuilder doCreateTestQueryBuilder() { + int min = randomIntBetween(0, Integer.MAX_VALUE / 2); + int max = randomIntBetween(min, Integer.MAX_VALUE); + + QueryBuilder innerQueryBuilder = new MatchAllQueryBuilder(); + if (randomBoolean()) { + requiresRewrite = true; + innerQueryBuilder = new WrapperQueryBuilder(innerQueryBuilder.toString()); + } + + HasChildQueryBuilder hqb = new HasChildQueryBuilder(CHILD_TYPE, innerQueryBuilder, + RandomPicks.randomFrom(random(), ScoreMode.values())); + hqb.minMaxChildren(min, max); + hqb.ignoreUnmapped(randomBoolean()); + if (randomBoolean()) { + hqb.innerHit(new InnerHitBuilder() + .setName(randomAlphaOfLengthBetween(1, 10)) + .setSize(randomIntBetween(0, 100)) + .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC)) + .setIgnoreUnmapped(hqb.ignoreUnmapped())); + } + return hqb; + } + + @Override + protected void doAssertLuceneQuery(HasChildQueryBuilder queryBuilder, Query query, SearchContext searchContext) throws IOException { + assertThat(query, instanceOf(HasChildQueryBuilder.LateParsingQuery.class)); + HasChildQueryBuilder.LateParsingQuery lpq = (HasChildQueryBuilder.LateParsingQuery) query; + assertEquals(queryBuilder.minChildren(), lpq.getMinChildren()); + assertEquals(queryBuilder.maxChildren(), lpq.getMaxChildren()); + assertEquals(queryBuilder.scoreMode(), lpq.getScoreMode()); // WTF is this why do we have two? + if (queryBuilder.innerHit() != null) { + // have to rewrite again because the provided queryBuilder hasn't been rewritten (directly returned from + // doCreateTestQueryBuilder) + queryBuilder = (HasChildQueryBuilder) queryBuilder.rewrite(searchContext.getQueryShardContext()); + Map innerHitBuilders = new HashMap<>(); + InnerHitContextBuilder.extractInnerHits(queryBuilder, innerHitBuilders); + for (InnerHitContextBuilder builder : innerHitBuilders.values()) { + builder.build(searchContext, searchContext.innerHits()); + } + assertNotNull(searchContext.innerHits()); + assertEquals(1, searchContext.innerHits().getInnerHits().size()); + assertTrue(searchContext.innerHits().getInnerHits().containsKey(queryBuilder.innerHit().getName())); + InnerHitsContext.InnerHitSubContext innerHits = + searchContext.innerHits().getInnerHits().get(queryBuilder.innerHit().getName()); + assertEquals(innerHits.size(), queryBuilder.innerHit().getSize()); + assertEquals(innerHits.sort().sort.getSort().length, 1); + assertEquals(innerHits.sort().sort.getSort()[0].getField(), STRING_FIELD_NAME_2); + } + } + + /** + * Test (de)serialization on all previous released versions + */ + public void testSerializationBWC() throws IOException { + for (Version version : VersionUtils.allReleasedVersions()) { + HasChildQueryBuilder testQuery = createTestQueryBuilder(); + if (version.before(Version.V_5_2_0) && testQuery.innerHit() != null) { + // ignore unmapped for inner_hits has been added on 5.2 + testQuery.innerHit().setIgnoreUnmapped(false); + } + assertSerialization(testQuery, version); + } + } + + public void testIllegalValues() { + QueryBuilder query = new MatchAllQueryBuilder(); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, + () -> hasChildQuery(null, query, ScoreMode.None)); + assertEquals("[has_child] requires 'type' field", e.getMessage()); + + e = expectThrows(IllegalArgumentException.class, () -> hasChildQuery("foo", null, ScoreMode.None)); + assertEquals("[has_child] requires 'query' field", e.getMessage()); + + e = expectThrows(IllegalArgumentException.class, () -> hasChildQuery("foo", query, null)); + assertEquals("[has_child] requires 'score_mode' field", e.getMessage()); + + int positiveValue = randomIntBetween(0, Integer.MAX_VALUE); + HasChildQueryBuilder foo = hasChildQuery("foo", query, ScoreMode.None); // all good + e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(randomIntBetween(Integer.MIN_VALUE, -1), positiveValue)); + assertEquals("[has_child] requires non-negative 'min_children' field", e.getMessage()); + + e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(positiveValue, randomIntBetween(Integer.MIN_VALUE, -1))); + assertEquals("[has_child] requires non-negative 'max_children' field", e.getMessage()); + + e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(positiveValue, positiveValue - 10)); + assertEquals("[has_child] 'max_children' is less than 'min_children'", e.getMessage()); + } + + public void testFromJson() throws IOException { + String query = + "{\n" + + " \"has_child\" : {\n" + + " \"query\" : {\n" + + " \"range\" : {\n" + + " \"mapped_string\" : {\n" + + " \"from\" : \"agJhRET\",\n" + + " \"to\" : \"zvqIq\",\n" + + " \"include_lower\" : true,\n" + + " \"include_upper\" : true,\n" + + " \"boost\" : 1.0\n" + + " }\n" + + " }\n" + + " },\n" + + " \"type\" : \"child\",\n" + + " \"score_mode\" : \"avg\",\n" + + " \"min_children\" : 883170873,\n" + + " \"max_children\" : 1217235442,\n" + + " \"ignore_unmapped\" : false,\n" + + " \"boost\" : 2.0,\n" + + " \"_name\" : \"WNzYMJKRwePuRBh\",\n" + + " \"inner_hits\" : {\n" + + " \"name\" : \"inner_hits_name\",\n" + + " \"ignore_unmapped\" : false,\n" + + " \"from\" : 0,\n" + + " \"size\" : 100,\n" + + " \"version\" : false,\n" + + " \"explain\" : false,\n" + + " \"track_scores\" : false,\n" + + " \"sort\" : [ {\n" + + " \"mapped_string\" : {\n" + + " \"order\" : \"asc\"\n" + + " }\n" + + " } ]\n" + + " }\n" + + " }\n" + + "}"; + HasChildQueryBuilder queryBuilder = (HasChildQueryBuilder) parseQuery(query); + checkGeneratedJson(query, queryBuilder); + assertEquals(query, queryBuilder.maxChildren(), 1217235442); + assertEquals(query, queryBuilder.minChildren(), 883170873); + assertEquals(query, queryBuilder.boost(), 2.0f, 0.0f); + assertEquals(query, queryBuilder.queryName(), "WNzYMJKRwePuRBh"); + assertEquals(query, queryBuilder.childType(), "child"); + assertEquals(query, queryBuilder.scoreMode(), ScoreMode.Avg); + assertNotNull(query, queryBuilder.innerHit()); + InnerHitBuilder expected = new InnerHitBuilder("child") + .setName("inner_hits_name") + .setSize(100) + .addSort(new FieldSortBuilder("mapped_string").order(SortOrder.ASC)); + assertEquals(query, queryBuilder.innerHit(), expected); + } + + public void testToQueryInnerQueryType() throws IOException { + String[] searchTypes = new String[]{PARENT_TYPE}; + QueryShardContext shardContext = createShardContext(); + shardContext.setTypes(searchTypes); + HasChildQueryBuilder hasChildQueryBuilder = hasChildQuery(CHILD_TYPE, new IdsQueryBuilder().addIds("id"), ScoreMode.None); + Query query = hasChildQueryBuilder.toQuery(shardContext); + //verify that the context types are still the same as the ones we previously set + assertThat(shardContext.getTypes(), equalTo(searchTypes)); + assertLateParsingQuery(query, CHILD_TYPE, "id"); + } + + static void assertLateParsingQuery(Query query, String type, String id) throws IOException { + assertThat(query, instanceOf(HasChildQueryBuilder.LateParsingQuery.class)); + HasChildQueryBuilder.LateParsingQuery lateParsingQuery = (HasChildQueryBuilder.LateParsingQuery) query; + assertThat(lateParsingQuery.getInnerQuery(), instanceOf(BooleanQuery.class)); + BooleanQuery booleanQuery = (BooleanQuery) lateParsingQuery.getInnerQuery(); + assertThat(booleanQuery.clauses().size(), equalTo(2)); + //check the inner ids query, we have to call rewrite to get to check the type it's executed against + assertThat(booleanQuery.clauses().get(0).getOccur(), equalTo(BooleanClause.Occur.MUST)); + assertThat(booleanQuery.clauses().get(0).getQuery(), instanceOf(TermInSetQuery.class)); + TermInSetQuery termsQuery = (TermInSetQuery) booleanQuery.clauses().get(0).getQuery(); + Query rewrittenTermsQuery = termsQuery.rewrite(null); + assertThat(rewrittenTermsQuery, instanceOf(ConstantScoreQuery.class)); + ConstantScoreQuery constantScoreQuery = (ConstantScoreQuery) rewrittenTermsQuery; + assertThat(constantScoreQuery.getQuery(), instanceOf(BooleanQuery.class)); + BooleanQuery booleanTermsQuery = (BooleanQuery) constantScoreQuery.getQuery(); + assertThat(booleanTermsQuery.clauses().toString(), booleanTermsQuery.clauses().size(), equalTo(1)); + assertThat(booleanTermsQuery.clauses().get(0).getOccur(), equalTo(BooleanClause.Occur.SHOULD)); + assertThat(booleanTermsQuery.clauses().get(0).getQuery(), instanceOf(TermQuery.class)); + TermQuery termQuery = (TermQuery) booleanTermsQuery.clauses().get(0).getQuery(); + assertThat(termQuery.getTerm().field(), equalTo(UidFieldMapper.NAME)); + //we want to make sure that the inner ids query gets executed against the child type rather + // than the main type we initially set to the context + BytesRef[] ids = Uid.createUidsForTypesAndIds(Collections.singletonList(type), Collections.singletonList(id)); + assertThat(termQuery.getTerm().bytes(), equalTo(ids[0])); + //check the type filter + assertThat(booleanQuery.clauses().get(1).getOccur(), equalTo(BooleanClause.Occur.FILTER)); + assertEquals(new TypeFieldMapper.TypesQuery(new BytesRef(type)), booleanQuery.clauses().get(1).getQuery()); + } + + @Override + public void testMustRewrite() throws IOException { + try { + super.testMustRewrite(); + } catch (UnsupportedOperationException e) { + if (requiresRewrite == false) { + throw e; + } + } + } + + public void testNonDefaultSimilarity() throws Exception { + QueryShardContext shardContext = createShardContext(); + HasChildQueryBuilder hasChildQueryBuilder = + hasChildQuery(CHILD_TYPE, new TermQueryBuilder("custom_string", "value"), ScoreMode.None); + HasChildQueryBuilder.LateParsingQuery query = (HasChildQueryBuilder.LateParsingQuery) hasChildQueryBuilder.toQuery(shardContext); + Similarity expected = SimilarityService.BUILT_IN.get(similarity) + .apply(similarity, Settings.EMPTY, Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()) + .get(); + assertThat(((PerFieldSimilarityWrapper) query.getSimilarity()).get("custom_string"), instanceOf(expected.getClass())); + } + + public void testIgnoreUnmapped() throws IOException { + final HasChildQueryBuilder queryBuilder = new HasChildQueryBuilder("unmapped", new MatchAllQueryBuilder(), ScoreMode.None); + queryBuilder.ignoreUnmapped(true); + Query query = queryBuilder.toQuery(createShardContext()); + assertThat(query, notNullValue()); + assertThat(query, instanceOf(MatchNoDocsQuery.class)); + + final HasChildQueryBuilder failingQueryBuilder = new HasChildQueryBuilder("unmapped", new MatchAllQueryBuilder(), ScoreMode.None); + failingQueryBuilder.ignoreUnmapped(false); + QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext())); + assertThat(e.getMessage(), containsString("[" + HasChildQueryBuilder.NAME + "] no mapping found for type [unmapped]")); + } + + public void testIgnoreUnmappedWithRewrite() throws IOException { + // WrapperQueryBuilder makes sure we always rewrite + final HasChildQueryBuilder queryBuilder + = new HasChildQueryBuilder("unmapped", new WrapperQueryBuilder(new MatchAllQueryBuilder().toString()), ScoreMode.None); + queryBuilder.ignoreUnmapped(true); + QueryShardContext queryShardContext = createShardContext(); + Query query = queryBuilder.rewrite(queryShardContext).toQuery(queryShardContext); + assertThat(query, notNullValue()); + assertThat(query, instanceOf(MatchNoDocsQuery.class)); + } +} diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyHasParentQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyHasParentQueryBuilderTests.java new file mode 100644 index 0000000000000..0d84be4874b03 --- /dev/null +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyHasParentQueryBuilderTests.java @@ -0,0 +1,274 @@ +/* + * 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.query; + +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.join.ScoreMode; +import org.elasticsearch.Version; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.query.IdsQueryBuilder; +import org.elasticsearch.index.query.InnerHitBuilder; +import org.elasticsearch.index.query.InnerHitContextBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.index.query.QueryShardException; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.query.WrapperQueryBuilder; +import org.elasticsearch.join.ParentJoinPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.fetch.subphase.InnerHitsContext; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.test.AbstractQueryTestCase; +import org.elasticsearch.test.VersionUtils; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.elasticsearch.join.query.JoinQueryBuilders.hasParentQuery; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.notNullValue; + +public class LegacyHasParentQueryBuilderTests extends AbstractQueryTestCase { + protected static final String PARENT_TYPE = "parent"; + protected static final String CHILD_TYPE = "child"; + + boolean requiresRewrite = false; + + @Override + protected Collection> getPlugins() { + return Collections.singletonList(ParentJoinPlugin.class); + } + + @Override + protected Settings indexSettings() { + return Settings.builder() + .put(super.indexSettings()) + .put("index.mapping.single_type", false) + .build(); + } + + @Override + protected void initializeAdditionalMappings(MapperService mapperService) throws IOException { + // TODO: use a single type when inner hits have been changed to work with join field, + // this test randomly generates queries with inner hits + mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE, + STRING_FIELD_NAME, "type=text", + STRING_FIELD_NAME_2, "type=keyword", + INT_FIELD_NAME, "type=integer", + DOUBLE_FIELD_NAME, "type=double", + BOOLEAN_FIELD_NAME, "type=boolean", + DATE_FIELD_NAME, "type=date", + OBJECT_FIELD_NAME, "type=object" + ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); + mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE, + "_parent", "type=" + PARENT_TYPE, + STRING_FIELD_NAME, "type=text", + STRING_FIELD_NAME_2, "type=keyword", + INT_FIELD_NAME, "type=integer", + DOUBLE_FIELD_NAME, "type=double", + BOOLEAN_FIELD_NAME, "type=boolean", + DATE_FIELD_NAME, "type=date", + OBJECT_FIELD_NAME, "type=object" + ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); + mapperService.merge("just_a_type", new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef("just_a_type" + ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); + } + + /** + * @return a {@link HasChildQueryBuilder} with random values all over the place + */ + @Override + protected HasParentQueryBuilder doCreateTestQueryBuilder() { + QueryBuilder innerQueryBuilder = new MatchAllQueryBuilder(); + if (randomBoolean()) { + requiresRewrite = true; + innerQueryBuilder = new WrapperQueryBuilder(innerQueryBuilder.toString()); + } + HasParentQueryBuilder hqb = new HasParentQueryBuilder(PARENT_TYPE, innerQueryBuilder, randomBoolean()); + hqb.ignoreUnmapped(randomBoolean()); + if (randomBoolean()) { + hqb.innerHit(new InnerHitBuilder() + .setName(randomAlphaOfLengthBetween(1, 10)) + .setSize(randomIntBetween(0, 100)) + .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC)) + .setIgnoreUnmapped(hqb.ignoreUnmapped())); + } + return hqb; + } + + @Override + protected void doAssertLuceneQuery(HasParentQueryBuilder queryBuilder, Query query, SearchContext searchContext) throws IOException { + assertThat(query, instanceOf(HasChildQueryBuilder.LateParsingQuery.class)); + HasChildQueryBuilder.LateParsingQuery lpq = (HasChildQueryBuilder.LateParsingQuery) query; + assertEquals(queryBuilder.score() ? ScoreMode.Max : ScoreMode.None, lpq.getScoreMode()); + + if (queryBuilder.innerHit() != null) { + // have to rewrite again because the provided queryBuilder hasn't been rewritten (directly returned from + // doCreateTestQueryBuilder) + queryBuilder = (HasParentQueryBuilder) queryBuilder.rewrite(searchContext.getQueryShardContext()); + + assertNotNull(searchContext); + Map innerHitBuilders = new HashMap<>(); + InnerHitContextBuilder.extractInnerHits(queryBuilder, innerHitBuilders); + for (InnerHitContextBuilder builder : innerHitBuilders.values()) { + builder.build(searchContext, searchContext.innerHits()); + } + assertNotNull(searchContext.innerHits()); + assertEquals(1, searchContext.innerHits().getInnerHits().size()); + assertTrue(searchContext.innerHits().getInnerHits().containsKey(queryBuilder.innerHit().getName())); + InnerHitsContext.InnerHitSubContext innerHits = searchContext.innerHits() + .getInnerHits().get(queryBuilder.innerHit().getName()); + assertEquals(innerHits.size(), queryBuilder.innerHit().getSize()); + assertEquals(innerHits.sort().sort.getSort().length, 1); + assertEquals(innerHits.sort().sort.getSort()[0].getField(), STRING_FIELD_NAME_2); + } + } + + /** + * Test (de)serialization on all previous released versions + */ + public void testSerializationBWC() throws IOException { + for (Version version : VersionUtils.allReleasedVersions()) { + HasParentQueryBuilder testQuery = createTestQueryBuilder(); + if (version.before(Version.V_5_2_0) && testQuery.innerHit() != null) { + // ignore unmapped for inner_hits has been added on 5.2 + testQuery.innerHit().setIgnoreUnmapped(false); + } + assertSerialization(testQuery, version); + } + } + + public void testIllegalValues() throws IOException { + QueryBuilder query = new MatchAllQueryBuilder(); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, + () -> hasParentQuery(null, query, false)); + assertThat(e.getMessage(), equalTo("[has_parent] requires 'type' field")); + + e = expectThrows(IllegalArgumentException.class, + () -> hasParentQuery("foo", null, false)); + assertThat(e.getMessage(), equalTo("[has_parent] requires 'query' field")); + + QueryShardContext context = createShardContext(); + HasParentQueryBuilder qb = hasParentQuery("just_a_type", new MatchAllQueryBuilder(), false); + QueryShardException qse = expectThrows(QueryShardException.class, () -> qb.doToQuery(context)); + assertThat(qse.getMessage(), equalTo("[has_parent] no child types found for type [just_a_type]")); + } + + public void testDeprecatedXContent() throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); + builder.startObject(); + builder.startObject("has_parent"); + builder.field("query"); + new TermQueryBuilder("a", "a").toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.field("type", "foo"); // deprecated + builder.endObject(); + builder.endObject(); + HasParentQueryBuilder queryBuilder = (HasParentQueryBuilder) parseQuery(builder.string()); + assertEquals("foo", queryBuilder.type()); + assertWarnings("Deprecated field [type] used, expected [parent_type] instead"); + } + + public void testToQueryInnerQueryType() throws IOException { + String[] searchTypes = new String[]{CHILD_TYPE}; + QueryShardContext shardContext = createShardContext(); + shardContext.setTypes(searchTypes); + HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(PARENT_TYPE, new IdsQueryBuilder().addIds("id"), + false); + Query query = hasParentQueryBuilder.toQuery(shardContext); + //verify that the context types are still the same as the ones we previously set + assertThat(shardContext.getTypes(), equalTo(searchTypes)); + LegacyHasChildQueryBuilderTests.assertLateParsingQuery(query, PARENT_TYPE, "id"); + } + + @Override + public void testMustRewrite() throws IOException { + try { + super.testMustRewrite(); + } catch (UnsupportedOperationException e) { + if (requiresRewrite == false) { + throw e; + } + } + } + + public void testFromJson() throws IOException { + String json = + "{\n" + + " \"has_parent\" : {\n" + + " \"query\" : {\n" + + " \"term\" : {\n" + + " \"tag\" : {\n" + + " \"value\" : \"something\",\n" + + " \"boost\" : 1.0\n" + + " }\n" + + " }\n" + + " },\n" + + " \"parent_type\" : \"blog\",\n" + + " \"score\" : true,\n" + + " \"ignore_unmapped\" : false,\n" + + " \"boost\" : 1.0\n" + + " }\n" + + "}"; + HasParentQueryBuilder parsed = (HasParentQueryBuilder) parseQuery(json); + checkGeneratedJson(json, parsed); + assertEquals(json, "blog", parsed.type()); + assertEquals(json, "something", ((TermQueryBuilder) parsed.query()).value()); + } + + public void testIgnoreUnmapped() throws IOException { + final HasParentQueryBuilder queryBuilder = new HasParentQueryBuilder("unmapped", new MatchAllQueryBuilder(), false); + queryBuilder.ignoreUnmapped(true); + Query query = queryBuilder.toQuery(createShardContext()); + assertThat(query, notNullValue()); + assertThat(query, instanceOf(MatchNoDocsQuery.class)); + + final HasParentQueryBuilder failingQueryBuilder = new HasParentQueryBuilder("unmapped", new MatchAllQueryBuilder(), false); + failingQueryBuilder.ignoreUnmapped(false); + QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext())); + assertThat(e.getMessage(), + containsString("[" + HasParentQueryBuilder.NAME + "] query configured 'parent_type' [unmapped] is not a valid type")); + } + + public void testIgnoreUnmappedWithRewrite() throws IOException { + // WrapperQueryBuilder makes sure we always rewrite + final HasParentQueryBuilder queryBuilder = + new HasParentQueryBuilder("unmapped", new WrapperQueryBuilder(new MatchAllQueryBuilder().toString()), false); + queryBuilder.ignoreUnmapped(true); + QueryShardContext queryShardContext = createShardContext(); + Query query = queryBuilder.rewrite(queryShardContext).toQuery(queryShardContext); + assertThat(query, notNullValue()); + assertThat(query, instanceOf(MatchNoDocsQuery.class)); + } +} diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyInnerHitsIT.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyInnerHitsIT.java new file mode 100644 index 0000000000000..7dff631837393 --- /dev/null +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/LegacyInnerHitsIT.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.query; + +public class LegacyInnerHitsIT extends InnerHitsIT { + + @Override + protected boolean legacy() { + return true; + } + +} diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/ParentChildTestCase.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/ParentChildTestCase.java new file mode 100644 index 0000000000000..099671d8214c9 --- /dev/null +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/ParentChildTestCase.java @@ -0,0 +1,114 @@ +/* + * 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.query; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.index.IndexModule; +import org.elasticsearch.join.ParentJoinPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE) +public abstract class ParentChildTestCase extends ESIntegTestCase { + + @Override + protected boolean ignoreExternalCluster() { + return true; + } + + @Override + protected Collection> nodePlugins() { + return Collections.singleton(ParentJoinPlugin.class); + } + + @Override + protected Collection> transportClientPlugins() { + return nodePlugins(); + } + + @Override + public Settings indexSettings() { + Settings.Builder builder = Settings.builder().put(super.indexSettings()) + // aggressive filter caching so that we can assert on the filter cache size + .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), true) + .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), true); + + if (legacy()) { + builder.put("index.mapping.single_type", false); + } + + return builder.build(); + } + + protected boolean legacy() { + return false; + } + + protected IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, Object... fields) { + Map source = new HashMap<>(); + for (int i = 0; i < fields.length; i += 2) { + source.put((String) fields[i], fields[i + 1]); + } + return createIndexRequest(index, type, id, parentId, source); + } + + protected IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, + XContentBuilder builder) throws IOException { + Map source = XContentHelper.convertToMap(JsonXContent.jsonXContent, builder.string(), false); + return createIndexRequest(index, type, id, parentId, source); + } + + private IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, Map source) { + String name = type; + if (legacy() == false) { + type = "doc"; + } + + IndexRequestBuilder indexRequestBuilder = client().prepareIndex(index, type, id); + if (legacy()) { + if (parentId != null) { + indexRequestBuilder.setParent(parentId); + } + indexRequestBuilder.setSource(source); + } else { + Map joinField = new HashMap<>(); + if (parentId != null) { + joinField.put("name", name); + joinField.put("parent", parentId); + indexRequestBuilder.setRouting(parentId); + } else { + joinField.put("name", name); + } + source.put("join_field", joinField); + indexRequestBuilder.setSource(source); + } + return indexRequestBuilder; + } + +} diff --git a/modules/parent-join/src/test/resources/rest-api-spec/test/11_parent_child.yml b/modules/parent-join/src/test/resources/rest-api-spec/test/11_parent_child.yml index 24f9931be0562..d4f77cbb1fd96 100644 --- a/modules/parent-join/src/test/resources/rest-api-spec/test/11_parent_child.yml +++ b/modules/parent-join/src/test/resources/rest-api-spec/test/11_parent_child.yml @@ -34,11 +34,12 @@ setup: - do: indices.refresh: {} - # TODO: re-add inner hits here - do: search: - body: { "query" : { "has_child" : { "type" : "child", "query" : { "match_all" : {} } } } } + body: { "query" : { "has_child" : { "type" : "child", "query" : { "match_all" : {} }, "inner_hits" : {} } } } - match: { hits.total: 1 } - match: { hits.hits.0._index: "test" } - - match: { hits.hits.0._type: "doc" } - match: { hits.hits.0._id: "1" } + - is_false: hits.hits.0.inner_hits.child.hits.hits.0._index + - match: { hits.hits.0.inner_hits.child.hits.hits.0._id: "2" } + - is_false: hits.hits.0.inner_hits.child.hits.hits.0._nested