Always return metadata in get/search APIs.

This commit makes the get and search APIs always return `_parent`, `_routing`,
`_timestamp` and `_ttl` in addition to `_id` and `_type`. This way, consumers
always have all required information in order to reindex a document.
This commit is contained in:
Adrien Grand 2015-06-22 18:12:07 +02:00
parent d32443bfb5
commit e4d475c700
20 changed files with 303 additions and 296 deletions

View File

@ -995,7 +995,6 @@ public abstract class Engine implements Closeable {
public static class Get {
private final boolean realtime;
private final Term uid;
private boolean loadSource = true;
private long version = Versions.MATCH_ANY;
private VersionType versionType = VersionType.INTERNAL;
@ -1012,15 +1011,6 @@ public abstract class Engine implements Closeable {
return uid;
}
public boolean loadSource() {
return this.loadSource;
}
public Get loadSource(boolean loadSource) {
this.loadSource = loadSource;
return this;
}
public long version() {
return version;
}

View File

@ -305,9 +305,6 @@ public class InternalEngine extends Engine {
Uid uid = Uid.createUid(get.uid().text());
throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), versionValue.version(), get.version());
}
if (!get.loadSource()) {
return new GetResult(true, versionValue.version(), null);
}
Translog.Operation op = translog.read(versionValue.translogLocation());
if (op != null) {
return new GetResult(true, versionValue.version(), op.getSource());

View File

@ -27,6 +27,7 @@ import java.io.IOException;
public class AllFieldsVisitor extends FieldsVisitor {
public AllFieldsVisitor() {
super(true);
}
@Override

View File

@ -19,8 +19,6 @@
package org.elasticsearch.index.fieldvisitor;
import org.apache.lucene.index.FieldInfo;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import java.io.IOException;
import java.util.Set;
@ -32,21 +30,16 @@ import java.util.Set;
*/
public class CustomFieldsVisitor extends FieldsVisitor {
private final boolean loadSource;
private final Set<String> fields;
public CustomFieldsVisitor(Set<String> fields, boolean loadSource) {
this.loadSource = loadSource;
super(loadSource);
this.fields = fields;
}
@Override
public Status needsField(FieldInfo fieldInfo) throws IOException {
if (loadSource && SourceFieldMapper.NAME.equals(fieldInfo.name)) {
return Status.YES;
}
if (UidFieldMapper.NAME.equals(fieldInfo.name)) {
if (super.needsField(fieldInfo) == Status.YES) {
return Status.YES;
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.fieldvisitor;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.StoredFieldVisitor;
@ -27,30 +28,63 @@ import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.FieldMappers;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.mapper.internal.RoutingFieldMapper;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static com.google.common.collect.Maps.newHashMap;
/**
* Base {@link StoredFieldsVisitor} that retrieves all non-redundant metadata.
*/
public abstract class FieldsVisitor extends StoredFieldVisitor {
public class FieldsVisitor extends StoredFieldVisitor {
private static final Set<String> BASE_REQUIRED_FIELDS = ImmutableSet.of(
UidFieldMapper.NAME,
TimestampFieldMapper.NAME,
TTLFieldMapper.NAME,
RoutingFieldMapper.NAME,
ParentFieldMapper.NAME
);
private final boolean loadSource;
private final Set<String> requiredFields;
protected BytesReference source;
protected Uid uid;
protected Map<String, List<Object>> fieldsValues;
public FieldsVisitor(boolean loadSource) {
this.loadSource = loadSource;
requiredFields = new HashSet<>();
reset();
}
@Override
public Status needsField(FieldInfo fieldInfo) throws IOException {
if (requiredFields.remove(fieldInfo.name)) {
return Status.YES;
}
// All these fields are single-valued so we can stop when the set is
// empty
return requiredFields.isEmpty()
? Status.STOP
: Status.NO;
}
public void postProcess(MapperService mapperService) {
if (uid != null) {
DocumentMapper documentMapper = mapperService.documentMapper(uid.type());
@ -133,6 +167,18 @@ public abstract class FieldsVisitor extends StoredFieldVisitor {
return uid;
}
public String routing() {
if (fieldsValues == null) {
return null;
}
List<Object> values = fieldsValues.get(RoutingFieldMapper.NAME);
if (values == null || values.isEmpty()) {
return null;
}
assert values.size() == 1;
return values.get(0).toString();
}
public Map<String, List<Object>> fields() {
return fieldsValues != null
? fieldsValues
@ -143,6 +189,11 @@ public abstract class FieldsVisitor extends StoredFieldVisitor {
if (fieldsValues != null) fieldsValues.clear();
source = null;
uid = null;
requiredFields.addAll(BASE_REQUIRED_FIELDS);
if (loadSource) {
requiredFields.add(SourceFieldMapper.NAME);
}
}
void addValue(String name, Object value) {

View File

@ -1,37 +0,0 @@
/*
* 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.index.fieldvisitor;
import org.apache.lucene.index.FieldInfo;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import java.io.IOException;
/**
*/
public class JustSourceFieldsVisitor extends FieldsVisitor {
@Override
public Status needsField(FieldInfo fieldInfo) throws IOException {
if (SourceFieldMapper.NAME.equals(fieldInfo.name)) {
return Status.YES;
}
return source != null ? Status.STOP : Status.NO;
}
}

View File

@ -27,6 +27,10 @@ import java.io.IOException;
*/
public class JustUidFieldsVisitor extends FieldsVisitor {
public JustUidFieldsVisitor() {
super(false);
}
@Override
public Status needsField(FieldInfo fieldInfo) throws IOException {
if (UidFieldMapper.NAME.equals(fieldInfo.name)) {

View File

@ -35,6 +35,7 @@ public class SingleFieldsVisitor extends FieldsVisitor {
private String field;
public SingleFieldsVisitor(String field) {
super(false);
this.field = field;
}

View File

@ -1,57 +0,0 @@
/*
* 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.index.fieldvisitor;
import org.apache.lucene.index.FieldInfo;
import org.elasticsearch.index.mapper.internal.RoutingFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
/**
*/
public class UidAndRoutingFieldsVisitor extends FieldsVisitor {
private String routing;
@Override
public Status needsField(FieldInfo fieldInfo) throws IOException {
if (RoutingFieldMapper.NAME.equals(fieldInfo.name)) {
return Status.YES;
} else if (UidFieldMapper.NAME.equals(fieldInfo.name)) {
return Status.YES;
}
return uid != null && routing != null ? Status.STOP : Status.NO;
}
@Override
public void stringField(FieldInfo fieldInfo, byte[] bytes) throws IOException {
if (RoutingFieldMapper.NAME.equals(fieldInfo.name)) {
routing = new String(bytes, StandardCharsets.UTF_8);;
} else {
super.stringField(fieldInfo, bytes);
}
}
public String routing() {
return routing;
}
}

View File

@ -1,41 +0,0 @@
/*
* 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.index.fieldvisitor;
import org.apache.lucene.index.FieldInfo;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import java.io.IOException;
/**
*/
public class UidAndSourceFieldsVisitor extends FieldsVisitor {
@Override
public Status needsField(FieldInfo fieldInfo) throws IOException {
if (SourceFieldMapper.NAME.equals(fieldInfo.name)) {
return Status.YES;
} else if (UidFieldMapper.NAME.equals(fieldInfo.name)) {
return Status.YES;
}
return uid != null && source != null ? Status.STOP : Status.NO;
}
}

View File

@ -27,37 +27,33 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.fieldvisitor.CustomFieldsVisitor;
import org.elasticsearch.index.fieldvisitor.FieldsVisitor;
import org.elasticsearch.index.fieldvisitor.JustSourceFieldsVisitor;
import org.elasticsearch.index.mapper.*;
import org.elasticsearch.index.mapper.internal.*;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.lookup.LeafSearchLookup;
import org.elasticsearch.search.lookup.SearchLookup;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static com.google.common.collect.Maps.newHashMapWithExpectedSize;
@ -154,13 +150,11 @@ public final class ShardGetService extends AbstractIndexShardComponent {
private GetResult innerGet(String type, String id, String[] gFields, boolean realtime, long version, VersionType versionType, FetchSourceContext fetchSourceContext, boolean ignoreErrorsOnGeneratedFields) {
fetchSourceContext = normalizeFetchSourceContent(fetchSourceContext, gFields);
boolean loadSource = (gFields != null && gFields.length > 0) || fetchSourceContext.fetchSource();
Engine.GetResult get = null;
if (type == null || type.equals("_all")) {
for (String typeX : mapperService.types()) {
get = indexShard.get(new Engine.Get(realtime, new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(typeX, id)))
.loadSource(loadSource).version(version).versionType(versionType));
.version(version).versionType(versionType));
if (get.exists()) {
type = typeX;
break;
@ -177,7 +171,7 @@ public final class ShardGetService extends AbstractIndexShardComponent {
}
} else {
get = indexShard.get(new Engine.Get(realtime, new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(type, id)))
.loadSource(loadSource).version(version).versionType(versionType));
.version(version).versionType(versionType));
if (!get.exists()) {
get.release();
return new GetResult(shardId.index().name(), type, id, -1, false, null, null);
@ -201,20 +195,35 @@ public final class ShardGetService extends AbstractIndexShardComponent {
SearchLookup searchLookup = null;
// we can only load scripts that can run against the source
if (gFields != null && gFields.length > 0) {
for (String field : gFields) {
Set<String> neededFields = new HashSet<>();
// add meta fields
neededFields.add(RoutingFieldMapper.NAME);
if (docMapper.parentFieldMapper().active()) {
neededFields.add(ParentFieldMapper.NAME);
}
if (docMapper.timestampFieldMapper().enabled()) {
neededFields.add(TimestampFieldMapper.NAME);
}
if (docMapper.TTLFieldMapper().enabled()) {
neededFields.add(TTLFieldMapper.NAME);
}
// add requested fields
if (gFields != null) {
neededFields.addAll(Arrays.asList(gFields));
}
for (String field : neededFields) {
if (SourceFieldMapper.NAME.equals(field)) {
// dealt with when normalizing fetchSourceContext.
continue;
}
Object value = null;
if (field.equals(RoutingFieldMapper.NAME) && docMapper.routingFieldMapper().fieldType().stored()) {
if (field.equals(RoutingFieldMapper.NAME)) {
value = source.routing;
} else if (field.equals(ParentFieldMapper.NAME) && docMapper.parentFieldMapper().active() && docMapper.parentFieldMapper().fieldType().stored()) {
} else if (field.equals(ParentFieldMapper.NAME) && docMapper.parentFieldMapper().active()) {
value = source.parent;
} else if (field.equals(TimestampFieldMapper.NAME) && docMapper.timestampFieldMapper().fieldType().stored()) {
} else if (field.equals(TimestampFieldMapper.NAME) && docMapper.timestampFieldMapper().enabled()) {
value = source.timestamp;
} else if (field.equals(TTLFieldMapper.NAME) && docMapper.TTLFieldMapper().fieldType().stored()) {
} else if (field.equals(TTLFieldMapper.NAME) && docMapper.TTLFieldMapper().enabled()) {
// Call value for search with timestamp + ttl here to display the live remaining ttl value and be consistent with the search result display
if (source.ttl > 0) {
value = docMapper.TTLFieldMapper().valueForSearch(source.timestamp + source.ttl);
@ -255,7 +264,6 @@ public final class ShardGetService extends AbstractIndexShardComponent {
}
}
}
}
// deal with source, but only if it's enabled (we always have it from the translog)
BytesReference sourceToBeReturned = null;
@ -408,7 +416,7 @@ public final class ShardGetService extends AbstractIndexShardComponent {
private static FieldsVisitor buildFieldsVisitors(String[] fields, FetchSourceContext fetchSourceContext) {
if (fields == null || fields.length == 0) {
return fetchSourceContext.fetchSource() ? new JustSourceFieldsVisitor() : null;
return fetchSourceContext.fetchSource() ? new FieldsVisitor(true) : null;
}
return new CustomFieldsVisitor(Sets.newHashSet(fields), fetchSourceContext.fetchSource());

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.percolator;
import com.google.common.collect.Maps;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Query;
@ -29,8 +30,7 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fieldvisitor.JustSourceFieldsVisitor;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.fieldvisitor.FieldsVisitor;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Uid;
@ -44,7 +44,7 @@ import java.util.Map;
final class QueriesLoaderCollector extends SimpleCollector {
private final Map<BytesRef, Query> queries = Maps.newHashMap();
private final JustSourceFieldsVisitor fieldsVisitor = new JustSourceFieldsVisitor();
private final FieldsVisitor fieldsVisitor = new FieldsVisitor(true);
private final PercolatorQueriesRegistry percolator;
private final IndexFieldData<?> uidFieldData;
private final ESLogger logger;

View File

@ -46,7 +46,6 @@ import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.get.GetField;
import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
@ -221,7 +220,7 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
/* generate term vectors from fetched document fields */
GetResult getResult = indexShard.getService().get(
get, request.id(), request.type(), validFields.toArray(Strings.EMPTY_ARRAY), null, false);
Fields generatedTermVectors = generateTermVectors(getResult.getFields().values(), request.offsets(), request.perFieldAnalyzer());
Fields generatedTermVectors = generateTermVectors(getResult.getFields().values(), request.offsets(), request.perFieldAnalyzer(), validFields);
/* merge with existing Fields */
if (termVectorsByField == null) {
@ -255,12 +254,16 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
return selectedFields;
}
private Fields generateTermVectors(Collection<GetField> getFields, boolean withOffsets, @Nullable Map<String, String> perFieldAnalyzer)
private Fields generateTermVectors(Collection<GetField> getFields, boolean withOffsets, @Nullable Map<String, String> perFieldAnalyzer, Set<String> fields)
throws IOException {
/* store document in memory index */
MemoryIndex index = new MemoryIndex(withOffsets);
for (GetField getField : getFields) {
String field = getField.getName();
if (fields.contains(field) == false) {
// some fields are returned even when not asked for, eg. _timestamp
continue;
}
Analyzer analyzer = getAnalyzerAtField(field, perFieldAnalyzer);
for (Object text : getField.getValues()) {
index.addField(field, text.toString(), analyzer);
@ -276,16 +279,10 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
// select the right fields and generate term vectors
ParseContext.Document doc = parsedDocument.rootDoc();
Collection<String> seenFields = new HashSet<>();
Set<String> seenFields = new HashSet<>();
Collection<GetField> getFields = new HashSet<>();
for (IndexableField field : doc.getFields()) {
MappedFieldType fieldType = indexShard.mapperService().smartNameFieldType(field.name());
if (seenFields.contains(field.name())) {
continue;
}
else {
seenFields.add(field.name());
}
if (!isValidField(fieldType)) {
continue;
}
@ -295,10 +292,16 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
if (request.selectedFields() != null && !request.selectedFields().contains(field.name())) {
continue;
}
if (seenFields.contains(field.name())) {
continue;
}
else {
seenFields.add(field.name());
}
String[] values = doc.getValues(field.name());
getFields.add(new GetField(field.name(), Arrays.asList((Object[]) values)));
}
return generateTermVectors(getFields, request.offsets(), request.perFieldAnalyzer());
return generateTermVectors(getFields, request.offsets(), request.perFieldAnalyzer(), seenFields);
}
private ParsedDocument parseDocument(String index, String type, BytesReference doc) throws Throwable {

View File

@ -40,7 +40,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fieldvisitor.UidAndRoutingFieldsVisitor;
import org.elasticsearch.index.fieldvisitor.FieldsVisitor;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
@ -252,7 +252,7 @@ public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLServ
@Override
public void collect(int doc) {
try {
UidAndRoutingFieldsVisitor fieldsVisitor = new UidAndRoutingFieldsVisitor();
FieldsVisitor fieldsVisitor = new FieldsVisitor(false);
context.reader().document(doc, fieldsVisitor);
Uid uid = fieldsVisitor.uid();
final long version = Versions.loadVersion(context.reader(), new Term(UidFieldMapper.NAME, uid.toBytesRef()));

View File

@ -42,7 +42,6 @@ import org.elasticsearch.index.fieldvisitor.AllFieldsVisitor;
import org.elasticsearch.index.fieldvisitor.CustomFieldsVisitor;
import org.elasticsearch.index.fieldvisitor.FieldsVisitor;
import org.elasticsearch.index.fieldvisitor.JustUidFieldsVisitor;
import org.elasticsearch.index.fieldvisitor.UidAndSourceFieldsVisitor;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
@ -120,13 +119,9 @@ public class FetchPhase implements SearchPhase {
if (!context.hasScriptFields() && !context.hasFetchSourceContext()) {
context.fetchSourceContext(new FetchSourceContext(true));
}
fieldsVisitor = context.sourceRequested() ? new UidAndSourceFieldsVisitor() : new JustUidFieldsVisitor();
fieldsVisitor = new FieldsVisitor(context.sourceRequested());
} else if (context.fieldNames().isEmpty()) {
if (context.sourceRequested()) {
fieldsVisitor = new UidAndSourceFieldsVisitor();
} else {
fieldsVisitor = new JustUidFieldsVisitor();
}
fieldsVisitor = new FieldsVisitor(context.sourceRequested());
} else {
for (String fieldName : context.fieldNames()) {
if (fieldName.equals("*")) {
@ -164,10 +159,8 @@ public class FetchPhase implements SearchPhase {
} else if (fieldNames != null) {
boolean loadSource = extractFieldNames != null || context.sourceRequested();
fieldsVisitor = new CustomFieldsVisitor(fieldNames, loadSource);
} else if (extractFieldNames != null || context.sourceRequested()) {
fieldsVisitor = new UidAndSourceFieldsVisitor();
} else {
fieldsVisitor = new JustUidFieldsVisitor();
fieldsVisitor = new FieldsVisitor(extractFieldNames != null || context.sourceRequested());
}
}
@ -271,15 +264,10 @@ public class FetchPhase implements SearchPhase {
}
private InternalSearchHit createNestedSearchHit(SearchContext context, int nestedTopDocId, int nestedSubDocId, int rootSubDocId, List<String> extractFieldNames, boolean loadAllStored, Set<String> fieldNames, LeafReaderContext subReaderContext) throws IOException {
final FieldsVisitor rootFieldsVisitor;
if (context.sourceRequested() || extractFieldNames != null || context.highlight() != null) {
// Also if highlighting is requested on nested documents we need to fetch the _source from the root document,
// otherwise highlighting will attempt to fetch the _source from the nested doc, which will fail,
// because the entire _source is only stored with the root document.
rootFieldsVisitor = new UidAndSourceFieldsVisitor();
} else {
rootFieldsVisitor = new JustUidFieldsVisitor();
}
final FieldsVisitor rootFieldsVisitor = new FieldsVisitor(context.sourceRequested() || extractFieldNames != null || context.highlight() != null);
loadStoredFields(context, subReaderContext, rootFieldsVisitor, rootSubDocId);
rootFieldsVisitor.postProcess(context.mapperService());

View File

@ -294,19 +294,14 @@ public final class InnerHitsContext {
field = ParentFieldMapper.NAME;
term = Uid.createUid(hitContext.hit().type(), hitContext.hit().id());
} else if (isChildHit(hitContext.hit())) {
DocumentMapper hitDocumentMapper = mapperService.documentMapper(hitContext.hit().type());
final String parentType = hitDocumentMapper.parentFieldMapper().type();
field = UidFieldMapper.NAME;
SearchHitField parentField = hitContext.hit().field(ParentFieldMapper.NAME);
if (parentField != null) {
term = parentField.getValue();
} else {
SingleFieldsVisitor fieldsVisitor = new SingleFieldsVisitor(ParentFieldMapper.NAME);
hitContext.reader().document(hitContext.docId(), fieldsVisitor);
if (fieldsVisitor.fields().isEmpty()) {
return Lucene.EMPTY_TOP_DOCS;
if (parentField == null) {
throw new IllegalStateException("All children must have a _parent");
}
term = (String) fieldsVisitor.fields().get(ParentFieldMapper.NAME).get(0);
}
term = Uid.createUid(parentType, (String) parentField.getValue());
} else {
return Lucene.EMPTY_TOP_DOCS;
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.lookup;
import com.google.common.collect.ImmutableMap;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchParseException;
@ -27,7 +28,7 @@ import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.fieldvisitor.JustSourceFieldsVisitor;
import org.elasticsearch.index.fieldvisitor.FieldsVisitor;
import java.util.Collection;
import java.util.List;
@ -66,7 +67,7 @@ public class SourceLookup implements Map {
return source;
}
try {
JustSourceFieldsVisitor sourceFieldVisitor = new JustSourceFieldsVisitor();
FieldsVisitor sourceFieldVisitor = new FieldsVisitor(true);
reader.document(docId, sourceFieldVisitor);
BytesReference source = sourceFieldVisitor.source();
if (source == null) {

View File

@ -19,6 +19,8 @@
package org.elasticsearch.explain;
import com.google.common.collect.ImmutableSet;
import org.apache.lucene.search.Explanation;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.explain.ExplainResponse;
@ -26,6 +28,7 @@ import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.joda.time.DateTime;
@ -35,7 +38,9 @@ import org.junit.Test;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery;
@ -138,7 +143,9 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getExplanation().getValue(), equalTo(1.0f));
assertThat(response.getGetResult().isExists(), equalTo(true));
assertThat(response.getGetResult().getId(), equalTo("1"));
assertThat(response.getGetResult().getFields().size(), equalTo(1));
Set<String> fields = new HashSet<>(response.getGetResult().getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly added via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("obj1.field1")));
assertThat(response.getGetResult().getFields().get("obj1.field1").getValue().toString(), equalTo("value1"));
assertThat(response.getGetResult().isSourceEmpty(), equalTo(true));
@ -153,7 +160,9 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getExplanation().getValue(), equalTo(1.0f));
assertThat(response.getGetResult().isExists(), equalTo(true));
assertThat(response.getGetResult().getId(), equalTo("1"));
assertThat(response.getGetResult().getFields().size(), equalTo(1));
fields = new HashSet<>(response.getGetResult().getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly added via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("obj1.field1")));
assertThat(response.getGetResult().getFields().get("obj1.field1").getValue().toString(), equalTo("value1"));
assertThat(response.getGetResult().isSourceEmpty(), equalTo(false));

View File

@ -19,6 +19,8 @@
package org.elasticsearch.get;
import com.google.common.collect.ImmutableSet;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ShardOperationFailedException;
@ -37,12 +39,16 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.junit.Test;
import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -74,14 +80,18 @@ public class GetActionTests extends ElasticsearchIntegrationTest {
response = client().prepareGet(indexOrAlias(), "type1", "1").setFields(Strings.EMPTY_ARRAY).get();
assertThat(response.isExists(), equalTo(true));
assertThat(response.getIndex(), equalTo("test"));
assertThat(response.getFields().size(), equalTo(0));
Set<String> fields = new HashSet<>(response.getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo(Collections.<String>emptySet()));
assertThat(response.getSourceAsBytes(), nullValue());
logger.info("--> realtime get 1 (no source, explicit)");
response = client().prepareGet(indexOrAlias(), "type1", "1").setFetchSource(false).get();
assertThat(response.isExists(), equalTo(true));
assertThat(response.getIndex(), equalTo("test"));
assertThat(response.getFields().size(), equalTo(0));
fields = new HashSet<>(response.getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo(Collections.<String>emptySet()));
assertThat(response.getSourceAsBytes(), nullValue());
logger.info("--> realtime get 1 (no type)");
@ -362,7 +372,9 @@ public class GetActionTests extends ElasticsearchIntegrationTest {
assertThat(response.isExists(), equalTo(true));
assertThat(response.getId(), equalTo("1"));
assertThat(response.getType(), equalTo("type1"));
assertThat(response.getFields().size(), equalTo(1));
Set<String> fields = new HashSet<>(response.getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("field")));
assertThat(response.getFields().get("field").getValues().size(), equalTo(2));
assertThat(response.getFields().get("field").getValues().get(0).toString(), equalTo("1"));
assertThat(response.getFields().get("field").getValues().get(1).toString(), equalTo("2"));
@ -372,7 +384,9 @@ public class GetActionTests extends ElasticsearchIntegrationTest {
assertThat(response.isExists(), equalTo(true));
assertThat(response.getType(), equalTo("type2"));
assertThat(response.getId(), equalTo("1"));
assertThat(response.getFields().size(), equalTo(1));
fields = new HashSet<>(response.getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("field")));
assertThat(response.getFields().get("field").getValues().size(), equalTo(2));
assertThat(response.getFields().get("field").getValues().get(0).toString(), equalTo("1"));
assertThat(response.getFields().get("field").getValues().get(1).toString(), equalTo("2"));
@ -382,7 +396,9 @@ public class GetActionTests extends ElasticsearchIntegrationTest {
response = client().prepareGet("test", "type1", "1").setFields("field").get();
assertThat(response.isExists(), equalTo(true));
assertThat(response.getId(), equalTo("1"));
assertThat(response.getFields().size(), equalTo(1));
fields = new HashSet<>(response.getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("field")));
assertThat(response.getFields().get("field").getValues().size(), equalTo(2));
assertThat(response.getFields().get("field").getValues().get(0).toString(), equalTo("1"));
assertThat(response.getFields().get("field").getValues().get(1).toString(), equalTo("2"));
@ -390,7 +406,9 @@ public class GetActionTests extends ElasticsearchIntegrationTest {
response = client().prepareGet("test", "type2", "1").setFields("field").get();
assertThat(response.isExists(), equalTo(true));
assertThat(response.getId(), equalTo("1"));
assertThat(response.getFields().size(), equalTo(1));
fields = new HashSet<>(response.getFields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("field")));
assertThat(response.getFields().get("field").getValues().size(), equalTo(2));
assertThat(response.getFields().get("field").getValues().get(0).toString(), equalTo("1"));
assertThat(response.getFields().get("field").getValues().get(1).toString(), equalTo("2"));
@ -736,28 +754,42 @@ public class GetActionTests extends ElasticsearchIntegrationTest {
@Test
public void testGetFields_metaData() throws Exception {
assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
assertAcked(prepareCreate("test")
.addMapping("parent")
.addMapping("my-type1", "_timestamp", "enabled=true", "_ttl", "enabled=true", "_parent", "type=parent")
.addAlias(new Alias("alias"))
.setSettings(Settings.settingsBuilder().put("index.refresh_interval", -1)));
client().prepareIndex("test", "my-type1", "1")
.setRouting("1")
.setTimestamp("205097")
.setTTL(10000000000000L)
.setParent("parent_1")
.setSource(jsonBuilder().startObject().field("field1", "value").endObject())
.get();
GetResponse getResponse = client().prepareGet(indexOrAlias(), "my-type1", "1")
.setRouting("1")
.setFields("field1", "_routing")
.setFields("field1")
.get();
assertThat(getResponse.isExists(), equalTo(true));
assertThat(getResponse.getField("field1").isMetadataField(), equalTo(false));
assertThat(getResponse.getField("field1").getValue().toString(), equalTo("value"));
assertThat(getResponse.getField("_routing").isMetadataField(), equalTo(true));
assertThat(getResponse.getField("_routing").getValue().toString(), equalTo("1"));
assertThat(getResponse.getField("_timestamp").isMetadataField(), equalTo(true));
assertThat(getResponse.getField("_timestamp").getValue().toString(), equalTo("205097"));
assertThat(getResponse.getField("_ttl").isMetadataField(), equalTo(true));
// TODO: _ttl should return the original value, but it does not work today because
// it would use now() instead of the value of _timestamp to rebase
// assertThat(getResponse.getField("_ttl").getValue().toString(), equalTo("10000000205097"));
assertThat(getResponse.getField("_parent").isMetadataField(), equalTo(true));
assertThat(getResponse.getField("_parent").getValue().toString(), equalTo("parent_1"));
flush();
client().prepareGet(indexOrAlias(), "my-type1", "1")
.setFields("field1", "_routing")
getResponse = client().prepareGet(indexOrAlias(), "my-type1", "1")
.setFields("field1")
.setRouting("1")
.get();
assertThat(getResponse.isExists(), equalTo(true));
@ -765,6 +797,14 @@ public class GetActionTests extends ElasticsearchIntegrationTest {
assertThat(getResponse.getField("field1").getValue().toString(), equalTo("value"));
assertThat(getResponse.getField("_routing").isMetadataField(), equalTo(true));
assertThat(getResponse.getField("_routing").getValue().toString(), equalTo("1"));
assertThat(getResponse.getField("_timestamp").isMetadataField(), equalTo(true));
assertThat(getResponse.getField("_timestamp").getValue().toString(), equalTo("205097"));
assertThat(getResponse.getField("_ttl").isMetadataField(), equalTo(true));
// TODO: _ttl should return the original value, but it does not work today because
// it would use now() instead of the value of _timestamp to rebase
//assertThat(getResponse.getField("_ttl").getValue().toString(), equalTo("10000000000000"));
assertThat(getResponse.getField("_parent").isMetadataField(), equalTo(true));
assertThat(getResponse.getField("_parent").getValue().toString(), equalTo("parent_1"));
}
@Test

View File

@ -19,6 +19,8 @@
package org.elasticsearch.search.fields;
import com.google.common.collect.ImmutableSet;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
@ -32,6 +34,7 @@ import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
@ -46,8 +49,10 @@ import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import static org.elasticsearch.client.Requests.refreshRequest;
@ -58,10 +63,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFail
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.*;
/**
*
@ -167,17 +169,23 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(response.getHits().totalHits(), equalTo(3l));
assertThat(response.getHits().getAt(0).isSourceEmpty(), equalTo(true));
assertThat(response.getHits().getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).fields().size(), equalTo(3));
Set<String> fields = new HashSet<>(response.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("sNum1", "sNum1_field", "date1")));
assertThat((Double) response.getHits().getAt(0).fields().get("sNum1").values().get(0), equalTo(1.0));
assertThat((Double) response.getHits().getAt(0).fields().get("sNum1_field").values().get(0), equalTo(1.0));
assertThat((Long) response.getHits().getAt(0).fields().get("date1").values().get(0), equalTo(0l));
assertThat(response.getHits().getAt(1).id(), equalTo("2"));
assertThat(response.getHits().getAt(1).fields().size(), equalTo(3));
fields = new HashSet<>(response.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("sNum1", "sNum1_field", "date1")));
assertThat((Double) response.getHits().getAt(1).fields().get("sNum1").values().get(0), equalTo(2.0));
assertThat((Double) response.getHits().getAt(1).fields().get("sNum1_field").values().get(0), equalTo(2.0));
assertThat((Long) response.getHits().getAt(1).fields().get("date1").values().get(0), equalTo(25000l));
assertThat(response.getHits().getAt(2).id(), equalTo("3"));
assertThat(response.getHits().getAt(2).fields().size(), equalTo(3));
fields = new HashSet<>(response.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("sNum1", "sNum1_field", "date1")));
assertThat((Double) response.getHits().getAt(2).fields().get("sNum1").values().get(0), equalTo(3.0));
assertThat((Double) response.getHits().getAt(2).fields().get("sNum1_field").values().get(0), equalTo(3.0));
assertThat((Long) response.getHits().getAt(2).fields().get("date1").values().get(0), equalTo(120000l));
@ -192,13 +200,19 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(response.getHits().totalHits(), equalTo(3l));
assertThat(response.getHits().getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).fields().size(), equalTo(1));
fields = new HashSet<>(response.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("sNum1")));
assertThat((Double) response.getHits().getAt(0).fields().get("sNum1").values().get(0), equalTo(2.0));
assertThat(response.getHits().getAt(1).id(), equalTo("2"));
assertThat(response.getHits().getAt(1).fields().size(), equalTo(1));
fields = new HashSet<>(response.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("sNum1")));
assertThat((Double) response.getHits().getAt(1).fields().get("sNum1").values().get(0), equalTo(4.0));
assertThat(response.getHits().getAt(2).id(), equalTo("3"));
assertThat(response.getHits().getAt(2).fields().size(), equalTo(1));
fields = new HashSet<>(response.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("sNum1")));
assertThat((Double) response.getHits().getAt(2).fields().get("sNum1").values().get(0), equalTo(6.0));
}
@ -224,7 +238,9 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(response.getHits().totalHits(), equalTo((long)numDocs));
for (int i = 0; i < numDocs; i++) {
assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i)));
assertThat(response.getHits().getAt(i).fields().size(), equalTo(1));
Set<String> fields = new HashSet<>(response.getHits().getAt(i).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("uid")));
assertThat((String)response.getHits().getAt(i).fields().get("uid").value(), equalTo("type1#" + Integer.toString(i)));
}
@ -237,7 +253,9 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(response.getHits().totalHits(), equalTo((long)numDocs));
for (int i = 0; i < numDocs; i++) {
assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i)));
assertThat(response.getHits().getAt(i).fields().size(), equalTo(1));
Set<String> fields = new HashSet<>(response.getHits().getAt(i).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("id")));
assertThat((String)response.getHits().getAt(i).fields().get("id").value(), equalTo(Integer.toString(i)));
}
@ -250,7 +268,9 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(response.getHits().totalHits(), equalTo((long)numDocs));
for (int i = 0; i < numDocs; i++) {
assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i)));
assertThat(response.getHits().getAt(i).fields().size(), equalTo(1));
Set<String> fields = new HashSet<>(response.getHits().getAt(i).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("type")));
assertThat((String)response.getHits().getAt(i).fields().get("type").value(), equalTo("type1"));
}
@ -264,7 +284,9 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(response.getHits().totalHits(), equalTo((long)numDocs));
for (int i = 0; i < numDocs; i++) {
assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i)));
assertThat(response.getHits().getAt(i).fields().size(), equalTo(3));
Set<String> fields = new HashSet<>(response.getHits().getAt(i).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("uid", "type", "id")));
assertThat((String)response.getHits().getAt(i).fields().get("uid").value(), equalTo("type1#" + Integer.toString(i)));
assertThat((String)response.getHits().getAt(i).fields().get("type").value(), equalTo("type1"));
assertThat((String)response.getHits().getAt(i).fields().get("id").value(), equalTo(Integer.toString(i)));
@ -383,7 +405,10 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(1l));
assertThat(searchResponse.getHits().hits().length, equalTo(1));
assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(9));
Set<String> fields = new HashSet<>(searchResponse.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("byte_field", "short_field", "integer_field", "long_field",
"float_field", "double_field", "date_field", "boolean_field", "binary_field")));
assertThat(searchResponse.getHits().getAt(0).fields().get("byte_field").value().toString(), equalTo("1"));
@ -559,7 +584,10 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(1l));
assertThat(searchResponse.getHits().hits().length, equalTo(1));
assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(9));
Set<String> fields = new HashSet<>(searchResponse.getHits().getAt(0).fields().keySet());
fields.remove(TimestampFieldMapper.NAME); // randomly enabled via templates
assertThat(fields, equalTo((Set<String>) ImmutableSet.of("byte_field", "short_field", "integer_field", "long_field",
"float_field", "double_field", "date_field", "boolean_field", "string_field")));
assertThat(searchResponse.getHits().getAt(0).fields().get("byte_field").value().toString(), equalTo("1"));
assertThat(searchResponse.getHits().getAt(0).fields().get("short_field").value().toString(), equalTo("2"));
@ -610,4 +638,37 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(fields.get("md").getValues(), equalTo(Arrays.<Object> asList((double) id, id + 1d)));
}
}
public void testLoadMetadata() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("parent")
.addMapping("my-type1", "_timestamp", "enabled=true", "_ttl", "enabled=true", "_parent", "type=parent"));
indexRandom(true,
client().prepareIndex("test", "my-type1", "1")
.setRouting("1")
.setTimestamp("205097")
.setTTL(10000000000000L)
.setParent("parent_1")
.setSource(jsonBuilder().startObject().field("field1", "value").endObject()));
SearchResponse response = client().prepareSearch("test").addField("field1").get();
assertSearchResponse(response);
assertHitCount(response, 1);
Map<String, SearchHitField> fields = response.getHits().getAt(0).getFields();
assertThat(fields.get("field1").isMetadataField(), equalTo(false));
assertThat(fields.get("field1").getValue().toString(), equalTo("value"));
assertThat(fields.get("_routing").isMetadataField(), equalTo(true));
assertThat(fields.get("_routing").getValue().toString(), equalTo("1"));
assertThat(fields.get("_timestamp").isMetadataField(), equalTo(true));
assertThat(fields.get("_timestamp").getValue().toString(), equalTo("205097"));
assertThat(fields.get("_ttl").isMetadataField(), equalTo(true));
// TODO: _ttl should return the original value, but it does not work today because
// it would use now() instead of the value of _timestamp to rebase
// assertThat(fields.get("_ttl").getValue().toString(), equalTo("10000000205097"));
assertThat(fields.get("_parent").isMetadataField(), equalTo(true));
assertThat(fields.get("_parent").getValue().toString(), equalTo("parent_1"));
}
}