Parent/child: refactored _parent field mapper and parent/child queries

* Cut the `has_child` and `has_parent` queries over to use Lucene's query time global ordinal join. The main benefit of this change is that parent/child queries can now efficiently execute if parent/child queries are wrapped in a bigger boolean query. If the rest of the query only hit a few documents both has_child and has_parent queries don't need to evaluate all parent or child documents any more.
* Cut the `_parent` field over to use doc values. This significantly reduces the on heap memory footprint of parent/child, because the parent id values are never loaded into memory.

Breaking changes:
* The `type` option on the `_parent` field can only point to a parent type that doesn't exist yet, so this means that an existing type/mapping can't become a parent type any longer.
* The `has_child` and `has_parent` queries can no longer be use in alias filters.

All these changes, improvements and breaks in compatibility only apply for indices created with ES version 2.0 or higher. For indices creates with ES <= 2.0 the older implementation is used.

It is highly recommended to re-index all your indices with parent and child documents to benefit from all the improvements that come with this refactoring. The easiest way to achieve this is by using the scan and bulk apis using a simple script.

Closes #6107
Closes #8134
This commit is contained in:
Martijn van Groningen 2015-03-25 10:42:49 +01:00
parent bd381b86ca
commit 1cfb6a79f1
25 changed files with 944 additions and 435 deletions

View File

@ -1,6 +1,9 @@
[[mapping-parent-field]]
=== `_parent`
TIP: It is highly recommend to reindex all indices with `_parent` field created before version 2.x.
The reason for this is to gain from all the optimizations added with the 2.0 release.
The parent field mapping is defined on a child mapping, and points to
the parent type this child relates to. For example, in case of a `blog`
type and a `blog_tag` type child document, the mapping for `blog_tag`
@ -20,8 +23,34 @@ should be:
The mapping is automatically stored and indexed (meaning it can be
searched on using the `_parent` field notation).
==== Field data loading
==== Limitations
Contrary to other fields the fielddata loading is not `lazy`, but `eager`. The reason for this is that when this
field has been enabled it is going to be used in parent/child queries, which heavily relies on field data to perform
efficiently. This can already be observed during indexing after refresh either automatically or manually has been executed.
The `_parent.type` setting can only point to a type that doesn't exist yet.
This means that a type can't become a parent type after is has been created.
The `parent.type` setting can't point to itself. This means self referential
parent/child isn't supported.
Parent/child queries (`has_child` & `has_parent`) can't be used in index aliases.
==== Global ordinals
Parent-child uses <<index-modules-fielddata-global-ordinals,global ordinals>> to speed up joins and global ordinals need to be rebuilt after any change to a shard.
The more parent id values are stored in a shard, the longer it takes to rebuild global ordinals for the `_parent` field.
Global ordinals, by default, are built lazily: the first parent-child query or aggregation after a refresh will trigger building of global ordinals.
This can introduce a significant latency spike for your users. You can use <<index-modules-fielddata-fielddata-loading,eager_global_ordinals>> to shift the cost of building global ordinals
from query time to refresh time, by mapping the _parent field as follows:
==== Memory usage
The only on heap memory used by parent/child is the global ordinals for the `_parent` field.
How much memory is used for the global ordianls for the `_parent` field in the fielddata cache
can be checked via the <<indices-stats,indices stats>> or <<cluster-nodes-stats,nodes stats>>
APIS, eg:
[source,js]
--------------------------------------------------
curl -XGET "http://localhost:9200/_stats/fielddata?pretty&human&fielddata_fields=_parent"
--------------------------------------------------

View File

@ -626,3 +626,19 @@ anymore, it will only highlight fields that were queried.
The `match` query with type set to `match_phrase_prefix` is not supported by the
postings highlighter. No highlighted snippets will be returned.
[float]
=== Parent/child
Parent/child has been rewritten completely to reduce memory usage and to execute
`has_child` and `has_parent` queries faster and more efficient. The `_parent` field
uses doc values by default. The refactored and improved implementation is only active
for indices created on or after version 2.0.
In order to benefit for all performance and memory improvements we recommend to reindex all
indices that have the `_parent` field created before was upgraded to 2.0.
The following breaks in backwards compatability have been made on indices with the `_parent` field
created on or after clusters with version 2.0:
* The `type` option on the `_parent` field can only point to a parent type that doesn't exist yet,
so this means that an existing type/mapping can no longer become a parent type.
* The `has_child` and `has_parent` queries can no longer be use in alias filters.

View File

@ -72,21 +72,3 @@ a match:
The `min_children` and `max_children` parameters can be combined with
the `score_mode` parameter.
[float]
=== Memory Considerations
In order to support parent-child joins, all of the (string) parent IDs
must be resident in memory (in the <<index-modules-fielddata,field data cache>>.
Additionally, every child document is mapped to its parent using a long
value (approximately). It is advisable to keep the string parent ID short
in order to reduce memory usage.
You can check how much memory is being used by the `_parent` field in the fielddata cache
using the <<indices-stats,indices stats>> or <<cluster-nodes-stats,nodes stats>>
APIS, eg:
[source,js]
--------------------------------------------------
curl -XGET "http://localhost:9200/_stats/fielddata?pretty&human&fielddata_fields=_parent"
--------------------------------------------------

View File

@ -47,23 +47,3 @@ matching parent document. The score type can be specified with the
}
}
--------------------------------------------------
[float]
=== Memory Considerations
In order to support parent-child joins, all of the (string) parent IDs
must be resident in memory (in the <<index-modules-fielddata,field data cache>>.
Additionally, every child document is mapped to its parent using a long
value (approximately). It is advisable to keep the string parent ID short
in order to reduce memory usage.
You can check how much memory is being used by the `_parent` field in the fielddata cache
using the <<indices-stats,indices stats>> or <<cluster-nodes-stats,nodes stats>>
APIS, eg:
[source,js]
--------------------------------------------------
curl -XGET "http://localhost:9200/_stats/fielddata?pretty&human&fielddata_fields=_parent"
--------------------------------------------------

View File

@ -22,6 +22,7 @@ package org.elasticsearch.cluster.metadata;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingClusterStateUpdateRequest;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
@ -386,8 +387,22 @@ public class MetaDataMappingService extends AbstractComponent {
if (mergeResult.hasConflicts()) {
throw new MergeMappingException(mergeResult.buildConflicts());
}
} else {
// TODO: can we find a better place for this validation?
// The reason this validation is here is that the mapper service doesn't learn about
// new types all at once , which can create a false error.
// For example in MapperService we can't distinguish between a create index api call
// and a put mapping api call, so we don't which type did exist before.
// Also the order of the mappings may be backwards.
if (Version.indexCreated(indexService.getIndexSettings()).onOrAfter(Version.V_2_0_0) && newMapper.parentFieldMapper().active()) {
if (indexService.mapperService().types().contains(newMapper.parentFieldMapper().type())) {
throw new IllegalArgumentException("can't add a _parent field that points to an already existing type");
}
}
}
}
newMappers.put(index, newMapper);
if (existingMapper != null) {

View File

@ -53,6 +53,10 @@ public abstract class AbstractIndexComponent implements IndexComponent {
return this.index;
}
public Settings indexSettings() {
return indexSettings;
}
public String nodeName() {
return indexSettings.get("name", "");
}

View File

@ -21,16 +21,10 @@ package org.elasticsearch.index.fielddata.plain;
import com.carrotsearch.hppc.ObjectObjectHashMap;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedSet;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.*;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
@ -39,6 +33,7 @@ import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -47,14 +42,8 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.AtomicParentChildFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexParentChildFieldData;
import org.elasticsearch.index.fielddata.RamAccountingTermsEnum;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
@ -70,17 +59,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeSet;
import java.util.*;
import java.util.concurrent.TimeUnit;
/**
@ -89,7 +68,7 @@ import java.util.concurrent.TimeUnit;
*/
public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicParentChildFieldData> implements IndexParentChildFieldData, DocumentTypeListener {
private final NavigableSet<BytesRef> parentTypes;
private final NavigableSet<String> parentTypes;
private final CircuitBreakerService breakerService;
// If child type (a type with _parent field) is added or removed, we want to make sure modifications don't happen
@ -100,7 +79,7 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
FieldDataType fieldDataType, IndexFieldDataCache cache, MapperService mapperService,
CircuitBreakerService breakerService) {
super(index, indexSettings, fieldNames, fieldDataType, cache);
parentTypes = new TreeSet<>(BytesRef.getUTF8SortedAsUnicodeComparator());
parentTypes = new TreeSet<>();
this.breakerService = breakerService;
for (DocumentMapper documentMapper : mapperService.docMappers(false)) {
beforeCreate(documentMapper);
@ -114,15 +93,60 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
}
@Override
public ParentChildAtomicFieldData loadDirect(LeafReaderContext context) throws Exception {
public AtomicParentChildFieldData load(LeafReaderContext context) {
if (Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0)) {
final LeafReader reader = context.reader();
final NavigableSet<String> parentTypes;
synchronized (lock) {
parentTypes = ImmutableSortedSet.copyOf(this.parentTypes);
}
return new AbstractAtomicParentChildFieldData() {
public Set<String> types() {
return parentTypes;
}
@Override
public SortedDocValues getOrdinalsValues(String type) {
try {
return DocValues.getSorted(reader, ParentFieldMapper.joinField(type));
} catch (IOException e) {
throw new IllegalStateException("cannot load join doc values field for type [" + type + "]", e);
}
}
@Override
public long ramBytesUsed() {
// unknown
return 0;
}
@Override
public Collection<Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public void close() throws ElasticsearchException {
}
};
} else {
return super.load(context);
}
}
@Override
public AbstractAtomicParentChildFieldData loadDirect(LeafReaderContext context) throws Exception {
LeafReader reader = context.reader();
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat(
"acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO
);
final NavigableSet<BytesRef> parentTypes;
final NavigableSet<BytesRef> parentTypes = new TreeSet<>();
synchronized (lock) {
parentTypes = ImmutableSortedSet.copyOf(BytesRef.getUTF8SortedAsUnicodeComparator(), this.parentTypes);
for (String parentType : this.parentTypes) {
parentTypes.add(new BytesRef(parentType));
}
}
boolean success = false;
ParentChildAtomicFieldData data = null;
@ -192,7 +216,7 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
if (parentFieldMapper.active()) {
// A _parent field can never be added to an existing mapping, so a _parent field either exists on
// a new created or doesn't exists. This is why we can update the known parent types via DocumentTypeListener
if (parentTypes.add(new BytesRef(parentFieldMapper.type()))) {
if (parentTypes.add(parentFieldMapper.type())) {
clear();
}
}
@ -320,11 +344,9 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
@Override
public IndexParentChildFieldData localGlobalDirect(IndexReader indexReader) throws Exception {
final long startTime = System.nanoTime();
final Set<String> parentTypes = new HashSet<>();
final Set<String> parentTypes;
synchronized (lock) {
for (BytesRef type : this.parentTypes) {
parentTypes.add(type.utf8ToString());
}
parentTypes = ImmutableSet.copyOf(this.parentTypes);
}
long ramBytesUsed = 0;
@ -352,7 +374,7 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
);
}
return new GlobalFieldData(indexReader, fielddata, ramBytesUsed);
return new GlobalFieldData(indexReader, fielddata, ramBytesUsed, perType);
}
private static class GlobalAtomicFieldData extends AbstractAtomicParentChildFieldData {
@ -436,16 +458,18 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
}
private class GlobalFieldData implements IndexParentChildFieldData, Accountable {
public class GlobalFieldData implements IndexParentChildFieldData, Accountable {
private final AtomicParentChildFieldData[] fielddata;
private final IndexReader reader;
private final long ramBytesUsed;
private final Map<String, OrdinalMapAndAtomicFieldData> ordinalMapPerType;
GlobalFieldData(IndexReader reader, AtomicParentChildFieldData[] fielddata, long ramBytesUsed) {
GlobalFieldData(IndexReader reader, AtomicParentChildFieldData[] fielddata, long ramBytesUsed, Map<String, OrdinalMapAndAtomicFieldData> ordinalMapPerType) {
this.reader = reader;
this.ramBytesUsed = ramBytesUsed;
this.fielddata = fielddata;
this.ordinalMapPerType = ordinalMapPerType;
}
@Override
@ -514,4 +538,20 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
}
/**
* Returns the global ordinal map for the specified type
*/
// TODO: OrdinalMap isn't expose in the field data framework, because it is an implementation detail.
// However the JoinUtil works directly with OrdinalMap, so this is a hack to get access to OrdinalMap
// I don't think we should expose OrdinalMap in IndexFieldData, because only parent/child relies on it and for the
// rest of the code OrdinalMap is an implementation detail, but maybe we can expose it in IndexParentChildFieldData interface?
public static MultiDocValues.OrdinalMap getOrdinalMap(IndexParentChildFieldData indexParentChildFieldData, String type) {
if (indexParentChildFieldData instanceof ParentChildIndexFieldData.GlobalFieldData) {
return ((GlobalFieldData) indexParentChildFieldData).ordinalMapPerType.get(type).ordMap;
} else {
// one segment, local ordinals are global
return null;
}
}
}

View File

@ -402,6 +402,10 @@ public class DocumentMapper implements ToXContent {
mapperService.addFieldMappers(fieldMappers);
}
public boolean isParent(String type) {
return mapperService.getParentTypes().contains(type);
}
private void addObjectMappers(Collection<ObjectMapper> objectMappers) {
assert mappingLock.isWriteLockedByCurrentThread();
MapBuilder<String, ObjectMapper> builder = MapBuilder.newMapBuilder(this.objectMappers);

View File

@ -21,10 +21,7 @@ package org.elasticsearch.index.mapper;
import com.carrotsearch.hppc.ObjectHashSet;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.*;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
@ -40,6 +37,7 @@ import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
@ -117,6 +115,8 @@ public class MapperService extends AbstractIndexComponent {
private volatile ImmutableMap<String, FieldMapper> unmappedFieldMappers = ImmutableMap.of();
private volatile ImmutableSet<String> parentTypes = ImmutableSet.of();
@Inject
public MapperService(Index index, @IndexSettings Settings indexSettings, AnalysisService analysisService, IndexFieldDataService fieldDataService,
SimilarityLookupService similarityLookupService,
@ -250,6 +250,9 @@ public class MapperService extends AbstractIndexComponent {
if (mapper.type().contains(",")) {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] should not include ',' in it");
}
if (Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0) && mapper.type().equals(mapper.parentFieldMapper().type())) {
throw new IllegalArgumentException("The [_parent.type] option can't point to the same type");
}
if (mapper.type().contains(".") && !PercolatorService.TYPE_NAME.equals(mapper.type())) {
logger.warn("Type [{}] contains a '.', it is recommended not to include it within a type name", mapper.type());
}
@ -285,6 +288,12 @@ public class MapperService extends AbstractIndexComponent {
typeListener.beforeCreate(mapper);
}
mappers = newMapBuilder(mappers).put(mapper.type(), mapper).map();
if (mapper.parentFieldMapper().active()) {
ImmutableSet.Builder<String> parentTypesCopy = ImmutableSet.builder();
parentTypesCopy.addAll(parentTypes);
parentTypesCopy.add(mapper.parentFieldMapper().type());
parentTypes = parentTypesCopy.build();
}
assert assertSerialization(mapper);
return mapper;
}
@ -645,6 +654,10 @@ public class MapperService extends AbstractIndexComponent {
return null;
}
public ImmutableSet<String> getParentTypes() {
return parentTypes;
}
/**
* @return Whether a field is a metadata field.
*/

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.mapper.internal;
import com.google.common.base.Objects;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.Query;
@ -33,9 +34,11 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
@ -55,7 +58,6 @@ import java.util.Map;
import static org.elasticsearch.common.settings.Settings.builder;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeMapValue;
import static org.elasticsearch.index.mapper.MapperBuilders.parent;
/**
*
@ -107,7 +109,7 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
@Override
public ParentFieldMapper build(BuilderContext context) {
if (type == null) {
throw new MapperParsingException("Parent mapping must contain the parent type");
throw new MapperParsingException("[_parent] field mapping must contain the [type] option");
}
return new ParentFieldMapper(name, indexName, type, fieldDataSettings, context.indexSettings());
}
@ -116,7 +118,7 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
ParentFieldMapper.Builder builder = parent();
ParentFieldMapper.Builder builder = MapperBuilders.parent();
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String fieldName = Strings.toUnderscoreCase(entry.getKey());
@ -145,8 +147,9 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
private final BytesRef typeAsBytes;
protected ParentFieldMapper(String name, String indexName, String type, @Nullable Settings fieldDataSettings, Settings indexSettings) {
super(new Names(name, indexName, indexName, name), Defaults.BOOST, new FieldType(Defaults.FIELD_TYPE), false,
Lucene.KEYWORD_ANALYZER, Lucene.KEYWORD_ANALYZER, null, null, fieldDataSettings, indexSettings);
super(new Names(name, indexName, indexName, name), Defaults.BOOST, new FieldType(Defaults.FIELD_TYPE),
Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0), Lucene.KEYWORD_ANALYZER, Lucene.KEYWORD_ANALYZER,
null, null, fieldDataSettings, indexSettings);
this.type = type;
this.typeAsBytes = type == null ? null : new BytesRef(type);
}
@ -181,6 +184,11 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
@Override
protected void parseCreateField(ParseContext context, List<Field> fields) throws IOException {
boolean parent = context.docMapper().isParent(context.type());
if (parent && hasDocValues()) {
fields.add(createJoinField(context.type(), context.id()));
}
if (!active()) {
return;
}
@ -190,6 +198,9 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
String parentId = context.parser().text();
context.sourceToParse().parent(parentId);
fields.add(new Field(names.indexName(), Uid.createUid(context.stringBuilder(), type, parentId), fieldType));
if (hasDocValues()) {
fields.add(createJoinField(type, parentId));
}
} else {
// otherwise, we are running it post processing of the xcontent
String parsedParentId = context.doc().get(Defaults.NAME);
@ -201,6 +212,9 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
}
// we did not add it in the parsing phase, add it now
fields.add(new Field(names.indexName(), Uid.createUid(context.stringBuilder(), type, parentId), fieldType));
if (hasDocValues()) {
fields.add(createJoinField(type, parentId));
}
} else if (parentId != null && !parsedParentId.equals(Uid.createUid(context.stringBuilder(), type, parentId))) {
throw new MapperParsingException("Parent id mismatch, document value is [" + Uid.createUid(parsedParentId).id() + "], while external value is [" + parentId + "]");
}
@ -209,6 +223,15 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
// we have parent mapping, yet no value was set, ignore it...
}
private SortedDocValuesField createJoinField(String parentType, String id) {
String joinField = joinField(parentType);
return new SortedDocValuesField(joinField, new BytesRef(id));
}
public static String joinField(String parentType) {
return ParentFieldMapper.NAME + "#" + parentType;
}
@Override
public Uid value(Object value) {
if (value == null) {
@ -303,7 +326,9 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
boolean includeDefaults = params.paramAsBoolean("include_defaults", false);
builder.startObject(CONTENT_TYPE);
if (type != null) {
builder.field("type", type);
}
if (customFieldDataSettings != null) {
builder.field("fielddata", (Map) customFieldDataSettings.getAsMap());
} else if (includeDefaults) {
@ -316,7 +341,7 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
ParentFieldMapper other = (ParentFieldMapper) mergeWith;
if (!Objects.equal(type, other.type)) {
if (Objects.equal(type, other.type) == false) {
mergeResult.addConflict("The _parent field's type option can't be changed: [" + type + "]->[" + other.type + "]");
}
@ -334,7 +359,7 @@ public class ParentFieldMapper extends AbstractFieldMapper implements RootMapper
}
/**
* @return Whether the _parent field is actually used.
* @return Whether the _parent field is actually configured.
*/
public boolean active() {
return type != null;

View File

@ -19,16 +19,23 @@
package org.elasticsearch.index.query;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.MultiDocValues;
import org.apache.lucene.search.*;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.join.BitDocIdSetFilter;
import org.elasticsearch.common.ParseField;
import org.apache.lucene.search.join.JoinUtil;
import org.apache.lucene.search.join.ScoreMode;
import org.elasticsearch.Version;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexParentChildFieldData;
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
@ -38,6 +45,7 @@ import org.elasticsearch.index.search.child.ChildrenConstantScoreQuery;
import org.elasticsearch.index.search.child.ChildrenQuery;
import org.elasticsearch.index.search.child.ScoreType;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SubSearchContext;
import java.io.IOException;
@ -137,8 +145,9 @@ public class HasChildQueryParser implements QueryParser {
if (childDocMapper == null) {
throw new QueryParsingException(parseContext, "[has_child] No mapping for for type [" + childType + "]");
}
if (!childDocMapper.parentFieldMapper().active()) {
throw new QueryParsingException(parseContext, "[has_child] Type [" + childType + "] does not have parent mapping");
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
if (parentFieldMapper.active() == false) {
throw new QueryParsingException(parseContext, "[has_child] _parent field has no parent type configured");
}
if (innerHits != null) {
@ -147,11 +156,6 @@ public class HasChildQueryParser implements QueryParser {
parseContext.addInnerHits(name, parentChildInnerHits);
}
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
if (!parentFieldMapper.active()) {
throw new QueryParsingException(parseContext, "[has_child] _parent field not configured");
}
String parentType = parentFieldMapper.type();
DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType);
if (parentDocMapper == null) {
@ -171,10 +175,13 @@ public class HasChildQueryParser implements QueryParser {
// wrap the query with type query
innerQuery = Queries.filtered(innerQuery, childDocMapper.typeFilter());
Query query;
final Query query;
final ParentChildIndexFieldData parentChildIndexFieldData = parseContext.getForField(parentFieldMapper);
if (parseContext.indexVersionCreated().onOrAfter(Version.V_2_0_0)) {
query = joinUtilHelper(parentType, parentChildIndexFieldData, parentDocMapper.typeFilter(), scoreType, innerQuery, minChildren, maxChildren);
} else {
// TODO: use the query API
Filter parentFilter = new QueryWrapperFilter(parentDocMapper.typeFilter());
ParentChildIndexFieldData parentChildIndexFieldData = parseContext.getForField(parentFieldMapper);
if (minChildren > 1 || maxChildren > 0 || scoreType != ScoreType.NONE) {
query = new ChildrenQuery(parentChildIndexFieldData, parentType, childType, parentFilter, innerQuery, scoreType, minChildren,
maxChildren, shortCircuitParentDocSet, nonNestedDocsFilter);
@ -182,10 +189,51 @@ public class HasChildQueryParser implements QueryParser {
query = new ChildrenConstantScoreQuery(parentChildIndexFieldData, innerQuery, parentType, childType, parentFilter,
shortCircuitParentDocSet, nonNestedDocsFilter);
}
}
if (queryName != null) {
parseContext.addNamedQuery(queryName, query);
}
query.setBoost(boost);
return query;
}
public static Query joinUtilHelper(String parentType, ParentChildIndexFieldData parentChildIndexFieldData, Query toQuery, ScoreType scoreType, Query innerQuery, int minChildren, int maxChildren) throws IOException {
SearchContext searchContext = SearchContext.current();
if (searchContext == null) {
throw new IllegalStateException("Search context is required to be set");
}
String joinField = ParentFieldMapper.joinField(parentType);
ScoreMode scoreMode;
// TODO: move entirely over from ScoreType to org.apache.lucene.join.ScoreMode, when we drop the 1.x parent child code.
switch (scoreType) {
case NONE:
scoreMode = ScoreMode.None;
break;
case MIN:
scoreMode = ScoreMode.Min;
break;
case MAX:
scoreMode = ScoreMode.Max;
break;
case SUM:
scoreMode = ScoreMode.Total;
break;
case AVG:
scoreMode = ScoreMode.Avg;
break;
default:
throw new UnsupportedOperationException("score type [" + scoreType + "] not supported");
}
IndexReader indexReader = searchContext.searcher().getIndexReader();
IndexSearcher indexSearcher = new IndexSearcher(indexReader);
IndexParentChildFieldData indexParentChildFieldData = parentChildIndexFieldData.loadGlobal(indexReader);
MultiDocValues.OrdinalMap ordinalMap = ParentChildIndexFieldData.getOrdinalMap(indexParentChildFieldData, parentType);
// 0 in pre 2.x p/c impl means unbounded
if (maxChildren == 0) {
maxChildren = Integer.MAX_VALUE;
}
return JoinUtil.createJoinQuery(joinField, innerQuery, toQuery, indexSearcher, scoreMode, ordinalMap, minChildren, maxChildren);
}
}

View File

@ -22,6 +22,7 @@ import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.elasticsearch.Version;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
@ -36,6 +37,7 @@ import org.elasticsearch.index.query.support.InnerHitsQueryParserHelper;
import org.elasticsearch.index.query.support.XContentStructure;
import org.elasticsearch.index.search.child.ParentConstantScoreQuery;
import org.elasticsearch.index.search.child.ParentQuery;
import org.elasticsearch.index.search.child.ScoreType;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.SubSearchContext;
@ -43,6 +45,8 @@ import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import static org.elasticsearch.index.query.HasChildQueryParser.joinUtilHelper;
public class HasParentQueryParser implements QueryParser {
public static final String NAME = "has_parent";
@ -142,7 +146,7 @@ public class HasParentQueryParser implements QueryParser {
return query;
}
static Query createParentQuery(Query innerQuery, String parentType, boolean score, QueryParseContext parseContext, Tuple<String, SubSearchContext> innerHits) {
static Query createParentQuery(Query innerQuery, String parentType, boolean score, QueryParseContext parseContext, Tuple<String, SubSearchContext> innerHits) throws IOException {
DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType);
if (parentDocMapper == null) {
throw new QueryParsingException(parseContext, "[has_parent] query configured 'parent_type' [" + parentType
@ -197,11 +201,16 @@ public class HasParentQueryParser implements QueryParser {
// wrap the query with type query
innerQuery = Queries.filtered(innerQuery, parentDocMapper.typeFilter());
Filter childrenFilter = new QueryWrapperFilter(Queries.not(parentFilter));
if (parseContext.indexVersionCreated().onOrAfter(Version.V_2_0_0)) {
ScoreType scoreMode = score ? ScoreType.MAX : ScoreType.NONE;
return joinUtilHelper(parentType, parentChildIndexFieldData, childrenFilter, scoreMode, innerQuery, 0, Integer.MAX_VALUE);
} else {
if (score) {
return new ParentQuery(parentChildIndexFieldData, innerQuery, parentDocMapper.type(), childrenFilter);
} else {
return new ParentConstantScoreQuery(parentChildIndexFieldData, innerQuery, parentDocMapper.type(), childrenFilter);
}
}
}
}

View File

@ -21,6 +21,8 @@ package org.elasticsearch.index.query;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.queryparser.classic.MapperQueryParser;
import org.apache.lucene.queryparser.classic.QueryParserSettings;
@ -79,6 +81,8 @@ public class QueryParseContext {
private final Index index;
private final Version indexVersionCreated;
private final IndexQueryParserService indexQueryParser;
private final Map<String, Query> namedQueries = Maps.newHashMap();
@ -99,6 +103,7 @@ public class QueryParseContext {
public QueryParseContext(Index index, IndexQueryParserService indexQueryParser) {
this.index = index;
this.indexVersionCreated = Version.indexCreated(indexQueryParser.indexSettings());
this.indexQueryParser = indexQueryParser;
}
@ -376,4 +381,8 @@ public class QueryParseContext {
public boolean isDeprecatedSetting(String setting) {
return CACHE.match(setting) || CACHE_KEY.match(setting);
}
public Version indexVersionCreated() {
return indexVersionCreated;
}
}

View File

@ -80,7 +80,7 @@ public class ChildrenParser implements Aggregator.Parser {
if (childDocMapper != null) {
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
if (!parentFieldMapper.active()) {
throw new SearchParseException(context, "[children] _parent field not configured", parser.getTokenLocation());
throw new SearchParseException(context, "[children] no [_parent] field not configured that points to a parent type", parser.getTokenLocation());
}
parentType = parentFieldMapper.type();
DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType);

View File

@ -20,6 +20,7 @@
package org.elasticsearch.aliases;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequestBuilder;
@ -33,6 +34,7 @@ import org.elasticsearch.cluster.metadata.AliasAction;
import org.elasticsearch.cluster.metadata.AliasMetaData;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.StopWatch;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
@ -56,30 +58,12 @@ import java.util.concurrent.TimeUnit;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.client.Requests.createIndexRequest;
import static org.elasticsearch.client.Requests.indexRequest;
import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_METADATA_BLOCK;
import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_READ_ONLY_BLOCK;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_METADATA;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_READ;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_WRITE;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY;
import static org.elasticsearch.cluster.metadata.IndexMetaData.*;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery;
import static org.elasticsearch.index.query.QueryBuilders.hasParentQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.rangeQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.test.hamcrest.CollectionAssertions.hasKey;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertBlocked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
/**
*
@ -986,6 +970,30 @@ public class IndexAliasesTests extends ElasticsearchIntegrationTest {
.addMapping("parent")
.addMapping("child", "_parent", "type=parent")
);
try {
assertAcked(admin().indices().prepareAliases().addAlias("my-index", "filter1", hasChildQuery("child", matchAllQuery())));
} catch (IllegalArgumentException e) {
assertThat(e.getCause(), instanceOf(IllegalStateException.class));
assertThat(e.getCause().getMessage(), equalTo("Search context is required to be set"));
}
try {
assertAcked(admin().indices().prepareAliases().addAlias("my-index", "filter2", hasParentQuery("child", matchAllQuery())));
} catch (IllegalArgumentException e) {
assertThat(e.getCause(), instanceOf(IllegalStateException.class));
assertThat(e.getCause().getMessage(), equalTo("Search context is required to be set"));
}
}
@Test
public void testAliasesFilterWithHasChildQueryPre2Dot0() throws Exception {
assertAcked(prepareCreate("my-index")
.setSettings(Settings.builder()
.put(indexSettings())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_6_0)
)
.addMapping("parent")
.addMapping("child", "_parent", "type=parent")
);
assertAcked(admin().indices().prepareAliases().addAlias("my-index", "filter1", hasChildQuery("child", matchAllQuery())));
assertAcked(admin().indices().prepareAliases().addAlias("my-index", "filter2", hasParentQuery("child", matchAllQuery())));
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.benchmark.search.child;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.search.SearchResponse;
@ -36,12 +37,7 @@ import static org.elasticsearch.client.Requests.createIndexRequest;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.index.query.QueryBuilders.filteredQuery;
import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery;
import static org.elasticsearch.index.query.QueryBuilders.hasParentQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
/**
@ -49,26 +45,49 @@ import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
*/
public class ChildSearchBenchmark {
/*
Run: MAVEN_OPTS=-Xmx4g mvn test-compile exec:java -Dexec.mainClass="org.elasticsearch.benchmark.search.child.ChildSearchBenchmark" -Dexec.classpathScope="test" -Dexec.args="bwc false"
*/
public static void main(String[] args) throws Exception {
Settings settings = settingsBuilder()
boolean bwcMode = false;
int numParents = (int) SizeValue.parseSizeValue("2m").singles();;
if (args.length % 2 != 0) {
throw new IllegalArgumentException("Uneven number of arguments");
}
for (int i = 0; i < args.length; i += 2) {
String value = args[i + 1];
if ("--bwc_mode".equals(args[i])) {
bwcMode = Boolean.valueOf(value);
} else if ("--num_parents".equals(args[i])) {
numParents = Integer.valueOf(value);
}
}
Settings.Builder settings = settingsBuilder()
.put("index.refresh_interval", "-1")
.put(SETTING_NUMBER_OF_SHARDS, 1)
.put(SETTING_NUMBER_OF_REPLICAS, 0)
.build();
.put(SETTING_NUMBER_OF_REPLICAS, 0);
// enable bwc parent child mode:
if (bwcMode) {
settings.put("tests.mock.version", Version.V_1_6_0);
}
String clusterName = ChildSearchBenchmark.class.getSimpleName();
Node node1 = nodeBuilder().clusterName(clusterName)
.settings(settingsBuilder().put(settings).put("name", "node1")).node();
.settings(settingsBuilder().put(settings.build()).put("name", "node1")).node();
Client client = node1.client();
int COUNT = (int) SizeValue.parseSizeValue("2m").singles();
int CHILD_COUNT = 15;
int QUERY_VALUE_RATIO = 3;
int QUERY_WARMUP = 10;
int QUERY_COUNT = 20;
String indexName = "test";
ParentChildIndexGenerator parentChildIndexGenerator = new ParentChildIndexGenerator(client, COUNT, CHILD_COUNT, QUERY_VALUE_RATIO);
ParentChildIndexGenerator parentChildIndexGenerator = new ParentChildIndexGenerator(client, numParents, CHILD_COUNT, QUERY_VALUE_RATIO);
client.admin().cluster().prepareHealth(indexName).setWaitForGreenStatus().setTimeout("10s").execute().actionGet();
try {
client.admin().indices().create(createIndexRequest(indexName)).actionGet();

View File

@ -21,16 +21,12 @@ package org.elasticsearch.index.fielddata;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TopFieldDocs;
import org.apache.lucene.search.*;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
import org.elasticsearch.common.compress.CompressedXContent;
@ -49,7 +45,6 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.nullValue;
/**
@ -71,37 +66,48 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTests {
Document d = new Document();
d.add(new StringField(UidFieldMapper.NAME, Uid.createUid(parentType, "1"), Field.Store.NO));
d.add(createJoinField(parentType, "1"));
writer.addDocument(d);
d = new Document();
d.add(new StringField(UidFieldMapper.NAME, Uid.createUid(childType, "2"), Field.Store.NO));
d.add(new StringField(ParentFieldMapper.NAME, Uid.createUid(parentType, "1"), Field.Store.NO));
d.add(createJoinField(parentType, "1"));
d.add(createJoinField(childType, "2"));
writer.addDocument(d);
writer.commit();
d = new Document();
d.add(new StringField(UidFieldMapper.NAME, Uid.createUid(childType, "3"), Field.Store.NO));
d.add(new StringField(ParentFieldMapper.NAME, Uid.createUid(parentType, "1"), Field.Store.NO));
d.add(createJoinField(parentType, "1"));
d.add(createJoinField(childType, "3"));
writer.addDocument(d);
d = new Document();
d.add(new StringField(UidFieldMapper.NAME, Uid.createUid(parentType, "2"), Field.Store.NO));
d.add(createJoinField(parentType, "2"));
writer.addDocument(d);
d = new Document();
d.add(new StringField(UidFieldMapper.NAME, Uid.createUid(childType, "4"), Field.Store.NO));
d.add(new StringField(ParentFieldMapper.NAME, Uid.createUid(parentType, "2"), Field.Store.NO));
d.add(createJoinField(parentType, "2"));
d.add(createJoinField(childType, "4"));
writer.addDocument(d);
d = new Document();
d.add(new StringField(UidFieldMapper.NAME, Uid.createUid(childType, "5"), Field.Store.NO));
d.add(new StringField(ParentFieldMapper.NAME, Uid.createUid(parentType, "1"), Field.Store.NO));
d.add(createJoinField(parentType, "1"));
d.add(createJoinField(childType, "5"));
writer.addDocument(d);
writer.commit();
d = new Document();
d.add(new StringField(UidFieldMapper.NAME, Uid.createUid(grandChildType, "6"), Field.Store.NO));
d.add(new StringField(ParentFieldMapper.NAME, Uid.createUid(childType, "2"), Field.Store.NO));
d.add(createJoinField(childType, "2"));
writer.addDocument(d);
d = new Document();
@ -109,11 +115,14 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTests {
writer.addDocument(d);
}
private SortedDocValuesField createJoinField(String parentType, String id) {
return new SortedDocValuesField(ParentFieldMapper.joinField(parentType), new BytesRef(id));
}
@Test
public void testGetBytesValues() throws Exception {
IndexFieldData indexFieldData = getForField(childType);
AtomicFieldData fieldData = indexFieldData.load(refreshReader());
assertThat(fieldData.ramBytesUsed(), greaterThan(0l));
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
bytesValues.setDocument(0);

View File

@ -21,53 +21,23 @@ package org.elasticsearch.index.query;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.*;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.queries.BoostingQuery;
import org.apache.lucene.queries.ExtendedCommonTermsQuery;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.*;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.FuzzyQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermRangeQuery;
import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.search.spans.FieldMaskingSpanQuery;
import org.apache.lucene.search.spans.SpanContainingQuery;
import org.apache.lucene.search.spans.SpanFirstQuery;
import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanNotQuery;
import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.spans.SpanWithinQuery;
import org.apache.lucene.search.join.ToParentBlockJoinQuery;
import org.apache.lucene.search.spans.*;
import org.apache.lucene.spatial.prefix.IntersectsPrefixTreeFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
import org.elasticsearch.action.termvectors.MultiTermVectorsItemResponse;
import org.elasticsearch.action.termvectors.MultiTermVectorsRequest;
import org.elasticsearch.action.termvectors.MultiTermVectorsResponse;
import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.action.termvectors.*;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.compress.CompressedXContent;
@ -90,7 +60,6 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
import org.elasticsearch.index.search.child.ParentConstantScoreQuery;
import org.elasticsearch.index.search.geo.GeoDistanceRangeQuery;
import org.elasticsearch.index.search.geo.GeoPolygonQuery;
import org.elasticsearch.index.search.geo.InMemoryGeoBoundingBoxQuery;
@ -109,43 +78,10 @@ import java.util.List;
import static org.elasticsearch.common.io.Streams.copyToBytesFromClasspath;
import static org.elasticsearch.common.io.Streams.copyToStringFromClasspath;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.andQuery;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.boostingQuery;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.disMaxQuery;
import static org.elasticsearch.index.query.QueryBuilders.filteredQuery;
import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.fuzzyQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.moreLikeThisQuery;
import static org.elasticsearch.index.query.QueryBuilders.notQuery;
import static org.elasticsearch.index.query.QueryBuilders.orQuery;
import static org.elasticsearch.index.query.QueryBuilders.prefixQuery;
import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery;
import static org.elasticsearch.index.query.QueryBuilders.rangeQuery;
import static org.elasticsearch.index.query.QueryBuilders.regexpQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanContainingQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanFirstQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanNearQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanNotQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanOrQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanTermQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanWithinQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.termsQuery;
import static org.elasticsearch.index.query.QueryBuilders.wildcardQuery;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.factorFunction;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertBooleanSubQuery;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.sameInstance;
import static org.hamcrest.Matchers.*;
/**
*
@ -2485,14 +2421,13 @@ public class SimpleIndexQueryParserTests extends ElasticsearchSingleNodeTest {
public void testEmptyBoolSubClausesIsMatchAll() throws IOException {
String query = copyToStringFromClasspath("/org/elasticsearch/index/query/bool-query-with-empty-clauses-for-parsing.json");
IndexService indexService = createIndex("testidx", client().admin().indices().prepareCreate("testidx")
.addMapping("foo")
.addMapping("test", "_parent", "type=foo"));
.addMapping("foo", "nested", "type=nested"));
SearchContext.setCurrent(createSearchContext(indexService));
IndexQueryParserService queryParser = indexService.queryParserService();
Query parsedQuery = queryParser.parse(query).query();
assertThat(parsedQuery, instanceOf(ConstantScoreQuery.class));
assertThat(((ConstantScoreQuery) parsedQuery).getQuery(), instanceOf(ParentConstantScoreQuery.class));
assertThat(((ConstantScoreQuery) parsedQuery).getQuery().toString(), equalTo("parent_filter[foo](+*:* #ConstantScore(_type:foo))"));
assertThat(((ConstantScoreQuery) parsedQuery).getQuery(), instanceOf(ToParentBlockJoinQuery.class));
assertThat(((ConstantScoreQuery) parsedQuery).getQuery().toString(), equalTo("ToParentBlockJoinQuery (+*:* #random_access(QueryWrapperFilter(_type:__nested)))"));
SearchContext.removeCurrent();
}

View File

@ -1,8 +1,8 @@
{
"filtered": {
"filter": {
"has_parent": {
"type": "foo",
"nested": {
"path": "nested",
"query": {
"bool": {
"must": [],

View File

@ -19,19 +19,15 @@
package org.elasticsearch.index.search.child;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.*;
import org.apache.lucene.search.join.BitDocIdSetFilter;
import org.apache.lucene.util.BitDocIdSet;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
@ -44,8 +40,6 @@ import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import org.hamcrest.Description;
import org.hamcrest.StringDescription;
import org.junit.After;
import org.junit.Before;
import org.junit.Ignore;
import java.io.IOException;
@ -63,7 +57,10 @@ public abstract class AbstractChildTests extends ElasticsearchSingleNodeTest {
protected static String CHILD_SCORE_NAME = "childScore";
static SearchContext createSearchContext(String indexName, String parentType, String childType) throws IOException {
IndexService indexService = createIndex(indexName);
Settings settings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_6_0)
.build();
IndexService indexService = createIndex(indexName, settings);
MapperService mapperService = indexService.mapperService();
// Parent/child parsers require that the parent and child type to be presented in mapping
// Sometimes we want a nested object field in the parent type that triggers nonNestedDocsFilter to be used

View File

@ -0,0 +1,219 @@
/*
* 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.search.child;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
import org.junit.Test;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
/**
*
*/
@ClusterScope(scope = Scope.SUITE)
public class ChildQuerySearchBwcTests extends ChildQuerySearchTests {
@Override
public Settings indexSettings() {
return Settings.builder()
.put(super.indexSettings()).put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_6_0).build();
}
public void testSelfReferentialIsForbidden() {
// we allowed this, but it was actually broken. The has_child/has_parent results were sometimes wrong...
assertAcked(prepareCreate("test").addMapping("type", "_parent", "type=type"));
}
@Test
public void testAdd_ParentFieldAfterIndexingParentDocButBeforeIndexingChildDoc() throws Exception {
assertAcked(prepareCreate("test")
.setSettings(Settings.builder()
.put(indexSettings())
.put("index.refresh_interval", -1)));
ensureGreen();
String parentId = "p1";
client().prepareIndex("test", "parent", parentId).setSource("p_field", "1").get();
refresh();
assertAcked(client().admin()
.indices()
.preparePutMapping("test")
.setType("child")
.setSource("_parent", "type=parent"));
client().prepareIndex("test", "child", "c1").setSource("c_field", "1").setParent(parentId).get();
client().admin().indices().prepareRefresh().get();
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(hasChildQuery("child", termQuery("c_field", "1")))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, parentId);
searchResponse = client().prepareSearch("test")
.setQuery(hasChildQuery("child", termQuery("c_field", "1")).scoreType("max"))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, parentId);
searchResponse = client().prepareSearch("test")
.setPostFilter(hasChildQuery("child", termQuery("c_field", "1")))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, parentId);
searchResponse = client().prepareSearch("test")
.setPostFilter(hasParentQuery("parent", termQuery("p_field", "1")))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, "c1");
searchResponse = client().prepareSearch("test")
.setQuery(hasParentQuery("parent", termQuery("p_field", "1")).scoreType("score"))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, "c1");
}
@Test
public void testExplainUsage() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("parent")
.addMapping("child", "_parent", "type=parent"));
ensureGreen();
String parentId = "p1";
client().prepareIndex("test", "parent", parentId).setSource("p_field", "1").get();
client().prepareIndex("test", "child", "c1").setSource("c_field", "1").setParent(parentId).get();
refresh();
SearchResponse searchResponse = client().prepareSearch("test")
.setExplain(true)
.setQuery(hasChildQuery("child", termQuery("c_field", "1")).scoreType("max"))
.get();
assertHitCount(searchResponse, 1l);
assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), equalTo("not implemented yet..."));
searchResponse = client().prepareSearch("test")
.setExplain(true)
.setQuery(hasParentQuery("parent", termQuery("p_field", "1")).scoreType("score"))
.get();
assertHitCount(searchResponse, 1l);
assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), equalTo("not implemented yet..."));
ExplainResponse explainResponse = client().prepareExplain("test", "parent", parentId)
.setQuery(hasChildQuery("child", termQuery("c_field", "1")).scoreType("max"))
.get();
assertThat(explainResponse.isExists(), equalTo(true));
// TODO: improve test once explanations are actually implemented
assertThat(explainResponse.getExplanation().toString(), startsWith("1.0 ="));
}
@Test
@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elasticsearch/elasticsearch/issues/9270")
public void testParentFieldDataCacheBug() throws Exception {
assertAcked(prepareCreate("test")
.setSettings(Settings.builder().put(indexSettings())
.put("index.refresh_interval", -1)) // Disable automatic refresh, so that the _parent doesn't get warmed
.addMapping("parent", jsonBuilder().startObject().startObject("parent")
.startObject("properties")
.startObject("p_field")
.field("type", "string")
.startObject("fielddata")
.field(FieldDataType.FORMAT_KEY, FieldMapper.Loading.LAZY)
.endObject()
.endObject()
.endObject().endObject().endObject()));
ensureGreen();
client().prepareIndex("test", "parent", "p0").setSource("p_field", "p_value0").get();
client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
refresh();
// No _parent field yet, there shouldn't be anything in the field data for _parent field
IndicesStatsResponse indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l));
// Now add mapping + children
client().admin().indices().preparePutMapping("test").setType("child")
.setSource(XContentFactory.jsonBuilder().startObject().startObject("child")
.startObject("_parent")
.field("type", "parent")
.endObject()
.startObject("properties")
.startObject("c_field")
.field("type", "string")
.startObject("fielddata")
.field(FieldDataType.FORMAT_KEY, FieldMapper.Loading.LAZY)
.endObject()
.endObject()
.endObject().endObject().endObject())
.get();
// index simple data
client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
refresh();
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l));
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"))))
.get();
assertNoFailures(searchResponse);
assertThat(searchResponse.getHits().totalHits(), equalTo(1l));
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l));
ClearIndicesCacheResponse clearCacheResponse = client().admin().indices().prepareClearCache("test").setFieldDataCache(true).get();
assertNoFailures(clearCacheResponse);
assertAllSuccessful(clearCacheResponse);
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), equalTo(0l));
}
}

