Rare deadlock when introducing new mapping fields/objects, closes #1222.

This commit is contained in:
Shay Banon 2011-08-09 15:31:17 +03:00
parent f1e23067cd
commit 794a284093
4 changed files with 150 additions and 147 deletions

View File

@ -51,6 +51,7 @@ import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;
import static org.elasticsearch.common.collect.Lists.*;
@ -309,9 +310,9 @@ public class DocumentMapper implements ToXContent {
private volatile ImmutableMap<String, ObjectMapper> objectMappers = ImmutableMap.of();
private final List<FieldMapperListener> fieldMapperListeners = newArrayList();
private final List<FieldMapperListener> fieldMapperListeners = new CopyOnWriteArrayList<FieldMapperListener>();
private final List<ObjectMapperListener> objectMapperListeners = newArrayList();
private final List<ObjectMapperListener> objectMapperListeners = new CopyOnWriteArrayList<ObjectMapperListener>();
private boolean hasNestedObjects = false;
@ -630,47 +631,43 @@ public class DocumentMapper implements ToXContent {
public void addFieldMapper(FieldMapper fieldMapper) {
synchronized (mutex) {
fieldMappers = fieldMappers.concat(this, fieldMapper);
for (FieldMapperListener listener : fieldMapperListeners) {
listener.fieldMapper(fieldMapper);
}
}
for (FieldMapperListener listener : fieldMapperListeners) {
listener.fieldMapper(fieldMapper);
}
}
public void addFieldMapperListener(FieldMapperListener fieldMapperListener, boolean includeExisting) {
synchronized (mutex) {
fieldMapperListeners.add(fieldMapperListener);
if (includeExisting) {
if (indexFieldMapper.enabled()) {
fieldMapperListener.fieldMapper(indexFieldMapper);
}
fieldMapperListener.fieldMapper(sourceFieldMapper);
fieldMapperListener.fieldMapper(sizeFieldMapper);
fieldMapperListener.fieldMapper(typeFieldMapper);
fieldMapperListener.fieldMapper(uidFieldMapper);
fieldMapperListener.fieldMapper(allFieldMapper);
rootObjectMapper.traverse(fieldMapperListener);
fieldMapperListeners.add(fieldMapperListener);
if (includeExisting) {
if (indexFieldMapper.enabled()) {
fieldMapperListener.fieldMapper(indexFieldMapper);
}
fieldMapperListener.fieldMapper(sourceFieldMapper);
fieldMapperListener.fieldMapper(sizeFieldMapper);
fieldMapperListener.fieldMapper(typeFieldMapper);
fieldMapperListener.fieldMapper(uidFieldMapper);
fieldMapperListener.fieldMapper(allFieldMapper);
rootObjectMapper.traverse(fieldMapperListener);
}
}
public void addObjectMapper(ObjectMapper objectMapper) {
synchronized (mutex) {
objectMappers = MapBuilder.newMapBuilder(objectMappers).put(objectMapper.fullPath(), objectMapper).immutableMap();
for (ObjectMapperListener objectMapperListener : objectMapperListeners) {
objectMapperListener.objectMapper(objectMapper);
}
if (objectMapper.nested().isNested()) {
hasNestedObjects = true;
}
}
for (ObjectMapperListener objectMapperListener : objectMapperListeners) {
objectMapperListener.objectMapper(objectMapper);
}
}
public void addObjectMapperListener(ObjectMapperListener objectMapperListener, boolean includeExisting) {
synchronized (mutex) {
objectMapperListeners.add(objectMapperListener);
if (includeExisting) {
rootObjectMapper.traverse(objectMapperListener);
}
objectMapperListeners.add(objectMapperListener);
if (includeExisting) {
rootObjectMapper.traverse(objectMapperListener);
}
}

View File

@ -182,9 +182,9 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
logger.warn("Type [{}] contains a '.', it is recommended not to include it within a type name", mapper.type());
}
remove(mapper.type()); // first remove it (in case its an update, we need to remove the aggregated mappers)
mappers = newMapBuilder(mappers).put(mapper.type(), mapper).immutableMap();
mapper.addFieldMapperListener(fieldMapperListener, true);
mapper.addObjectMapperListener(objectMapperListener, true);
mappers = newMapBuilder(mappers).put(mapper.type(), mapper).immutableMap();
}
}

View File

@ -35,6 +35,7 @@ import org.elasticsearch.index.mapper.core.AbstractFieldMapper;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -219,13 +220,14 @@ public class MultiFieldMapper implements Mapper, AllFieldMapper.IncludeInAll {
mergeContext.addConflict("Can't merge a non multi_field / non simple mapping [" + mergeWith.name() + "] with a multi_field mapping [" + name() + "]");
return;
}
List<FieldMapper> mappersToAddToDocMapper = new ArrayList<FieldMapper>();
synchronized (mutex) {
if (mergeWith instanceof AbstractFieldMapper) {
// its a single field mapper, upgraded into a multi field mapper, just update the default mapper
if (defaultMapper == null) {
if (!mergeContext.mergeFlags().simulate()) {
defaultMapper = mergeWith;
mergeContext.docMapper().addFieldMapper((FieldMapper) defaultMapper);
mappersToAddToDocMapper.add((FieldMapper) defaultMapper);
}
}
} else {
@ -235,7 +237,7 @@ public class MultiFieldMapper implements Mapper, AllFieldMapper.IncludeInAll {
if (mergeWithMultiField.defaultMapper != null) {
if (!mergeContext.mergeFlags().simulate()) {
defaultMapper = mergeWithMultiField.defaultMapper;
mergeContext.docMapper().addFieldMapper((FieldMapper) defaultMapper);
mappersToAddToDocMapper.add((FieldMapper) defaultMapper);
}
}
} else {
@ -256,7 +258,7 @@ public class MultiFieldMapper implements Mapper, AllFieldMapper.IncludeInAll {
}
mappers = newMapBuilder(mappers).put(mergeWithMapper.name(), mergeWithMapper).immutableMap();
if (mergeWithMapper instanceof AbstractFieldMapper) {
mergeContext.docMapper().addFieldMapper((FieldMapper) mergeWithMapper);
mappersToAddToDocMapper.add((FieldMapper) mergeWithMapper);
}
}
} else {
@ -265,6 +267,10 @@ public class MultiFieldMapper implements Mapper, AllFieldMapper.IncludeInAll {
}
}
}
// call it outside of the mutex
for (FieldMapper fieldMapper : mappersToAddToDocMapper) {
mergeContext.docMapper().addFieldMapper(fieldMapper);
}
}
@Override public void close() {

View File

@ -40,6 +40,7 @@ import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.multifield.MultiFieldMapper;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -493,11 +494,11 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
} else if (dynamic == Dynamic.TRUE) {
// we sync here just so we won't add it twice. Its not the end of the world
// to sync here since next operations will get it before
boolean newMapper = false;
synchronized (mutex) {
objectMapper = mappers.get(currentFieldName);
if (objectMapper != null) {
objectMapper.parse(context);
} else {
if (objectMapper == null) {
newMapper = true;
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "object");
if (builder == null) {
builder = MapperBuilders.object(currentFieldName).enabled(true).dynamic(dynamic).pathType(pathType);
@ -507,26 +508,29 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
BuilderContext builderContext = new BuilderContext(context.path());
objectMapper = builder.build(builderContext);
putMapper(objectMapper);
// we need to traverse in case we have a dynamic template and need to add field mappers
// introduced by it
objectMapper.traverse(new FieldMapperListener() {
@Override public void fieldMapper(FieldMapper fieldMapper) {
context.docMapper().addFieldMapper(fieldMapper);
}
});
objectMapper.traverse(new ObjectMapperListener() {
@Override public void objectMapper(ObjectMapper objectMapper) {
context.docMapper().addObjectMapper(objectMapper);
}
});
// now re add it and parse...
// now re add it
context.path().add(currentFieldName);
objectMapper.parse(context);
context.addedMapper();
}
}
// traverse and parse outside of the mutex
if (newMapper) {
// we need to traverse in case we have a dynamic template and need to add field mappers
// introduced by it
objectMapper.traverse(new FieldMapperListener() {
@Override public void fieldMapper(FieldMapper fieldMapper) {
context.docMapper().addFieldMapper(fieldMapper);
}
});
objectMapper.traverse(new ObjectMapperListener() {
@Override public void objectMapper(ObjectMapper objectMapper) {
context.docMapper().addObjectMapper(objectMapper);
}
});
}
// now, parse it
objectMapper.parse(context);
} else {
// not dynamic, read everything up to end object
context.parser().skipChildren();
@ -578,39 +582,37 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
// we sync here since we don't want to add this field twice to the document mapper
// its not the end of the world, since we add it to the mappers once we create it
// so next time we won't even get here for this field
boolean newMapper = false;
synchronized (mutex) {
mapper = mappers.get(currentFieldName);
if (mapper != null) {
mapper.parse(context);
return;
}
BuilderContext builderContext = new BuilderContext(context.path());
if (token == XContentParser.Token.VALUE_STRING) {
String text = context.parser().text();
// check if it fits one of the date formats
boolean resolved = false;
// a safe check since "1" gets parsed as well
if (context.root().dateDetection()) {
if (text.contains(":") || text.contains("-") || text.contains("/")) {
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);
if (mapper == null) {
newMapper = true;
BuilderContext builderContext = new BuilderContext(context.path());
if (token == XContentParser.Token.VALUE_STRING) {
String text = context.parser().text();
// check if it fits one of the date formats
boolean resolved = false;
// a safe check since "1" gets parsed as well
if (context.root().dateDetection()) {
if (text.contains(":") || text.contains("-") || text.contains("/")) {
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
}
mapper = builder.build(builderContext);
resolved = true;
break;
} catch (Exception e) {
// failure to parse this, continue
}
}
}
}
// DON'T do automatic ip detection logic, since it messes up with docs that have hosts and ips
// check if its an ip
// DON'T do automatic ip detection logic, since it messes up with docs that have hosts and ips
// check if its an ip
// if (!resolved && text.indexOf('.') != -1) {
// try {
// IpFieldMapper.ipToLong(text);
@ -624,81 +626,83 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
// // failure to parse, not ip...
// }
// }
if (!resolved) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "string");
if (builder == null) {
builder = stringField(currentFieldName);
if (!resolved) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "string");
if (builder == null) {
builder = stringField(currentFieldName);
}
mapper = builder.build(builderContext);
}
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()) {
} 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 {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "integer");
if (builder == null) {
builder = integerField(currentFieldName);
} 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);
}
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()) {
} 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 {
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");
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "boolean");
if (builder == null) {
builder = doubleField(currentFieldName);
builder = booleanField(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 {
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.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 + "]");
}
}
putMapper(mapper);
context.addedMapper();
}
putMapper(mapper);
}
if (newMapper) {
mapper.traverse(new FieldMapperListener() {
@Override public void fieldMapper(FieldMapper fieldMapper) {
context.docMapper().addFieldMapper(fieldMapper);
}
});
mapper.parse(context);
context.addedMapper();
}
mapper.parse(context);
}
@Override public void merge(final Mapper mergeWith, final MergeContext mergeContext) throws MergeMappingException {
@ -710,6 +714,7 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
doMerge(mergeWithObject, mergeContext);
List<Mapper> mappersToTraverse = new ArrayList<Mapper>();
synchronized (mutex) {
for (Mapper mergeWithMapper : mergeWithObject.mappers.values()) {
Mapper mergeIntoMapper = mappers.get(mergeWithMapper.name());
@ -717,16 +722,7 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
// no mapping, simply add it if not simulating
if (!mergeContext.mergeFlags().simulate()) {
putMapper(mergeWithMapper);
mergeWithMapper.traverse(new FieldMapperListener() {
@Override public void fieldMapper(FieldMapper fieldMapper) {
mergeContext.docMapper().addFieldMapper(fieldMapper);
}
});
mergeWithMapper.traverse(new ObjectMapperListener() {
@Override public void objectMapper(ObjectMapper objectMapper) {
mergeContext.docMapper().addObjectMapper(objectMapper);
}
});
mappersToTraverse.add(mergeWithMapper);
}
} else {
if ((mergeWithMapper instanceof MultiFieldMapper) && !(mergeIntoMapper instanceof MultiFieldMapper)) {
@ -734,18 +730,9 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
mergeWithMultiField.merge(mergeIntoMapper, mergeContext);
if (!mergeContext.mergeFlags().simulate()) {
putMapper(mergeWithMultiField);
// now, raise events for all mappers
// now, record mappers to traverse events for all mappers
for (Mapper mapper : mergeWithMultiField.mappers().values()) {
mapper.traverse(new FieldMapperListener() {
@Override public void fieldMapper(FieldMapper fieldMapper) {
mergeContext.docMapper().addFieldMapper(fieldMapper);
}
});
mapper.traverse(new ObjectMapperListener() {
@Override public void objectMapper(ObjectMapper objectMapper) {
mergeContext.docMapper().addObjectMapper(objectMapper);
}
});
mappersToTraverse.add(mapper);
}
}
} else {
@ -754,6 +741,19 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
}
}
}
// call this outside of the mutex
for (Mapper mapper : mappersToTraverse) {
mapper.traverse(new FieldMapperListener() {
@Override public void fieldMapper(FieldMapper fieldMapper) {
mergeContext.docMapper().addFieldMapper(fieldMapper);
}
});
mapper.traverse(new ObjectMapperListener() {
@Override public void objectMapper(ObjectMapper objectMapper) {
mergeContext.docMapper().addObjectMapper(objectMapper);
}
});
}
}
protected void doMerge(ObjectMapper mergeWith, MergeContext mergeContext) {
@ -784,7 +784,7 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
} else if (mappers.isEmpty()) { // only write the object content type if there are no properties, otherwise, it is automatically detected
builder.field("type", CONTENT_TYPE);
}
// grr, ugly! on root, dynamic defaults to TRUE, on childs, it defaults to null to
// grr, ugly! on root, dynamic defaults to TRUE, on children, it defaults to null to
// inherit the root behavior
if (this instanceof RootObjectMapper) {
if (dynamic != Dynamic.TRUE) {