Changed inner_hits to work with the new join field type and
at the same time maintaining support for the `_parent` meta field type/ Relates to #20257
This commit is contained in:
parent
14913fdc37
commit
db8aa8e94e
|
@ -38,7 +38,6 @@ import org.elasticsearch.search.SearchHit;
|
|||
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.ArrayList;
|
|||
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.action.support.WriteRequest.RefreshPolicy.IMMEDI
|
|||
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.index.query.QueryBuilders.termQuery;
|
|||
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 class InnerHitsIT extends ESIntegTestCase {
|
|||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -316,6 +316,14 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
|
|||
|
||||
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 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
|
|||
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 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
|
|||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.QueryShardContext;
|
|||
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.Map;
|
|||
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 @@ public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBu
|
|||
|
||||
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 @@ public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBu
|
|||
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 @@ public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBu
|
|||
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<String, InnerHitContextBuilder> 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, InnerHitContextBuilder> 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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,8 +25,7 @@ import org.elasticsearch.action.update.UpdateResponse;
|
|||
import org.elasticsearch.client.Requests;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.join.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.bucket.terms.Terms;
|
|||
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.greaterThan;
|
|||
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<String, Control> categoryToControl = new HashMap<>();
|
||||
|
||||
@Override
|
||||
protected boolean ignoreExternalCluster() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
return Collections.singleton(ParentJoinPlugin.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> 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<String, Object> source = new HashMap<>();
|
||||
for (int i = 0; i < fields.length; i += 2) {
|
||||
source.put((String) fields[i], fields[i + 1]);
|
||||
}
|
||||
Map<String, Object> 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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,11 +29,7 @@ import org.elasticsearch.common.lucene.search.function.CombineFunction;
|
|||
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.MatchQueryBuilder;
|
|||
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.HighlightBuilder.Field;
|
|||
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.containsString;
|
|||
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<Class<? extends Plugin>> nodePlugins() {
|
||||
return Collections.singleton(ParentJoinPlugin.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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, () ->
|
||||
|
|
|
@ -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.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.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.instanceOf;
|
|||
import static org.hamcrest.CoreMatchers.notNullValue;
|
||||
|
||||
public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQueryBuilder> {
|
||||
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 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
|
|||
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 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
|
|||
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 class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
|
|||
}
|
||||
|
||||
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 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
|
|||
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 class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
|
|||
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 class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
|
|||
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 {
|
||||
|
|
|
@ -24,6 +24,7 @@ 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.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.instanceOf;
|
|||
import static org.hamcrest.CoreMatchers.notNullValue;
|
||||
|
||||
public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQueryBuilder> {
|
||||
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 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
|
|||
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 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
|
|||
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 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
|
|||
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 class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
|
|||
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 class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
|
|||
}
|
||||
|
||||
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 class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
|
|||
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 {
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.search.join.ScoreMode;
|
|||
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.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;
|
||||
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.Locale;
|
|||
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.equalTo;
|
|||
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<Class<? extends Plugin>> nodePlugins() {
|
||||
return Arrays.asList(ParentJoinPlugin.class, CustomScriptPlugin.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> transportClientPlugins() {
|
||||
return nodePlugins();
|
||||
}
|
||||
|
||||
public static class CustomScriptPlugin extends MockScriptPlugin {
|
||||
@Override
|
||||
protected Map<String, Function<Map<String, Object>, Object>> pluginScripts() {
|
||||
|
@ -98,21 +83,27 @@ public class InnerHitsIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
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<IndexRequestBuilder> 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 class InnerHitsIT extends ESIntegTestCase {
|
|||
.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 class InnerHitsIT extends ESIntegTestCase {
|
|||
.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 class InnerHitsIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
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<IndexRequestBuilder> requestBuilders = new ArrayList<>();
|
||||
|
||||
|
@ -186,20 +187,18 @@ public class InnerHitsIT extends ESIntegTestCase {
|
|||
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 class InnerHitsIT extends ESIntegTestCase {
|
|||
.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 class InnerHitsIT extends ESIntegTestCase {
|
|||
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 class InnerHitsIT extends ESIntegTestCase {
|
|||
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 class InnerHitsIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
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<IndexRequestBuilder> 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 class InnerHitsIT extends ESIntegTestCase {
|
|||
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<IndexRequestBuilder> 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 class InnerHitsIT extends ESIntegTestCase {
|
|||
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 class InnerHitsIT extends ESIntegTestCase {
|
|||
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<IndexRequestBuilder> 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 class InnerHitsIT extends ESIntegTestCase {
|
|||
.setName("earls")
|
||||
.setSize(4))
|
||||
)
|
||||
)
|
||||
.get();
|
||||
).get();
|
||||
assertHitCount(response, 1);
|
||||
assertThat(response.getHits().getAt(0).getId(), equalTo("duke"));
|
||||
|
||||
|
@ -445,15 +447,19 @@ public class InnerHitsIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
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<IndexRequestBuilder> 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 class InnerHitsIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
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<IndexRequestBuilder> 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 class InnerHitsIT extends ESIntegTestCase {
|
|||
.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 class InnerHitsIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.elasticsearch.action.bulk.BulkResponse;
|
|||
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 @@ public class LegacyChildQuerySearchIT extends ChildQuerySearchIT {
|
|||
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")
|
||||
|
|
|
@ -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<HasChildQueryBuilder> {
|
||||
protected static final String PARENT_TYPE = "parent";
|
||||
protected static final String CHILD_TYPE = "child";
|
||||
|
||||
private static String similarity;
|
||||
|
||||
boolean requiresRewrite = false;
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> 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<String, InnerHitContextBuilder> 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));
|
||||
}
|
||||
}
|
|
@ -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<HasParentQueryBuilder> {
|
||||
protected static final String PARENT_TYPE = "parent";
|
||||
protected static final String CHILD_TYPE = "child";
|
||||
|
||||
boolean requiresRewrite = false;
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> 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<String, InnerHitContextBuilder> 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));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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<Class<? extends Plugin>> nodePlugins() {
|
||||
return Collections.singleton(ParentJoinPlugin.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue