Mappings: Consolidate document parsing logic

The code to parse a document was spread across 3 different classes,
and depended on traversing the ObjectMapper hiearchy. This change
consolidates all the doc parsing code into a new DocumentParser.
This should allow adding unit tests (future issue) for document
parsing so the logic can be simplified.  All code was copied
directly for this change with only minor modifications to make
it work within the new location.

closes #10802
This commit is contained in:
Ryan Ernst 2015-04-25 00:15:48 -07:00
parent f6d8b12796
commit 99584deae8
13 changed files with 740 additions and 705 deletions

View File

@ -21,15 +21,10 @@ package org.elasticsearch.index.mapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Filter;
import org.apache.lucene.util.BitDocIdSet;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.common.Nullable;
@ -46,8 +41,6 @@ import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.mapper.Mapping.SourceTransform;
@ -75,12 +68,10 @@ import org.elasticsearch.script.ScriptService.ScriptType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
/**
@ -172,27 +163,15 @@ public class DocumentMapper implements ToXContent {
}
}
private CloseableThreadLocal<ParseContext.InternalParseContext> cache = new CloseableThreadLocal<ParseContext.InternalParseContext>() {
@Override
protected ParseContext.InternalParseContext initialValue() {
return new ParseContext.InternalParseContext(index, indexSettings, docMapperParser, DocumentMapper.this, new ContentPath(0));
}
};
private final String index;
private final Settings indexSettings;
private final String type;
private final StringAndBytesText typeText;
private final DocumentMapperParser docMapperParser;
private volatile CompressedString mappingSource;
private final Mapping mapping;
private final DocumentParser documentParser;
private volatile DocumentFieldMappers fieldMappers;
private volatile ImmutableMap<String, ObjectMapper> objectMappers = ImmutableMap.of();
@ -211,16 +190,14 @@ public class DocumentMapper implements ToXContent {
RootObjectMapper rootObjectMapper,
ImmutableMap<String, Object> meta,
Map<Class<? extends RootMapper>, RootMapper> rootMappers, List<SourceTransform> sourceTransforms) {
this.index = index;
this.indexSettings = indexSettings;
this.type = rootObjectMapper.name();
this.typeText = new StringAndBytesText(this.type);
this.docMapperParser = docMapperParser;
this.mapping = new Mapping(
rootObjectMapper,
rootMappers.values().toArray(new RootMapper[rootMappers.values().size()]),
sourceTransforms.toArray(new SourceTransform[sourceTransforms.size()]),
meta);
this.documentParser = new DocumentParser(index, indexSettings, docMapperParser, this);
this.typeFilter = typeMapper().termFilter(type, null);
@ -364,109 +341,13 @@ public class DocumentMapper implements ToXContent {
}
public ParsedDocument parse(SourceToParse source) throws MapperParsingException {
return parse(source, null);
return documentParser.parseDocument(source, null);
}
// NOTE: do not use this method, it will be removed in the future once
// https://github.com/elastic/elasticsearch/issues/10736 is done (MLT api is the only user of this listener)
public ParsedDocument parse(SourceToParse source, @Nullable ParseListener listener) throws MapperParsingException {
ParseContext.InternalParseContext context = cache.get();
if (source.type() != null && !source.type().equals(this.type)) {
throw new MapperParsingException("Type mismatch, provide type [" + source.type() + "] but mapper is of type [" + this.type + "]");
}
source.type(this.type);
XContentParser parser = source.parser();
try {
if (parser == null) {
parser = XContentHelper.createParser(source.source());
}
if (mapping.sourceTransforms.length > 0) {
parser = transform(parser);
}
context.reset(parser, new ParseContext.Document(), source, listener);
// will result in START_OBJECT
int countDownTokens = 0;
XContentParser.Token token = parser.nextToken();
if (token != XContentParser.Token.START_OBJECT) {
throw new MapperParsingException("Malformed content, must start with an object");
}
boolean emptyDoc = false;
token = parser.nextToken();
if (token == XContentParser.Token.END_OBJECT) {
// empty doc, we can handle it...
emptyDoc = true;
} else if (token != XContentParser.Token.FIELD_NAME) {
throw new MapperParsingException("Malformed content, after first object, either the type field or the actual properties should exist");
}
for (RootMapper rootMapper : mapping.rootMappers) {
rootMapper.preParse(context);
}
if (!emptyDoc) {
Mapper update = mapping.root.parse(context);
if (update != null) {
context.addDynamicMappingsUpdate((RootObjectMapper) update);
}
}
for (int i = 0; i < countDownTokens; i++) {
parser.nextToken();
}
for (RootMapper rootMapper : mapping.rootMappers) {
rootMapper.postParse(context);
}
} catch (Throwable e) {
// if its already a mapper parsing exception, no need to wrap it...
if (e instanceof MapperParsingException) {
throw (MapperParsingException) e;
}
// Throw a more meaningful message if the document is empty.
if (source.source() != null && source.source().length() == 0) {
throw new MapperParsingException("failed to parse, document is empty");
}
throw new MapperParsingException("failed to parse", e);
} finally {
// only close the parser when its not provided externally
if (source.parser() == null && parser != null) {
parser.close();
}
}
// reverse the order of docs for nested docs support, parent should be last
if (context.docs().size() > 1) {
Collections.reverse(context.docs());
}
// apply doc boost
if (context.docBoost() != 1.0f) {
Set<String> encounteredFields = Sets.newHashSet();
for (ParseContext.Document doc : context.docs()) {
encounteredFields.clear();
for (IndexableField field : doc) {
if (field.fieldType().indexOptions() != IndexOptions.NONE && !field.fieldType().omitNorms()) {
if (!encounteredFields.contains(field.name())) {
((Field) field).setBoost(context.docBoost() * field.boost());
encounteredFields.add(field.name());
}
}
}
}
}
Mapper rootDynamicUpdate = context.dynamicMappingsUpdate();
Mapping update = null;
if (rootDynamicUpdate != null) {
update = mapping.mappingUpdate(rootDynamicUpdate);
}
ParsedDocument doc = new ParsedDocument(context.uid(), context.version(), context.id(), context.type(), source.routing(), source.timestamp(), source.ttl(), context.docs(),
context.source(), update).parent(source.parent());
// reset the context to free up memory
context.reset(null, null, null, null);
return doc;
return documentParser.parseDocument(source, listener);
}
/**
@ -514,19 +395,7 @@ public class DocumentMapper implements ToXContent {
* @return transformed version of transformMe. This may actually be the same object as sourceAsMap
*/
public Map<String, Object> transformSourceAsMap(Map<String, Object> sourceAsMap) {
if (mapping.sourceTransforms.length == 0) {
return sourceAsMap;
}
for (SourceTransform transform : mapping.sourceTransforms) {
sourceAsMap = transform.transformSourceAsMap(sourceAsMap);
}
return sourceAsMap;
}
private XContentParser transform(XContentParser parser) throws IOException {
Map<String, Object> transformed = transformSourceAsMap(parser.mapOrderedAndClose());
XContentBuilder builder = XContentFactory.contentBuilder(parser.contentType()).value(transformed);
return parser.contentType().xContent().createParser(builder.bytes());
return DocumentParser.transformSourceAsMap(mapping, sourceAsMap);
}
public void addFieldMappers(Collection<FieldMapper<?>> fieldMappers) {
@ -638,7 +507,7 @@ public class DocumentMapper implements ToXContent {
}
public void close() {
cache.close();
documentParser.close();
mapping.root.close();
for (RootMapper rootMapper : mapping.rootMappers) {
rootMapper.close();

View File

@ -0,0 +1,701 @@
/*
* 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.mapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Sets;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.ArrayValueMapperParser;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
/** A parser for documents, given mappings from a DocumentMapper */
class DocumentParser implements Closeable {
private CloseableThreadLocal<ParseContext.InternalParseContext> cache = new CloseableThreadLocal<ParseContext.InternalParseContext>() {
@Override
protected ParseContext.InternalParseContext initialValue() {
return new ParseContext.InternalParseContext(index, indexSettings, docMapperParser, docMapper, new ContentPath(0));
}
};
private String index;
private Settings indexSettings;
private DocumentMapperParser docMapperParser;
private DocumentMapper docMapper;
public DocumentParser(String index, Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper) {
this.index = index;
this.indexSettings = indexSettings;
this.docMapperParser = docMapperParser;
this.docMapper = docMapper;
}
public ParsedDocument parseDocument(SourceToParse source, @Nullable DocumentMapper.ParseListener listener) throws MapperParsingException {
ParseContext.InternalParseContext context = cache.get();
final Mapping mapping = docMapper.mapping();
if (source.type() != null && !source.type().equals(docMapper.type())) {
throw new MapperParsingException("Type mismatch, provide type [" + source.type() + "] but mapper is of type [" + docMapper.type() + "]");
}
source.type(docMapper.type());
XContentParser parser = source.parser();
try {
if (parser == null) {
parser = XContentHelper.createParser(source.source());
}
if (mapping.sourceTransforms.length > 0) {
parser = transform(mapping, parser);
}
context.reset(parser, new ParseContext.Document(), source, listener);
// will result in START_OBJECT
int countDownTokens = 0;
XContentParser.Token token = parser.nextToken();
if (token != XContentParser.Token.START_OBJECT) {
throw new MapperParsingException("Malformed content, must start with an object");
}
boolean emptyDoc = false;
token = parser.nextToken();
if (token == XContentParser.Token.END_OBJECT) {
// empty doc, we can handle it...
emptyDoc = true;
} else if (token != XContentParser.Token.FIELD_NAME) {
throw new MapperParsingException("Malformed content, after first object, either the type field or the actual properties should exist");
}
for (RootMapper rootMapper : mapping.rootMappers) {
rootMapper.preParse(context);
}
if (!emptyDoc) {
Mapper update = parseObject(context, mapping.root);
if (update != null) {
context.addDynamicMappingsUpdate(update);
}
}
for (int i = 0; i < countDownTokens; i++) {
parser.nextToken();
}
for (RootMapper rootMapper : mapping.rootMappers) {
rootMapper.postParse(context);
}
} catch (Throwable e) {
// if its already a mapper parsing exception, no need to wrap it...
if (e instanceof MapperParsingException) {
throw (MapperParsingException) e;
}
// Throw a more meaningful message if the document is empty.
if (source.source() != null && source.source().length() == 0) {
throw new MapperParsingException("failed to parse, document is empty");
}
throw new MapperParsingException("failed to parse", e);
} finally {
// only close the parser when its not provided externally
if (source.parser() == null && parser != null) {
parser.close();
}
}
// reverse the order of docs for nested docs support, parent should be last
if (context.docs().size() > 1) {
Collections.reverse(context.docs());
}
// apply doc boost
if (context.docBoost() != 1.0f) {
Set<String> encounteredFields = Sets.newHashSet();
for (ParseContext.Document doc : context.docs()) {
encounteredFields.clear();
for (IndexableField field : doc) {
if (field.fieldType().indexOptions() != IndexOptions.NONE && !field.fieldType().omitNorms()) {
if (!encounteredFields.contains(field.name())) {
((Field) field).setBoost(context.docBoost() * field.boost());
encounteredFields.add(field.name());
}
}
}
}
}
Mapper rootDynamicUpdate = context.dynamicMappingsUpdate();
Mapping update = null;
if (rootDynamicUpdate != null) {
update = mapping.mappingUpdate(rootDynamicUpdate);
}
ParsedDocument doc = new ParsedDocument(context.uid(), context.version(), context.id(), context.type(), source.routing(), source.timestamp(), source.ttl(), context.docs(),
context.source(), update).parent(source.parent());
// reset the context to free up memory
context.reset(null, null, null, null);
return doc;
}
static ObjectMapper parseObject(ParseContext context, ObjectMapper mapper) throws IOException {
if (mapper.isEnabled() == false) {
context.parser().skipChildren();
return null;
}
XContentParser parser = context.parser();
String currentFieldName = parser.currentName();
XContentParser.Token token = parser.currentToken();
if (token == XContentParser.Token.VALUE_NULL) {
// the object is null ("obj1" : null), simply bail
return null;
}
if (token.isValue()) {
throw new MapperParsingException("object mapping for [" + mapper.name() + "] tried to parse field [" + currentFieldName + "] as object, but found a concrete value");
}
ObjectMapper.Nested nested = mapper.nested();
if (nested.isNested()) {
context = context.createNestedContext(mapper.fullPath());
ParseContext.Document nestedDoc = context.doc();
ParseContext.Document parentDoc = nestedDoc.getParent();
// pre add the uid field if possible (id was already provided)
IndexableField uidField = parentDoc.getField(UidFieldMapper.NAME);
if (uidField != null) {
// we don't need to add it as a full uid field in nested docs, since we don't need versioning
// we also rely on this for UidField#loadVersion
// this is a deeply nested field
nestedDoc.add(new Field(UidFieldMapper.NAME, uidField.stringValue(), UidFieldMapper.Defaults.NESTED_FIELD_TYPE));
}
// the type of the nested doc starts with __, so we can identify that its a nested one in filters
// note, we don't prefix it with the type of the doc since it allows us to execute a nested query
// across types (for example, with similar nested objects)
nestedDoc.add(new Field(TypeFieldMapper.NAME, mapper.nestedTypePathAsString(), TypeFieldMapper.Defaults.FIELD_TYPE));
}
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(mapper.pathType());
// if we are at the end of the previous object, advance
if (token == XContentParser.Token.END_OBJECT) {
token = parser.nextToken();
}
if (token == XContentParser.Token.START_OBJECT) {
// if we are just starting an OBJECT, advance, this is the object we are parsing, we need the name first
token = parser.nextToken();
}
ObjectMapper update = null;
while (token != XContentParser.Token.END_OBJECT) {
ObjectMapper newUpdate = null;
if (token == XContentParser.Token.START_OBJECT) {
newUpdate = parseObject(context, mapper, currentFieldName);
} else if (token == XContentParser.Token.START_ARRAY) {
newUpdate = parseArray(context, mapper, currentFieldName);
} else if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
parseNullValue(context, mapper, currentFieldName);
} else if (token == null) {
throw new MapperParsingException("object mapping for [" + mapper.name() + "] tried to parse field [" + currentFieldName + "] as object, but got EOF, has a concrete value been provided to it?");
} else if (token.isValue()) {
newUpdate = parseValue(context, mapper, currentFieldName, token);
}
token = parser.nextToken();
if (newUpdate != null) {
if (update == null) {
update = newUpdate;
} else {
MapperUtils.merge(update, newUpdate);
}
}
}
// restore the enable path flag
context.path().pathType(origPathType);
if (nested.isNested()) {
ParseContext.Document nestedDoc = context.doc();
ParseContext.Document parentDoc = nestedDoc.getParent();
if (nested.isIncludeInParent()) {
for (IndexableField field : nestedDoc.getFields()) {
if (field.name().equals(UidFieldMapper.NAME) || field.name().equals(TypeFieldMapper.NAME)) {
continue;
} else {
parentDoc.add(field);
}
}
}
if (nested.isIncludeInRoot()) {
ParseContext.Document rootDoc = context.rootDoc();
// don't add it twice, if its included in parent, and we are handling the master doc...
if (!nested.isIncludeInParent() || parentDoc != rootDoc) {
for (IndexableField field : nestedDoc.getFields()) {
if (field.name().equals(UidFieldMapper.NAME) || field.name().equals(TypeFieldMapper.NAME)) {
continue;
} else {
rootDoc.add(field);
}
}
}
}
}
return update;
}
private static Mapper parseObjectOrField(ParseContext context, Mapper mapper) throws IOException {
if (mapper instanceof ObjectMapper) {
return parseObject(context, (ObjectMapper) mapper);
} else {
FieldMapper fieldMapper = (FieldMapper)mapper;
Mapper update = fieldMapper.parse(context);
if (fieldMapper.copyTo() != null) {
parseCopyFields(context, fieldMapper, fieldMapper.copyTo().copyToFields());
}
return update;
}
}
private static ObjectMapper parseObject(final ParseContext context, ObjectMapper mapper, String currentFieldName) throws IOException {
if (currentFieldName == null) {
throw new MapperParsingException("object mapping [" + mapper.name() + "] trying to serialize an object with no field associated with it, current value [" + context.parser().textOrNull() + "]");
}
context.path().add(currentFieldName);
ObjectMapper update = null;
Mapper objectMapper = mapper.getMapper(currentFieldName);
if (objectMapper != null) {
final Mapper subUpdate = parseObjectOrField(context, objectMapper);
if (subUpdate != null) {
// propagate mapping update
update = mapper.mappingUpdate(subUpdate);
}
} else {
ObjectMapper.Dynamic dynamic = mapper.dynamic();
if (dynamic == null) {
dynamic = dynamicOrDefault(context.root().dynamic());
}
if (dynamic == ObjectMapper.Dynamic.STRICT) {
throw new StrictDynamicMappingException(mapper.fullPath(), currentFieldName);
} else if (dynamic == ObjectMapper.Dynamic.TRUE) {
// remove the current field name from path, since template search and the object builder add it as well...
context.path().remove();
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "object");
if (builder == null) {
builder = MapperBuilders.object(currentFieldName).enabled(true).pathType(mapper.pathType());
// if this is a non root object, then explicitly set the dynamic behavior if set
if (!(mapper instanceof RootObjectMapper) && mapper.dynamic() != ObjectMapper.Defaults.DYNAMIC) {
((ObjectMapper.Builder) builder).dynamic(mapper.dynamic());
}
}
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
objectMapper = builder.build(builderContext);
context.path().add(currentFieldName);
update = mapper.mappingUpdate(parseAndMergeUpdate(objectMapper, context));
} else {
// not dynamic, read everything up to end object
context.parser().skipChildren();
}
}
context.path().remove();
return update;
}
private static ObjectMapper parseArray(ParseContext context, ObjectMapper parentMapper, String lastFieldName) throws IOException {
String arrayFieldName = lastFieldName;
Mapper mapper = parentMapper.getMapper(lastFieldName);
if (mapper != null) {
// There is a concrete mapper for this field already. Need to check if the mapper
// expects an array, if so we pass the context straight to the mapper and if not
// we serialize the array components
if (mapper instanceof ArrayValueMapperParser) {
final Mapper subUpdate = parseObjectOrField(context, mapper);
if (subUpdate != null) {
// propagate the mapping update
return parentMapper.mappingUpdate(subUpdate);
} else {
return null;
}
} else {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
}
} else {
ObjectMapper.Dynamic dynamic = parentMapper.dynamic();
if (dynamic == null) {
dynamic = dynamicOrDefault(context.root().dynamic());
}
if (dynamic == ObjectMapper.Dynamic.STRICT) {
throw new StrictDynamicMappingException(parentMapper.fullPath(), arrayFieldName);
} else if (dynamic == ObjectMapper.Dynamic.TRUE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, arrayFieldName, "object");
if (builder == null) {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
}
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
mapper = builder.build(builderContext);
if (mapper != null && mapper instanceof ArrayValueMapperParser) {
context.path().add(arrayFieldName);
mapper = parseAndMergeUpdate(mapper, context);
return parentMapper.mappingUpdate(mapper);
} else {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
}
} else {
return parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
}
}
}
private static ObjectMapper parseNonDynamicArray(ParseContext context, ObjectMapper mapper, String lastFieldName, String arrayFieldName) throws IOException {
XContentParser parser = context.parser();
XContentParser.Token token;
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.START_OBJECT) {
return parseObject(context, mapper, lastFieldName);
} else if (token == XContentParser.Token.START_ARRAY) {
return parseArray(context, mapper, lastFieldName);
} else if (token == XContentParser.Token.FIELD_NAME) {
lastFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
parseNullValue(context, mapper, lastFieldName);
} else if (token == null) {
throw new MapperParsingException("object mapping for [" + mapper.name() + "] with array for [" + arrayFieldName + "] tried to parse as array, but got EOF, is there a mismatch in types for the same field?");
} else {
return parseValue(context, mapper, lastFieldName, token);
}
}
return null;
}
private static ObjectMapper parseValue(final ParseContext context, ObjectMapper parentMapper, String currentFieldName, XContentParser.Token token) throws IOException {
if (currentFieldName == null) {
throw new MapperParsingException("object mapping [" + parentMapper.name() + "] trying to serialize a value with no field associated with it, current value [" + context.parser().textOrNull() + "]");
}
Mapper mapper = parentMapper.getMapper(currentFieldName);
if (mapper != null) {
Mapper subUpdate = parseObjectOrField(context, mapper);
if (subUpdate == null) {
return null;
}
return parentMapper.mappingUpdate(subUpdate);
} else {
return parseDynamicValue(context, parentMapper, currentFieldName, token);
}
}
private static void parseNullValue(ParseContext context, ObjectMapper parentMapper, String lastFieldName) throws IOException {
// we can only handle null values if we have mappings for them
Mapper mapper = parentMapper.getMapper(lastFieldName);
if (mapper != null) {
if (mapper instanceof FieldMapper) {
if (!((FieldMapper) mapper).supportsNullValue()) {
throw new MapperParsingException("no object mapping found for null value in [" + lastFieldName + "]");
}
}
parseObjectOrField(context, mapper);
} else if (parentMapper.dynamic() == ObjectMapper.Dynamic.STRICT) {
throw new StrictDynamicMappingException(parentMapper.fullPath(), lastFieldName);
}
}
private static ObjectMapper parseDynamicValue(final ParseContext context, ObjectMapper parentMapper, String currentFieldName, XContentParser.Token token) throws IOException {
ObjectMapper.Dynamic dynamic = parentMapper.dynamic();
if (dynamic == null) {
dynamic = dynamicOrDefault(context.root().dynamic());
}
if (dynamic == ObjectMapper.Dynamic.STRICT) {
throw new StrictDynamicMappingException(parentMapper.fullPath(), currentFieldName);
}
if (dynamic == ObjectMapper.Dynamic.FALSE) {
return null;
}
Mapper mapper = null;
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
if (token == XContentParser.Token.VALUE_STRING) {
boolean resolved = false;
// do a quick test to see if its fits a dynamic template, if so, use it.
// we need to do it here so we can handle things like attachment templates, where calling
// text (to see if its a date) causes the binary value to be cleared
if (!resolved) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "string", null);
if (builder != null) {
mapper = builder.build(builderContext);
resolved = true;
}
}
if (!resolved && context.root().dateDetection()) {
String text = context.parser().text();
// a safe check since "1" gets parsed as well
if (Strings.countOccurrencesOf(text, ":") > 1 || Strings.countOccurrencesOf(text, "-") > 1 || Strings.countOccurrencesOf(text, "/") > 1) {
for (FormatDateTimeFormatter dateTimeFormatter : context.root().dynamicDateTimeFormatters()) {
try {
dateTimeFormatter.parser().parseMillis(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "date");
if (builder == null) {
builder = MapperBuilders.dateField(currentFieldName).dateTimeFormatter(dateTimeFormatter);
}
mapper = builder.build(builderContext);
resolved = true;
break;
} catch (Exception e) {
// failure to parse this, continue
}
}
}
}
if (!resolved && context.root().numericDetection()) {
String text = context.parser().text();
try {
Long.parseLong(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
if (builder == null) {
builder = MapperBuilders.longField(currentFieldName);
}
mapper = builder.build(builderContext);
resolved = true;
} catch (Exception e) {
// not a long number
}
if (!resolved) {
try {
Double.parseDouble(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
if (builder == null) {
builder = MapperBuilders.doubleField(currentFieldName);
}
mapper = builder.build(builderContext);
resolved = true;
} catch (Exception e) {
// not a long number
}
}
}
if (!resolved) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "string");
if (builder == null) {
builder = MapperBuilders.stringField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (token == XContentParser.Token.VALUE_NUMBER) {
XContentParser.NumberType numberType = context.parser().numberType();
if (numberType == XContentParser.NumberType.INT) {
if (context.parser().estimatedNumberType()) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
if (builder == null) {
builder = MapperBuilders.longField(currentFieldName);
}
mapper = builder.build(builderContext);
} else {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "integer");
if (builder == null) {
builder = MapperBuilders.integerField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (numberType == XContentParser.NumberType.LONG) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
if (builder == null) {
builder = MapperBuilders.longField(currentFieldName);
}
mapper = builder.build(builderContext);
} else if (numberType == XContentParser.NumberType.FLOAT) {
if (context.parser().estimatedNumberType()) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
if (builder == null) {
builder = MapperBuilders.doubleField(currentFieldName);
}
mapper = builder.build(builderContext);
} else {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "float");
if (builder == null) {
builder = MapperBuilders.floatField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (numberType == XContentParser.NumberType.DOUBLE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
if (builder == null) {
builder = MapperBuilders.doubleField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "boolean");
if (builder == null) {
builder = MapperBuilders.booleanField(currentFieldName);
}
mapper = builder.build(builderContext);
} else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "binary");
if (builder == null) {
builder = MapperBuilders.binaryField(currentFieldName);
}
mapper = builder.build(builderContext);
} else {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, null);
if (builder != null) {
mapper = builder.build(builderContext);
} else {
// TODO how do we identify dynamically that its a binary value?
throw new ElasticsearchIllegalStateException("Can't handle serializing a dynamic type with content token [" + token + "] and field name [" + currentFieldName + "]");
}
}
mapper = parseAndMergeUpdate(mapper, context);
ObjectMapper update = null;
if (mapper != null) {
update = parentMapper.mappingUpdate(mapper);
}
return update;
}
/** Creates instances of the fields that the current field should be copied to */
private static void parseCopyFields(ParseContext context, FieldMapper<?> fieldMapper, ImmutableList<String> copyToFields) throws IOException {
if (!context.isWithinCopyTo() && copyToFields.isEmpty() == false) {
context = context.createCopyToContext();
for (String field : copyToFields) {
// In case of a hierarchy of nested documents, we need to figure out
// which document the field should go to
ParseContext.Document targetDoc = null;
for (ParseContext.Document doc = context.doc(); doc != null; doc = doc.getParent()) {
if (field.startsWith(doc.getPrefix())) {
targetDoc = doc;
break;
}
}
assert targetDoc != null;
final ParseContext copyToContext;
if (targetDoc == context.doc()) {
copyToContext = context;
} else {
copyToContext = context.switchDoc(targetDoc);
}
parseCopy(field, copyToContext);
}
}
}
/** Creates an copy of the current field with given field name and boost */
private static void parseCopy(String field, ParseContext context) throws IOException {
// TODO: this should not be indexName...
FieldMappers mappers = context.docMapper().mappers().indexName(field);
if (mappers != null && !mappers.isEmpty()) {
mappers.mapper().parse(context);
} else {
// The path of the dest field might be completely different from the current one so we need to reset it
context = context.overridePath(new ContentPath(0));
ObjectMapper mapper = context.root();
String objectPath = "";
String fieldPath = field;
int posDot = field.lastIndexOf('.');
if (posDot > 0) {
objectPath = field.substring(0, posDot);
context.path().add(objectPath);
mapper = context.docMapper().objectMappers().get(objectPath);
fieldPath = field.substring(posDot + 1);
}
if (mapper == null) {
//TODO: Create an object dynamically?
throw new MapperParsingException("attempt to copy value to non-existing object [" + field + "]");
}
ObjectMapper update = parseDynamicValue(context, mapper, fieldPath, context.parser().currentToken());
assert update != null; // we are parsing a dynamic value so we necessarily created a new mapping
// propagate the update to the root
while (objectPath.length() > 0) {
String parentPath = "";
ObjectMapper parent = context.root();
posDot = objectPath.lastIndexOf('.');
if (posDot > 0) {
parentPath = objectPath.substring(0, posDot);
parent = context.docMapper().objectMappers().get(parentPath);
}
if (parent == null) {
throw new ElasticsearchIllegalStateException("[" + objectPath + "] has no parent for path [" + parentPath + "]");
}
update = parent.mappingUpdate(update);
objectPath = parentPath;
}
context.addDynamicMappingsUpdate(update);
}
}
/**
* Parse the given {@code context} with the given {@code mapper} and apply
* the potential mapping update in-place. This method is useful when
* composing mapping updates.
*/
private static <M extends Mapper> M parseAndMergeUpdate(M mapper, ParseContext context) throws IOException {
final Mapper update = parseObjectOrField(context, mapper);
if (update != null) {
MapperUtils.merge(mapper, update);
}
return mapper;
}
private static XContentParser transform(Mapping mapping, XContentParser parser) throws IOException {
Map<String, Object> transformed = transformSourceAsMap(mapping, parser.mapOrderedAndClose());
XContentBuilder builder = XContentFactory.contentBuilder(parser.contentType()).value(transformed);
return parser.contentType().xContent().createParser(builder.bytes());
}
private static ObjectMapper.Dynamic dynamicOrDefault(ObjectMapper.Dynamic dynamic) {
return dynamic == null ? ObjectMapper.Dynamic.TRUE : dynamic;
}
static Map<String, Object> transformSourceAsMap(Mapping mapping, Map<String, Object> sourceAsMap) {
if (mapping.sourceTransforms.length == 0) {
return sourceAsMap;
}
for (Mapping.SourceTransform transform : mapping.sourceTransforms) {
sourceAsMap = transform.transformSourceAsMap(sourceAsMap);
}
return sourceAsMap;
}
@Override
public void close() {
cache.close();
}
}