View File

@ -19,10 +19,8 @@
package org.elasticsearch.search.child;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
@ -32,12 +30,9 @@ import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.common.lucene.search.function.CombineFunction;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.cache.filter.FilterCacheModule;
import org.elasticsearch.index.cache.filter.FilterCacheModule.FilterCacheSettings;
import org.elasticsearch.index.cache.filter.index.IndexFilterCache;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.FieldMapper.Loading;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.query.HasChildQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
@ -58,53 +53,23 @@ import org.hamcrest.Matchers;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.*;
import static com.google.common.collect.Maps.newHashMap;
import static org.elasticsearch.common.io.Streams.copyToStringFromClasspath;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.filteredQuery;
import static org.elasticsearch.index.query.QueryBuilders.hasParentQuery;
import static org.elasticsearch.index.query.QueryBuilders.idsQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery;
import static org.elasticsearch.index.query.QueryBuilders.notQuery;
import static org.elasticsearch.index.query.QueryBuilders.prefixQuery;
import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.termsQuery;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.factorFunction;
import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.scriptFunction;
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.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.anyOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.startsWith;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
/**
*
*/
@ClusterScope(scope = Scope.SUITE)
public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest {
public class ChildQuerySearchTests extends ElasticsearchIntegrationTest {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
@ -115,6 +80,18 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest {
.build();
}
@Test
public void testSelfReferentialIsForbidden() {
try {
prepareCreate("test").addMapping("type", "_parent", "type=type").get();
fail("self referential should be forbidden");
} catch (Exception e) {
Throwable cause = e.getCause();
assertThat(cause, instanceOf(IllegalArgumentException.class));
assertThat(cause.getMessage(), equalTo("The [_parent.type] option can't point to the same type"));
}
}
@Test
public void multiLevelChild() throws Exception {
assertAcked(prepareCreate("test")
@ -281,83 +258,6 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getAt(1).id(), equalTo("c2"));
}
@Test
@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elasticsearch/elasticsearch/issues/9270")
public void testParentFieldDataCacheBug() throws Exception {
assertAcked(prepareCreate("test")
.setSettings(Settings.builder().put(indexSettings())
.put("index.refresh_interval", -1)) // Disable automatic refresh, so that the _parent doesn't get warmed
.addMapping("parent", XContentFactory.jsonBuilder().startObject().startObject("parent")
.startObject("properties")
.startObject("p_field")
.field("type", "string")
.startObject("fielddata")
.field(FieldDataType.FORMAT_KEY, Loading.LAZY)
.endObject()
.endObject()
.endObject().endObject().endObject()));
ensureGreen();
client().prepareIndex("test", "parent", "p0").setSource("p_field", "p_value0").get();
client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
refresh();
// No _parent field yet, there shouldn't be anything in the field data for _parent field
IndicesStatsResponse indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l));
// Now add mapping + children
client().admin().indices().preparePutMapping("test").setType("child")
.setSource(XContentFactory.jsonBuilder().startObject().startObject("child")
.startObject("_parent")
.field("type", "parent")
.endObject()
.startObject("properties")
.startObject("c_field")
.field("type", "string")
.startObject("fielddata")
.field(FieldDataType.FORMAT_KEY, Loading.LAZY)
.endObject()
.endObject()
.endObject().endObject().endObject())
.get();
// index simple data
client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
refresh();
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l));
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"))))
.get();
assertNoFailures(searchResponse);
assertThat(searchResponse.getHits().totalHits(), equalTo(1l));
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l));
ClearIndicesCacheResponse clearCacheResponse = client().admin().indices().prepareClearCache("test").setFieldDataCache(true).get();
assertNoFailures(clearCacheResponse);
assertAllSuccessful(clearCacheResponse);
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), equalTo(0l));
}
@Test
// See: https://github.com/elasticsearch/elasticsearch/issues/3290
public void testCachingBug_withFqueryFilter() throws Exception {
@ -685,21 +585,20 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest {
.setQuery(hasChildQuery("child", termQuery("c_field", "1")).scoreType("max"))
.get();
assertHitCount(searchResponse, 1l);
assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), equalTo("not implemented yet..."));
assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), equalTo("Score based on join value p1"));
searchResponse = client().prepareSearch("test")
.setExplain(true)
.setQuery(hasParentQuery("parent", termQuery("p_field", "1")).scoreType("score"))
.get();
assertHitCount(searchResponse, 1l);
assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), equalTo("not implemented yet..."));
assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), equalTo("Score based on join value p1"));
ExplainResponse explainResponse = client().prepareExplain("test", "parent", parentId)
.setQuery(hasChildQuery("child", termQuery("c_field", "1")).scoreType("max"))
.get();
assertThat(explainResponse.isExists(), equalTo(true));
// TODO: improve test once explanations are actually implemented
assertThat(explainResponse.getExplanation().toString(), startsWith("1.0 ="));
assertThat(explainResponse.getExplanation().getDetails()[0].getDescription(), equalTo("Score based on join value p1"));
}
List<IndexRequestBuilder> createDocBuilders() {
@ -1454,44 +1353,17 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest {
String parentId = "p1";
client().prepareIndex("test", "parent", parentId).setSource("p_field", "1").get();
refresh();
try {
assertAcked(client().admin()
.indices()
.preparePutMapping("test")
.setType("child")
.setSource("_parent", "type=parent"));
client().prepareIndex("test", "child", "c1").setSource("c_field", "1").setParent(parentId).get();
client().admin().indices().prepareRefresh().get();
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(hasChildQuery("child", termQuery("c_field", "1")))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, parentId);
searchResponse = client().prepareSearch("test")
.setQuery(hasChildQuery("child", termQuery("c_field", "1")).scoreType("max"))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, parentId);
searchResponse = client().prepareSearch("test")
.setPostFilter(hasChildQuery("child", termQuery("c_field", "1")))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, parentId);
searchResponse = client().prepareSearch("test")
.setPostFilter(hasParentQuery("parent", termQuery("p_field", "1")))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, "c1");
searchResponse = client().prepareSearch("test")
.setQuery(hasParentQuery("parent", termQuery("p_field", "1")).scoreType("score"))
.get();
assertHitCount(searchResponse, 1l);
assertSearchHits(searchResponse, "c1");
fail("Shouldn't be able the add the _parent field pointing to an already existing parent type");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), equalTo("can't add a _parent field that points to an already existing type"));
}
}
@Test
@ -2175,7 +2047,7 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest {
assertHitCount(response, 0);
}
private static HasChildQueryBuilder hasChildQuery(String type, QueryBuilder queryBuilder) {
static HasChildQueryBuilder hasChildQuery(String type, QueryBuilder queryBuilder) {
HasChildQueryBuilder hasChildQueryBuilder = QueryBuilders.hasChildQuery(type, queryBuilder);
hasChildQueryBuilder.setShortCircuitCutoff(randomInt(10));
return hasChildQueryBuilder;

View File

@ -0,0 +1,269 @@
/*
* 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.search.child;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.merge.policy.MergePolicyModule;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.index.merge.NoMergePolicyProvider;
import org.junit.Test;
import java.io.IOException;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.search.child.ChildQuerySearchTests.hasChildQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
/**
*/
public class ParentFieldLoadingBwcTest extends ElasticsearchIntegrationTest {
private final Settings indexSettings = Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
.put(IndexShard.INDEX_REFRESH_INTERVAL, -1)
// We never want merges in this test to ensure we have two segments for the last validation
.put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_6_0)
.build();
@Test
@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elasticsearch/elasticsearch/issues/9270")
public void testParentFieldDataCacheBug() throws Exception {
assertAcked(prepareCreate("test")
.setSettings(Settings.builder().put(indexSettings())
.put("index.refresh_interval", -1)) // Disable automatic refresh, so that the _parent doesn't get warmed
.addMapping("parent", XContentFactory.jsonBuilder().startObject().startObject("parent")
.startObject("properties")
.startObject("p_field")
.field("type", "string")
.startObject("fielddata")
.field(FieldDataType.FORMAT_KEY, FieldMapper.Loading.LAZY)
.endObject()
.endObject()
.endObject().endObject().endObject()));
ensureGreen();
client().prepareIndex("test", "parent", "p0").setSource("p_field", "p_value0").get();
client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
refresh();
// No _parent field yet, there shouldn't be anything in the field data for _parent field
IndicesStatsResponse indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l));
// Now add mapping + children
client().admin().indices().preparePutMapping("test").setType("child")
.setSource(XContentFactory.jsonBuilder().startObject().startObject("child")
.startObject("_parent")
.field("type", "parent")
.endObject()
.startObject("properties")
.startObject("c_field")
.field("type", "string")
.startObject("fielddata")
.field(FieldDataType.FORMAT_KEY, FieldMapper.Loading.LAZY)
.endObject()
.endObject()
.endObject().endObject().endObject())
.get();
// index simple data
client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
refresh();
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l));
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"))))
.get();
assertNoFailures(searchResponse);
assertThat(searchResponse.getHits().totalHits(), equalTo(1l));
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l));
ClearIndicesCacheResponse clearCacheResponse = client().admin().indices().prepareClearCache("test").setFieldDataCache(true).get();
assertNoFailures(clearCacheResponse);
assertAllSuccessful(clearCacheResponse);
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get();
assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l));
assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), equalTo(0l));
}
@Test
public void testEagerParentFieldLoading() throws Exception {
logger.info("testing lazy loading...");
assertAcked(prepareCreate("test")
.setSettings(indexSettings)
.addMapping("parent")
.addMapping("child", childMapping(FieldMapper.Loading.LAZY)));
ensureGreen();
client().prepareIndex("test", "parent", "1").setSource("{}").get();
client().prepareIndex("test", "child", "1").setParent("1").setSource("{}").get();
refresh();
ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(0l));
logger.info("testing default loading...");
assertAcked(client().admin().indices().prepareDelete("test").get());
assertAcked(prepareCreate("test")
.setSettings(indexSettings)
.addMapping("parent")
.addMapping("child", "_parent", "type=parent"));
ensureGreen();
client().prepareIndex("test", "parent", "1").setSource("{}").get();
client().prepareIndex("test", "child", "1").setParent("1").setSource("{}").get();
refresh();
response = client().admin().cluster().prepareClusterStats().get();
long fielddataSizeDefault = response.getIndicesStats().getFieldData().getMemorySizeInBytes();
assertThat(fielddataSizeDefault, greaterThan(0l));
logger.info("testing eager loading...");
assertAcked(client().admin().indices().prepareDelete("test").get());
assertAcked(prepareCreate("test")
.setSettings(indexSettings)
.addMapping("parent")
.addMapping("child", childMapping(FieldMapper.Loading.EAGER)));
ensureGreen();
client().prepareIndex("test", "parent", "1").setSource("{}").get();
client().prepareIndex("test", "child", "1").setParent("1").setSource("{}").get();
refresh();
response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(fielddataSizeDefault));
logger.info("testing eager global ordinals loading...");
assertAcked(client().admin().indices().prepareDelete("test").get());
assertAcked(prepareCreate("test")
.setSettings(indexSettings)
.addMapping("parent")
.addMapping("child", childMapping(FieldMapper.Loading.EAGER_GLOBAL_ORDINALS)));
ensureGreen();
// Need to do 2 separate refreshes, otherwise we have 1 segment and then we can't measure if global ordinals
// is loaded by the size of the field data cache, because global ordinals on 1 segment shards takes no extra memory.
client().prepareIndex("test", "parent", "1").setSource("{}").get();
refresh();
client().prepareIndex("test", "child", "1").setParent("1").setSource("{}").get();
refresh();
response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(fielddataSizeDefault));
}
@Test
public void testChangingEagerParentFieldLoadingAtRuntime() throws Exception {
assertAcked(prepareCreate("test")
.setSettings(indexSettings)
.addMapping("parent")
.addMapping("child", "_parent", "type=parent"));
ensureGreen();
client().prepareIndex("test", "parent", "1").setSource("{}").get();
client().prepareIndex("test", "child", "1").setParent("1").setSource("{}").get();
refresh();
ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get();
long fielddataSizeDefault = response.getIndicesStats().getFieldData().getMemorySizeInBytes();
assertThat(fielddataSizeDefault, greaterThan(0l));
PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping("test").setType("child")
.setSource(childMapping(FieldMapper.Loading.EAGER_GLOBAL_ORDINALS))
.get();
assertAcked(putMappingResponse);
assertBusy(new Runnable() {
@Override
public void run() {
ClusterState clusterState = internalCluster().clusterService().state();
ShardRouting shardRouting = clusterState.routingTable().index("test").shard(0).getShards().get(0);
String nodeName = clusterState.getNodes().get(shardRouting.currentNodeId()).getName();
boolean verified = false;
IndicesService indicesService = internalCluster().getInstance(IndicesService.class, nodeName);
IndexService indexService = indicesService.indexService("test");
if (indexService != null) {
MapperService mapperService = indexService.mapperService();
DocumentMapper documentMapper = mapperService.documentMapper("child");
if (documentMapper != null) {
verified = documentMapper.parentFieldMapper().fieldDataType().getLoading() == FieldMapper.Loading.EAGER_GLOBAL_ORDINALS;
}
}
assertTrue(verified);
}
});
// Need to add a new doc otherwise the refresh doesn't trigger a new searcher
// Because it ends up in its own segment, but isn't of type parent or child, this doc doesn't contribute to the size of the fielddata cache
client().prepareIndex("test", "dummy", "dummy").setSource("{}").get();
refresh();
response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(fielddataSizeDefault));
}
private XContentBuilder childMapping(FieldMapper.Loading loading) throws IOException {
return jsonBuilder().startObject().startObject("child").startObject("_parent")
.field("type", "parent")
.startObject("fielddata").field(FieldMapper.Loading.KEY, loading).endObject()
.endObject().endObject().endObject();
}
}

