summaryrefslogtreecommitdiff
path: root/modules/parent-join/src/main/java/org
diff options
context:
space:
mode:
authorJim Ferenczi <jim.ferenczi@elastic.co>2017-05-19 17:11:23 +0200
committerGitHub <noreply@github.com>2017-05-19 17:11:23 +0200
commitd241c4898e5d5ab87284cc3b351989d26947b552 (patch)
tree2aefac4d68554a2a4d0de3d26f8e79414858f14f /modules/parent-join/src/main/java/org
parentb18df27d74664ac3254b8395b67d097b487fd7ce (diff)
Removes parent child fielddata specialization (#24737)
This change removes the field data specialization needed for the parent field and replaces it with a simple DocValuesIndexFieldData. The underlying global ordinals are retrieved via a new function called IndexOrdinalsFieldData#getOrdinalMap. The children aggregation is also modified to use a simple WithOrdinals value source rather than the deleted WithOrdinals.Parent. Relates #20257
Diffstat (limited to 'modules/parent-join/src/main/java/org')
-rw-r--r--modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java33
-rw-r--r--modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java22
-rw-r--r--modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java23
-rw-r--r--modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java27
-rw-r--r--modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java23
5 files changed, 63 insertions, 65 deletions
diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java
index d04b1f0a66..35dc4eacbf 100644
--- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java
+++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java
@@ -25,15 +25,16 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
+import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
import org.elasticsearch.index.mapper.DocumentMapper;
+import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.FieldContext;
import org.elasticsearch.search.aggregations.support.ValueType;
-import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
+import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.WithOrdinals;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
@@ -43,10 +44,11 @@ import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
-public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<ParentChild, ChildrenAggregationBuilder> {
+public class ChildrenAggregationBuilder
+ extends ValuesSourceAggregationBuilder<WithOrdinals, ChildrenAggregationBuilder> {
+
public static final String NAME = "children";
- private String parentType;
private final String childType;
private Query parentFilter;
private Query childFilter;
@@ -79,15 +81,17 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
}
@Override
- protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(SearchContext context,
- ValuesSourceConfig<ParentChild> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
- return new ChildrenAggregatorFactory(name, config, parentType, childFilter, parentFilter, context, parent,
+ protected ValuesSourceAggregatorFactory<WithOrdinals, ?> innerBuild(SearchContext context,
+ ValuesSourceConfig<WithOrdinals> config,
+ AggregatorFactory<?> parent,
+ Builder subFactoriesBuilder) throws IOException {
+ return new ChildrenAggregatorFactory(name, config, childFilter, parentFilter, context, parent,
subFactoriesBuilder, metaData);
}
@Override
- protected ValuesSourceConfig<ParentChild> resolveConfig(SearchContext context) {
- ValuesSourceConfig<ParentChild> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
+ protected ValuesSourceConfig<WithOrdinals> resolveConfig(SearchContext context) {
+ ValuesSourceConfig<WithOrdinals> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
DocumentMapper childDocMapper = context.mapperService().documentMapper(childType);
if (childDocMapper != null) {
@@ -95,15 +99,15 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
if (!parentFieldMapper.active()) {
throw new IllegalArgumentException("[children] no [_parent] field not configured that points to a parent type");
}
- parentType = parentFieldMapper.type();
+ String parentType = parentFieldMapper.type();
DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType);
if (parentDocMapper != null) {
parentFilter = parentDocMapper.typeFilter(context.getQueryShardContext());
childFilter = childDocMapper.typeFilter(context.getQueryShardContext());
- ParentChildIndexFieldData parentChildIndexFieldData = context.fieldData()
- .getForField(parentFieldMapper.fieldType());
- config.fieldContext(new FieldContext(parentFieldMapper.fieldType().name(), parentChildIndexFieldData,
- parentFieldMapper.fieldType()));
+ MappedFieldType parentFieldType = parentDocMapper.parentFieldMapper().getParentJoinFieldType();
+ final SortedSetDVOrdinalsIndexFieldData fieldData = context.fieldData().getForField(parentFieldType);
+ config.fieldContext(new FieldContext(parentFieldType.name(), fieldData,
+ parentFieldType));
} else {
config.unmapped(true);
}
@@ -145,7 +149,6 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
"Missing [child_type] field for children aggregation [" + aggregationName + "]");
}
-
return new ChildrenAggregationBuilder(aggregationName, childType);
}
diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java
index 800be74ba6..9c38fa2eae 100644
--- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java
+++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java
@@ -26,8 +26,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
-import org.elasticsearch.search.aggregations.support.ValuesSource;
-import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
+import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.WithOrdinals;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
@@ -37,17 +36,15 @@ import java.util.List;
import java.util.Map;
public class ChildrenAggregatorFactory
- extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild, ChildrenAggregatorFactory> {
+ extends ValuesSourceAggregatorFactory<WithOrdinals, ChildrenAggregatorFactory> {
- private final String parentType;
private final Query parentFilter;
private final Query childFilter;
- public ChildrenAggregatorFactory(String name, ValuesSourceConfig<ParentChild> config, String parentType, Query childFilter,
- Query parentFilter, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
- Map<String, Object> metaData) throws IOException {
+ public ChildrenAggregatorFactory(String name, ValuesSourceConfig<WithOrdinals> config,
+ Query childFilter, Query parentFilter, SearchContext context, AggregatorFactory<?> parent,
+ AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
- this.parentType = parentType;
this.childFilter = childFilter;
this.parentFilter = parentFilter;
}
@@ -66,12 +63,11 @@ public class ChildrenAggregatorFactory
}
@Override
- protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, Aggregator parent,
+ protected Aggregator doCreateInternal(WithOrdinals valuesSource, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
- long maxOrd = valuesSource.globalMaxOrd(context.searcher(), parentType);
- return new ParentToChildrenAggregator(name, factories, context, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd,
- pipelineAggregators, metaData);
+ long maxOrd = valuesSource.globalMaxOrd(context.searcher());
+ return new ParentToChildrenAggregator(name, factories, context, parent, childFilter,
+ parentFilter, valuesSource, maxOrd, pipelineAggregators, metaData);
}
-
}
diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java
index c1ffb097ab..93ba1b98da 100644
--- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java
+++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java
@@ -20,7 +20,7 @@ package org.elasticsearch.join.aggregations;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.ConstantScoreScorer;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Query;
@@ -52,10 +52,9 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
static final ParseField TYPE_FIELD = new ParseField("type");
- private final String parentType;
private final Weight childFilter;
private final Weight parentFilter;
- private final ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource;
+ private final ValuesSource.Bytes.WithOrdinals valuesSource;
// Maybe use PagedGrowableWriter? This will be less wasteful than LongArray,
// but then we don't have the reuse feature of BigArrays.
@@ -72,12 +71,11 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
private boolean multipleBucketsPerParentOrd = false;
public ParentToChildrenAggregator(String name, AggregatorFactories factories,
- SearchContext context, Aggregator parent, String parentType, Query childFilter,
- Query parentFilter, ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource,
+ SearchContext context, Aggregator parent, Query childFilter,
+ Query parentFilter, ValuesSource.Bytes.WithOrdinals valuesSource,
long maxOrd, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
super(name, factories, context, parent, pipelineAggregators, metaData);
- this.parentType = parentType;
// these two filters are cached in the parser
this.childFilter = context.searcher().createNormalizedWeight(childFilter, false);
this.parentFilter = context.searcher().createNormalizedWeight(parentFilter, false);
@@ -105,9 +103,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
}
-
- final SortedDocValues globalOrdinals = valuesSource.globalOrdinalsValues(parentType, ctx);
- assert globalOrdinals != null;
+ final SortedSetDocValues globalOrdinals = valuesSource.globalOrdinalsValues(ctx);
Scorer parentScorer = parentFilter.scorer(ctx);
final Bits parentDocs = Lucene.asSequentialAccessBits(ctx.reader().maxDoc(), parentScorer);
return new LeafBucketCollector() {
@@ -115,7 +111,8 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
@Override
public void collect(int docId, long bucket) throws IOException {
if (parentDocs.get(docId) && globalOrdinals.advanceExact(docId)) {
- long globalOrdinal = globalOrdinals.ordValue();
+ long globalOrdinal = globalOrdinals.nextOrd();
+ assert globalOrdinals.nextOrd() == SortedSetDocValues.NO_MORE_ORDS;
if (globalOrdinal != -1) {
if (parentOrdToBuckets.get(globalOrdinal) == -1) {
parentOrdToBuckets.set(globalOrdinal, bucket);
@@ -147,9 +144,8 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
DocIdSetIterator childDocsIter = childDocsScorer.iterator();
final LeafBucketCollector sub = collectableSubAggregators.getLeafCollector(ctx);
- final SortedDocValues globalOrdinals = valuesSource.globalOrdinalsValues(parentType,
- ctx);
+ final SortedSetDocValues globalOrdinals = valuesSource.globalOrdinalsValues(ctx);
// Set the scorer, since we now replay only the child docIds
sub.setScorer(new ConstantScoreScorer(null, 1f, childDocsIter));
@@ -161,7 +157,8 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
continue;
}
if (globalOrdinals.advanceExact(docId)) {
- long globalOrdinal = globalOrdinals.ordValue();
+ long globalOrdinal = globalOrdinals.nextOrd();
+ assert globalOrdinals.nextOrd() == SortedSetDocValues.NO_MORE_ORDS;
long bucketOrd = parentOrdToBuckets.get(globalOrdinal);
if (bucketOrd != -1) {
collectBucket(sub, docId, bucketOrd);
diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java
index 494c5e498e..95d000e3cc 100644
--- a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java
+++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java
@@ -34,9 +34,10 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.fielddata.IndexParentChildFieldData;
-import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
+import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
+import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
import org.elasticsearch.index.mapper.DocumentMapper;
+import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.query.AbstractQueryBuilder;
import org.elasticsearch.index.query.InnerHitBuilder;
@@ -48,7 +49,6 @@ import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import java.io.IOException;
-import java.util.Locale;
import java.util.Map;
import java.util.Objects;
@@ -324,9 +324,10 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
// wrap the query with type query
innerQuery = Queries.filtered(innerQuery, childDocMapper.typeFilter(context));
- final ParentChildIndexFieldData parentChildIndexFieldData = context.getForField(parentFieldMapper.fieldType());
+ final MappedFieldType parentFieldType = parentDocMapper.parentFieldMapper().getParentJoinFieldType();
+ final SortedSetDVOrdinalsIndexFieldData fieldData = context.getForField(parentFieldType);
return new LateParsingQuery(parentDocMapper.typeFilter(context), innerQuery, minChildren(), maxChildren(),
- parentType, scoreMode, parentChildIndexFieldData, context.getSearchSimilarity());
+ parentType, scoreMode, fieldData, context.getSearchSimilarity());
}
/**
@@ -347,19 +348,19 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
private final int maxChildren;
private final String parentType;
private final ScoreMode scoreMode;
- private final ParentChildIndexFieldData parentChildIndexFieldData;
+ private final SortedSetDVOrdinalsIndexFieldData fieldDataJoin;
private final Similarity similarity;
LateParsingQuery(Query toQuery, Query innerQuery, int minChildren, int maxChildren,
- String parentType, ScoreMode scoreMode, ParentChildIndexFieldData parentChildIndexFieldData,
- Similarity similarity) {
+ String parentType, ScoreMode scoreMode,
+ SortedSetDVOrdinalsIndexFieldData fieldData, Similarity similarity) {
this.toQuery = toQuery;
this.innerQuery = innerQuery;
this.minChildren = minChildren;
this.maxChildren = maxChildren;
this.parentType = parentType;
this.scoreMode = scoreMode;
- this.parentChildIndexFieldData = parentChildIndexFieldData;
+ this.fieldDataJoin = fieldData;
this.similarity = similarity;
}
@@ -374,10 +375,10 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
IndexSearcher indexSearcher = new IndexSearcher(reader);
indexSearcher.setQueryCache(null);
indexSearcher.setSimilarity(similarity);
- IndexParentChildFieldData indexParentChildFieldData = parentChildIndexFieldData.loadGlobal((DirectoryReader) reader);
- MultiDocValues.OrdinalMap ordinalMap = ParentChildIndexFieldData.getOrdinalMap(indexParentChildFieldData, parentType);
+ IndexOrdinalsFieldData indexParentChildFieldData = fieldDataJoin.loadGlobal((DirectoryReader) reader);
+ MultiDocValues.OrdinalMap ordinalMap = indexParentChildFieldData.getOrdinalMap();
return JoinUtil.createJoinQuery(joinField, innerQuery, toQuery, indexSearcher, scoreMode,
- ordinalMap, minChildren, maxChildren);
+ ordinalMap, minChildren, maxChildren);
} else {
if (reader.leaves().isEmpty() && reader.numDocs() == 0) {
// asserting reader passes down a MultiReader during rewrite which makes this
@@ -387,7 +388,7 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
return new MatchNoDocsQuery();
}
throw new IllegalStateException("can't load global ordinals for reader of type: " +
- reader.getClass() + " must be a DirectoryReader");
+ reader.getClass() + " must be a DirectoryReader");
}
}
diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java
index ca0bfd623d..b216e886a5 100644
--- a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java
+++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java
@@ -30,8 +30,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
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.ParentChildIndexFieldData;
+import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
import org.elasticsearch.index.mapper.DocumentMapper;
+import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.query.AbstractQueryBuilder;
import org.elasticsearch.index.query.InnerHitBuilder;
@@ -176,15 +177,12 @@ public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBu
}
Set<String> childTypes = new HashSet<>();
- ParentChildIndexFieldData parentChildIndexFieldData = null;
for (DocumentMapper documentMapper : context.getMapperService().docMappers(false)) {
ParentFieldMapper parentFieldMapper = documentMapper.parentFieldMapper();
if (parentFieldMapper.active() && type.equals(parentFieldMapper.type())) {
childTypes.add(documentMapper.type());
- parentChildIndexFieldData = context.getForField(parentFieldMapper.fieldType());
}
}
-
if (childTypes.isEmpty()) {
throw new QueryShardException(context, "[" + NAME + "] no child types found for type [" + type + "]");
}
@@ -204,14 +202,17 @@ public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBu
// wrap the query with type query
innerQuery = Queries.filtered(innerQuery, parentDocMapper.typeFilter(context));
+
+ final MappedFieldType parentType = parentDocMapper.parentFieldMapper().getParentJoinFieldType();
+ final SortedSetDVOrdinalsIndexFieldData fieldData = context.getForField(parentType);
return new HasChildQueryBuilder.LateParsingQuery(childrenQuery,
- innerQuery,
- HasChildQueryBuilder.DEFAULT_MIN_CHILDREN,
- HasChildQueryBuilder.DEFAULT_MAX_CHILDREN,
- type,
- score ? ScoreMode.Max : ScoreMode.None,
- parentChildIndexFieldData,
- context.getSearchSimilarity());
+ innerQuery,
+ HasChildQueryBuilder.DEFAULT_MIN_CHILDREN,
+ HasChildQueryBuilder.DEFAULT_MAX_CHILDREN,
+ type,
+ score ? ScoreMode.Max : ScoreMode.None,
+ fieldData,
+ context.getSearchSimilarity());
}
@Override