View File

@ -297,6 +297,13 @@ public interface FieldMapper<T> extends Mapper {
* */
public boolean isGenerated();
/**
* Parse using the provided {@link ParseContext} and return a mapping
* update if dynamic mappings modified the mappings, or {@code null} if
* mappings were not modified.
*/
Mapper parse(ParseContext context) throws IOException;
/**
* @return a {@link FieldStats} instance that maps to the type of this field based on the provided {@link Terms} instance.
*/

View File

@ -125,13 +125,6 @@ public interface Mapper extends ToXContent {
String name();
/**
* Parse using the provided {@link ParseContext} and return a mapping
* update if dynamic mappings modified the mappings, or {@code null} if
* mappings were not modified.
*/
Mapper parse(ParseContext context) throws IOException;
void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException;
void traverse(FieldMapperListener fieldMapperListener);

View File

@ -28,18 +28,7 @@ import java.util.Collection;
public enum MapperUtils {
;
/**
* Parse the given {@code context} with the given {@code mapper} and apply
* the potential mapping update in-place. This method is useful when
* composing mapping updates.
*/
public static <M extends Mapper> M parseAndMergeUpdate(M mapper, ParseContext context) throws IOException {
final Mapper update = mapper.parse(context);
if (update != null) {
merge(mapper, update);
}
return mapper;
}
private static MergeResult newStrictMergeContext() {
return new MergeResult(false) {

View File

@ -429,9 +429,6 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
throw new MapperParsingException("failed to parse [" + names.fullName() + "]", e);
}
multiFields.parse(this, context);
if (copyTo != null) {
copyTo.parse(context);
}
return null;
}
@ -897,6 +894,7 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
}
public void parse(AbstractFieldMapper mainField, ParseContext context) throws IOException {
// TODO: multi fields are really just copy fields, we just need to expose "sub fields" or something that can be part of the mappings
if (mappers.isEmpty()) {
return;
}
@ -1003,34 +1001,6 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
this.copyToFields = copyToFields;
}
/**
* Creates instances of the fields that the current field should be copied to
*/
public void parse(ParseContext context) throws IOException {
if (!context.isWithinCopyTo() && copyToFields.isEmpty() == false) {
context = context.createCopyToContext();
for (String field : copyToFields) {
// In case of a hierarchy of nested documents, we need to figure out
// which document the field should go to
Document targetDoc = null;
for (Document doc = context.doc(); doc != null; doc = doc.getParent()) {
if (field.startsWith(doc.getPrefix())) {
targetDoc = doc;
break;
}
}
assert targetDoc != null;
final ParseContext copyToContext;
if (targetDoc == context.doc()) {
copyToContext = context;
} else {
copyToContext = context.switchDoc(targetDoc);
}
parse(field, copyToContext);
}
}
}
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (!copyToFields.isEmpty()) {
builder.startArray("copy_to");
@ -1058,53 +1028,6 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
public ImmutableList<String> copyToFields() {
return copyToFields;
}
/**
* Creates an copy of the current field with given field name and boost
*/
public void parse(String field, ParseContext context) throws IOException {
FieldMappers mappers = context.docMapper().mappers().indexName(field);
if (mappers != null && !mappers.isEmpty()) {
mappers.mapper().parse(context);
} else {
// The path of the dest field might be completely different from the current one so we need to reset it
context = context.overridePath(new ContentPath(0));
ObjectMapper mapper = context.root();
String objectPath = "";
String fieldPath = field;
int posDot = field.lastIndexOf('.');
if (posDot > 0) {
objectPath = field.substring(0, posDot);
context.path().add(objectPath);
mapper = context.docMapper().objectMappers().get(objectPath);
fieldPath = field.substring(posDot + 1);
}
if (mapper == null) {
//TODO: Create an object dynamically?
throw new MapperParsingException("attempt to copy value to non-existing object [" + field + "]");
}
ObjectMapper update = mapper.parseDynamicValue(context, fieldPath, context.parser().currentToken());
assert update != null; // we are parsing a dynamic value so we necessarily created a new mapping
// propagate the update to the root
while (objectPath.length() > 0) {
String parentPath = "";
ObjectMapper parent = context.root();
posDot = objectPath.lastIndexOf('.');
if (posDot > 0) {
parentPath = objectPath.substring(0, posDot);
parent = context.docMapper().objectMappers().get(parentPath);
}
if (parent == null) {
throw new ElasticsearchIllegalStateException("[" + objectPath + "] has no parent for path [" + parentPath + "]");
}
update = parent.mappingUpdate(update);
objectPath = parentPath;
}
context.addDynamicMappingsUpdate((RootObjectMapper) update);
}
}
}
/**

View File

@ -20,44 +20,31 @@
package org.elasticsearch.index.mapper.object;
import com.google.common.collect.Iterables;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.CopyOnWriteHashMap;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.FieldMapperListener;
import org.elasticsearch.index.mapper.InternalMapper;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperUtils;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ObjectMapperListener;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.StrictDynamicMappingException;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.settings.IndexSettings;
import java.io.IOException;
@ -74,15 +61,7 @@ import java.util.TreeMap;
import static com.google.common.collect.Lists.newArrayList;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeBooleanValue;
import static org.elasticsearch.index.mapper.MapperBuilders.binaryField;
import static org.elasticsearch.index.mapper.MapperBuilders.booleanField;
import static org.elasticsearch.index.mapper.MapperBuilders.dateField;
import static org.elasticsearch.index.mapper.MapperBuilders.doubleField;
import static org.elasticsearch.index.mapper.MapperBuilders.floatField;
import static org.elasticsearch.index.mapper.MapperBuilders.integerField;
import static org.elasticsearch.index.mapper.MapperBuilders.longField;
import static org.elasticsearch.index.mapper.MapperBuilders.object;
import static org.elasticsearch.index.mapper.MapperBuilders.stringField;
import static org.elasticsearch.index.mapper.core.TypeParsers.parsePathType;
/**
@ -418,6 +397,18 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll, Clonea
return this.name;
}
public boolean isEnabled() {
return this.enabled;
}
public ContentPath.Type pathType() {
return pathType;
}
public Mapper getMapper(String field) {
return mappers.get(field);
}
@Override
public void includeInAll(Boolean includeInAll) {
if (includeInAll == null) {
@ -500,422 +491,7 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll, Clonea
}
public final Dynamic dynamic() {
return this.dynamic == null ? Dynamic.TRUE : this.dynamic;
}
public void setDynamic(Dynamic dynamic) {
this.dynamic = dynamic;
}
protected boolean allowValue() {
return true;
}
@Override
public ObjectMapper parse(ParseContext context) throws IOException {
if (!enabled) {
context.parser().skipChildren();
return null;
}
XContentParser parser = context.parser();
String currentFieldName = parser.currentName();
XContentParser.Token token = parser.currentToken();
if (token == XContentParser.Token.VALUE_NULL) {
// the object is null ("obj1" : null), simply bail
return null;
}
if (token.isValue() && !allowValue()) {
// if we are parsing an object but it is just a value, its only allowed on root level parsers with there
// is a field name with the same name as the type
throw new MapperParsingException("object mapping for [" + name + "] tried to parse field [" + currentFieldName + "] as object, but found a concrete value");
}
if (nested.isNested()) {
context = context.createNestedContext(fullPath);
Document nestedDoc = context.doc();
Document parentDoc = nestedDoc.getParent();
// pre add the uid field if possible (id was already provided)
IndexableField uidField = parentDoc.getField(UidFieldMapper.NAME);
if (uidField != null) {
// we don't need to add it as a full uid field in nested docs, since we don't need versioning
// we also rely on this for UidField#loadVersion
// this is a deeply nested field
nestedDoc.add(new Field(UidFieldMapper.NAME, uidField.stringValue(), UidFieldMapper.Defaults.NESTED_FIELD_TYPE));
}
// the type of the nested doc starts with __, so we can identify that its a nested one in filters
// note, we don't prefix it with the type of the doc since it allows us to execute a nested query
// across types (for example, with similar nested objects)
nestedDoc.add(new Field(TypeFieldMapper.NAME, nestedTypePathAsString, TypeFieldMapper.Defaults.FIELD_TYPE));
}
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(pathType);
// if we are at the end of the previous object, advance
if (token == XContentParser.Token.END_OBJECT) {
token = parser.nextToken();
}
if (token == XContentParser.Token.START_OBJECT) {
// if we are just starting an OBJECT, advance, this is the object we are parsing, we need the name first
token = parser.nextToken();
}
ObjectMapper update = null;
while (token != XContentParser.Token.END_OBJECT) {
ObjectMapper newUpdate = null;
if (token == XContentParser.Token.START_OBJECT) {
newUpdate = serializeObject(context, currentFieldName);
} else if (token == XContentParser.Token.START_ARRAY) {
newUpdate = serializeArray(context, currentFieldName);
} else if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
serializeNullValue(context, currentFieldName);
} else if (token == null) {
throw new MapperParsingException("object mapping for [" + name + "] tried to parse field [" + currentFieldName + "] as object, but got EOF, has a concrete value been provided to it?");
} else if (token.isValue()) {
newUpdate = serializeValue(context, currentFieldName, token);
}
token = parser.nextToken();
if (newUpdate != null) {
if (update == null) {
update = newUpdate;
} else {
MapperUtils.merge(update, newUpdate);
}
}
}
// restore the enable path flag
context.path().pathType(origPathType);
if (nested.isNested()) {
Document nestedDoc = context.doc();
Document parentDoc = nestedDoc.getParent();
if (nested.isIncludeInParent()) {
for (IndexableField field : nestedDoc.getFields()) {
if (field.name().equals(UidFieldMapper.NAME) || field.name().equals(TypeFieldMapper.NAME)) {
continue;
} else {
parentDoc.add(field);
}
}
}
if (nested.isIncludeInRoot()) {
Document rootDoc = context.rootDoc();
// don't add it twice, if its included in parent, and we are handling the master doc...
if (!nested.isIncludeInParent() || parentDoc != rootDoc) {
for (IndexableField field : nestedDoc.getFields()) {
if (field.name().equals(UidFieldMapper.NAME) || field.name().equals(TypeFieldMapper.NAME)) {
continue;
} else {
rootDoc.add(field);
}
}
}
}
}
return update;
}
private void serializeNullValue(ParseContext context, String lastFieldName) throws IOException {
// we can only handle null values if we have mappings for them
Mapper mapper = mappers.get(lastFieldName);
if (mapper != null) {
if (mapper instanceof FieldMapper) {
if (!((FieldMapper) mapper).supportsNullValue()) {
throw new MapperParsingException("no object mapping found for null value in [" + lastFieldName + "]");
}
}
mapper.parse(context);
} else if (dynamic == Dynamic.STRICT) {
throw new StrictDynamicMappingException(fullPath, lastFieldName);
}
}
private ObjectMapper serializeObject(final ParseContext context, String currentFieldName) throws IOException {
if (currentFieldName == null) {
throw new MapperParsingException("object mapping [" + name + "] trying to serialize an object with no field associated with it, current value [" + context.parser().textOrNull() + "]");
}
context.path().add(currentFieldName);
ObjectMapper update = null;
Mapper objectMapper = mappers.get(currentFieldName);
if (objectMapper != null) {
final Mapper subUpdate = objectMapper.parse(context);
if (subUpdate != null) {
// propagate mapping update
update = mappingUpdate(subUpdate);
}
} else {
Dynamic dynamic = this.dynamic;
if (dynamic == null) {
dynamic = context.root().dynamic();
}
if (dynamic == Dynamic.STRICT) {
throw new StrictDynamicMappingException(fullPath, currentFieldName);
} else if (dynamic == Dynamic.TRUE) {
// remove the current field name from path, since template search and the object builder add it as well...
context.path().remove();
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "object");
if (builder == null) {
builder = MapperBuilders.object(currentFieldName).enabled(true).pathType(pathType);
// if this is a non root object, then explicitly set the dynamic behavior if set
if (!(this instanceof RootObjectMapper) && this.dynamic != Defaults.DYNAMIC) {
((Builder) builder).dynamic(this.dynamic);
}
}
BuilderContext builderContext = new BuilderContext(context.indexSettings(), context.path());
objectMapper = builder.build(builderContext);
context.path().add(currentFieldName);
update = mappingUpdate(MapperUtils.parseAndMergeUpdate(objectMapper, context));
} else {
// not dynamic, read everything up to end object
context.parser().skipChildren();
}
}
context.path().remove();
return update;
}
private ObjectMapper serializeArray(ParseContext context, String lastFieldName) throws IOException {
String arrayFieldName = lastFieldName;
Mapper mapper = mappers.get(lastFieldName);
if (mapper != null) {
// There is a concrete mapper for this field already. Need to check if the mapper
// expects an array, if so we pass the context straight to the mapper and if not
// we serialize the array components
if (mapper instanceof ArrayValueMapperParser) {
final Mapper subUpdate = mapper.parse(context);
if (subUpdate != null) {
// propagate the mapping update
return mappingUpdate(subUpdate);
} else {
return null;
}
} else {
return serializeNonDynamicArray(context, lastFieldName, arrayFieldName);
}
} else {
Dynamic dynamic = this.dynamic;
if (dynamic == null) {
dynamic = context.root().dynamic();
}
if (dynamic == Dynamic.STRICT) {
throw new StrictDynamicMappingException(fullPath, arrayFieldName);
} else if (dynamic == Dynamic.TRUE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, arrayFieldName, "object");
if (builder == null) {
return serializeNonDynamicArray(context, lastFieldName, arrayFieldName);
}
BuilderContext builderContext = new BuilderContext(context.indexSettings(), context.path());
mapper = builder.build(builderContext);
if (mapper != null && mapper instanceof ArrayValueMapperParser) {
context.path().add(arrayFieldName);
mapper = MapperUtils.parseAndMergeUpdate(mapper, context);
return mappingUpdate(mapper);
} else {
return serializeNonDynamicArray(context, lastFieldName, arrayFieldName);
}
} else {
return serializeNonDynamicArray(context, lastFieldName, arrayFieldName);
}
}
}
private ObjectMapper serializeNonDynamicArray(ParseContext context, String lastFieldName, String arrayFieldName) throws IOException {
XContentParser parser = context.parser();
XContentParser.Token token;
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.START_OBJECT) {
return serializeObject(context, lastFieldName);
} else if (token == XContentParser.Token.START_ARRAY) {
return serializeArray(context, lastFieldName);
} else if (token == XContentParser.Token.FIELD_NAME) {
lastFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
serializeNullValue(context, lastFieldName);
} else if (token == null) {
throw new MapperParsingException("object mapping for [" + name + "] with array for [" + arrayFieldName + "] tried to parse as array, but got EOF, is there a mismatch in types for the same field?");
} else {
return serializeValue(context, lastFieldName, token);
}
}
return null;
}
private ObjectMapper serializeValue(final ParseContext context, String currentFieldName, XContentParser.Token token) throws IOException {
if (currentFieldName == null) {
throw new MapperParsingException("object mapping [" + name + "] trying to serialize a value with no field associated with it, current value [" + context.parser().textOrNull() + "]");
}
Mapper mapper = mappers.get(currentFieldName);
if (mapper != null) {
Mapper subUpdate = mapper.parse(context);
if (subUpdate == null) {
return null;
}
return mappingUpdate(subUpdate);
} else {
return parseDynamicValue(context, currentFieldName, token);
}
}
public ObjectMapper parseDynamicValue(final ParseContext context, String currentFieldName, XContentParser.Token token) throws IOException {
Dynamic dynamic = this.dynamic;
if (dynamic == null) {
dynamic = context.root().dynamic();
}
if (dynamic == Dynamic.STRICT) {
throw new StrictDynamicMappingException(fullPath, currentFieldName);
}
if (dynamic == Dynamic.FALSE) {
return null;
}
Mapper mapper = null;
BuilderContext builderContext = new BuilderContext(context.indexSettings(), context.path());
if (token == XContentParser.Token.VALUE_STRING) {
boolean resolved = false;
// do a quick test to see if its fits a dynamic template, if so, use it.
// we need to do it here so we can handle things like attachment templates, where calling
// text (to see if its a date) causes the binary value to be cleared
if (!resolved) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "string", null);
if (builder != null) {
mapper = builder.build(builderContext);
resolved = true;
}
}
if (!resolved && context.root().dateDetection()) {
String text = context.parser().text();
// a safe check since "1" gets parsed as well
if (Strings.countOccurrencesOf(text, ":") > 1 || Strings.countOccurrencesOf(text, "-") > 1 || Strings.countOccurrencesOf(text, "/") > 1) {
for (FormatDateTimeFormatter dateTimeFormatter : context.root().dynamicDateTimeFormatters()) {
try {
dateTimeFormatter.parser().parseMillis(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "date");
if (builder == null) {
builder = dateField(currentFieldName).dateTimeFormatter(dateTimeFormatter);
}
mapper = builder.build(builderContext);
resolved = true;
break;
} catch (Exception e) {
// failure to parse this, continue
}
}
}
}
if (!resolved && context.root().numericDetection()) {
String text = context.parser().text();
try {
Long.parseLong(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
if (builder == null) {
builder = longField(currentFieldName);
}
mapper = builder.build(builderContext);
resolved = true;
} catch (Exception e) {
// not a long number
}
if (!resolved) {
try {
Double.parseDouble(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
if (builder == null) {
builder = doubleField(currentFieldName);
}
mapper = builder.build(builderContext);
resolved = true;
} catch (Exception e) {
// not a long number
}
}
}
if (!resolved) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "string");
if (builder == null) {
builder = stringField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (token == XContentParser.Token.VALUE_NUMBER) {
XContentParser.NumberType numberType = context.parser().numberType();
if (numberType == XContentParser.NumberType.INT) {
if (context.parser().estimatedNumberType()) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
if (builder == null) {
builder = longField(currentFieldName);
}
mapper = builder.build(builderContext);
} else {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "integer");
if (builder == null) {
builder = integerField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (numberType == XContentParser.NumberType.LONG) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
if (builder == null) {
builder = longField(currentFieldName);
}
mapper = builder.build(builderContext);
} else if (numberType == XContentParser.NumberType.FLOAT) {
if (context.parser().estimatedNumberType()) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
if (builder == null) {
builder = doubleField(currentFieldName);
}
mapper = builder.build(builderContext);
} else {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "float");
if (builder == null) {
builder = floatField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (numberType == XContentParser.NumberType.DOUBLE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
if (builder == null) {
builder = doubleField(currentFieldName);
}
mapper = builder.build(builderContext);
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "boolean");
if (builder == null) {
builder = booleanField(currentFieldName);
}
mapper = builder.build(builderContext);
} else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "binary");
if (builder == null) {
builder = binaryField(currentFieldName);
}
mapper = builder.build(builderContext);
} else {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, null);
if (builder != null) {
mapper = builder.build(builderContext);
} else {
// TODO how do we identify dynamically that its a binary value?
throw new ElasticsearchIllegalStateException("Can't handle serializing a dynamic type with content token [" + token + "] and field name [" + currentFieldName + "]");
}
}
mapper = MapperUtils.parseAndMergeUpdate(mapper, context);
ObjectMapper update = null;
if (mapper != null) {
update = mappingUpdate(mapper);
}
return update;
return dynamic;
}
@Override

View File

@ -254,11 +254,6 @@ public class RootObjectMapper extends ObjectMapper {
return null;
}
@Override
protected boolean allowValue() {
return true;
}
@Override
protected void doMerge(ObjectMapper mergeWith, MergeResult mergeResult) {
RootObjectMapper mergeWithObject = (RootObjectMapper) mergeWith;

View File

@ -16,13 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.dynamic;
package org.elasticsearch.index.mapper;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import java.io.IOException;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.dynamic;
package org.elasticsearch.index.mapper;
import com.google.common.collect.ImmutableMap;
@ -31,15 +31,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMappers;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.mapper.StrictDynamicMappingException;
import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import java.io.IOException;
@ -210,7 +201,7 @@ public class DynamicMappingTests extends ElasticsearchSingleNodeTest {
ctx.reset(XContentHelper.createParser(source.source()), new ParseContext.Document(), source, null);
assertEquals(XContentParser.Token.START_OBJECT, ctx.parser().nextToken());
ctx.parser().nextToken();
return mapper.root().parse(ctx);
return DocumentParser.parseObject(ctx, mapper.root());
}
public void testDynamicMappingsNotNeeded() throws Exception {

View File

@ -111,7 +111,7 @@ public class ExternalMapper extends AbstractFieldMapper<Object> {
BooleanFieldMapper boolMapper = boolBuilder.build(context);
GeoPointFieldMapper pointMapper = pointBuilder.build(context);
GeoShapeFieldMapper shapeMapper = shapeBuilder.build(context);
Mapper stringMapper = stringBuilder.build(context);
FieldMapper stringMapper = (FieldMapper)stringBuilder.build(context);
context.path().remove();
context.path().pathType(origPathType);
@ -157,12 +157,12 @@ public class ExternalMapper extends AbstractFieldMapper<Object> {
private final BooleanFieldMapper boolMapper;
private final GeoPointFieldMapper pointMapper;
private final GeoShapeFieldMapper shapeMapper;
private final Mapper stringMapper;
private final FieldMapper stringMapper;
public ExternalMapper(FieldMapper.Names names,
String generatedValue, String mapperName,
BinaryFieldMapper binMapper, BooleanFieldMapper boolMapper, GeoPointFieldMapper pointMapper,
GeoShapeFieldMapper shapeMapper, Mapper stringMapper, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
GeoShapeFieldMapper shapeMapper, FieldMapper stringMapper, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(names, 1.0f, Defaults.FIELD_TYPE, false, null, null, null, null, null, indexSettings,
multiFields, copyTo);
this.generatedValue = generatedValue;
@ -207,9 +207,6 @@ public class ExternalMapper extends AbstractFieldMapper<Object> {
stringMapper.parse(context);
multiFields.parse(this, context);
if (copyTo != null) {
copyTo.parse(context);
}
return null;
}

View File

@ -38,11 +38,6 @@ public class ExternalRootMapper implements RootMapper {
return CONTENT_TYPE;
}
@Override
public Mapper parse(ParseContext context) throws IOException {
return null;
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
if (!(mergeWith instanceof ExternalRootMapper)) {

View File

@ -70,7 +70,7 @@ public class TestMergeMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
String objectMapping = XContentFactory.jsonBuilder().startObject().startObject("type1").endObject().endObject().string();
DocumentMapper mapper = parser.parse(objectMapping);
assertThat(mapper.root().dynamic(), equalTo(ObjectMapper.Dynamic.TRUE));
assertNull(mapper.root().dynamic());
String withDynamicMapping = XContentFactory.jsonBuilder().startObject().startObject("type1").field("dynamic", "false").endObject().endObject().string();
DocumentMapper withDynamicMapper = parser.parse(withDynamicMapping);