View File

@ -85,8 +85,7 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest {
refresh();
response = client().admin().cluster().prepareClusterStats().get();
long fielddataSizeDefault = response.getIndicesStats().getFieldData().getMemorySizeInBytes();
assertThat(fielddataSizeDefault, greaterThan(0l));
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(0l));
logger.info("testing eager loading...");
assertAcked(client().admin().indices().prepareDelete("test").get());
@ -101,7 +100,7 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest {
refresh();
response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(fielddataSizeDefault));
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(0l));
logger.info("testing eager global ordinals loading...");
assertAcked(client().admin().indices().prepareDelete("test").get());
@ -119,7 +118,7 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest {
refresh();
response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(fielddataSizeDefault));
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
}
@Test
@ -135,8 +134,7 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest {
refresh();
ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get();
long fielddataSizeDefault = response.getIndicesStats().getFieldData().getMemorySizeInBytes();
assertThat(fielddataSizeDefault, greaterThan(0l));
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(0l));
PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping("test").setType("child")
.setSource(childMapping(FieldMapper.Loading.EAGER_GLOBAL_ORDINALS))
@ -168,7 +166,7 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest {
client().prepareIndex("test", "dummy", "dummy").setSource("{}").get();
refresh();
response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(fielddataSizeDefault));
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(0l));
}
private XContentBuilder childMapping(FieldMapper.Loading loading) throws IOException {

View File

@ -473,12 +473,15 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
@Test
public void testContextVariables() throws Exception {
createTestIndex();
// Add child type for testing the _parent context variable
client().admin().indices().preparePutMapping("test")
.setType("subtype1")
.setSource(XContentFactory.jsonBuilder()
assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("type1", XContentFactory.jsonBuilder()
.startObject()
.startObject("type1")
.startObject("_timestamp").field("enabled", true).field("store", "yes").endObject()
.startObject("_ttl").field("enabled", true).endObject()
.endObject()
.endObject())
.addMapping("subtype1", XContentFactory.jsonBuilder()
.startObject()
.startObject("subtype1")
.startObject("_parent").field("type", "type1").endObject()
@ -486,7 +489,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
.startObject("_ttl").field("enabled", true).endObject()
.endObject()
.endObject())
.execute().actionGet();
);
ensureGreen();
// Index some documents
@ -1215,17 +1218,23 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
*/
@Test
public void testContextVariablesOldScriptAPI() throws Exception {
createTestIndex();
// Add child type for testing the _parent context variable
client().admin()
.indices()
.preparePutMapping("test")
.setType("subtype1")
.setSource(
XContentFactory.jsonBuilder().startObject().startObject("subtype1").startObject("_parent").field("type", "type1")
.endObject().startObject("_timestamp").field("enabled", true).field("store", "yes").endObject()
.startObject("_ttl").field("enabled", true).endObject().endObject().endObject()).execute().actionGet();
assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("type1", XContentFactory.jsonBuilder()
.startObject()
.startObject("type1")
.startObject("_timestamp").field("enabled", true).field("store", "yes").endObject()
.startObject("_ttl").field("enabled", true).endObject()
.endObject()
.endObject())
.addMapping("subtype1", XContentFactory.jsonBuilder()
.startObject()
.startObject("subtype1")
.startObject("_parent").field("type", "type1").endObject()
.startObject("_timestamp").field("enabled", true).field("store", "yes").endObject()
.startObject("_ttl").field("enabled", true).endObject()
.endObject()
.endObject())
);
ensureGreen();
// Index some documents