Merge remote-tracking branch 'upstream/master' into query_profiler

Conflicts:
	core/src/main/java/org/elasticsearch/search/SearchService.java
This commit is contained in:
Zachary Tong 2015-12-15 13:27:18 -05:00
commit 117dd9992e
121 changed files with 2971 additions and 1031 deletions

View file

@ -97,6 +97,7 @@ subprojects {
// the "value" -quiet is added, separated by a space. This is ok since the javadoc
// command already adds -quiet, so we are just duplicating it
// see https://discuss.gradle.org/t/add-custom-javadoc-option-that-does-not-take-an-argument/5959
javadoc.options.encoding='UTF8'
javadoc.options.addStringOption('Xdoclint:all,-missing', '-quiet')
}
}

View file

@ -19,8 +19,16 @@
package org.elasticsearch.action.support.broadcast.node;
import org.elasticsearch.action.*;
import org.elasticsearch.action.support.*;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.NoShardAvailableActionException;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
@ -37,7 +45,14 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.NodeShouldNotConnectException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
@ -394,7 +409,15 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
e.setIndex(shardRouting.getIndex());
e.setShard(shardRouting.shardId());
shardResults[shardIndex] = e;
logger.debug("[{}] failed to execute operation for shard [{}]", e, actionName, shardRouting.shortSummary());
if (TransportActions.isShardNotAvailableException(t)) {
if (logger.isTraceEnabled()) {
logger.trace("[{}] failed to execute operation for shard [{}]", t, actionName, shardRouting.shortSummary());
}
} else {
if (logger.isDebugEnabled()) {
logger.debug("[{}] failed to execute operation for shard [{}]", t, actionName, shardRouting.shortSummary());
}
}
}
}
}

View file

@ -300,11 +300,15 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
@Override
public void onFailure(Throwable t) {
if (t instanceof RetryOnReplicaException) {
logger.trace("Retrying operation on replica, action [{}], request [{}]", t, actionName, request);
logger.trace("Retrying operation on replica, action [{}], request [{}]", t, transportReplicaAction, request);
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
threadPool.executor(executor).execute(AsyncReplicaAction.this);
// Forking a thread on local node via transport service so that custom transport service have an
// opportunity to execute custom logic before the replica operation begins
String extraMessage = "action [" + transportReplicaAction + "], request[" + request + "]";
TransportChannelResponseHandler<TransportResponse.Empty> handler = TransportChannelResponseHandler.emptyResponseHandler(logger, channel, extraMessage);
transportService.sendRequest(clusterService.localNode(), transportReplicaAction, request, handler);
}
@Override

View file

@ -37,7 +37,6 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InvalidTypeNameException;
import org.elasticsearch.percolator.PercolatorService;
@ -251,11 +250,8 @@ public class MetaDataMappingService extends AbstractComponent {
newMapper = indexService.mapperService().parse(request.type(), new CompressedXContent(request.source()), existingMapper == null);
if (existingMapper != null) {
// first, simulate
MergeResult mergeResult = existingMapper.merge(newMapper.mapping(), true, request.updateAllTypes());
// if we have conflicts, throw an exception
if (mergeResult.hasConflicts()) {
throw new IllegalArgumentException("Merge failed with failures {" + Arrays.toString(mergeResult.buildConflicts()) + "}");
}
// this will just throw exceptions in case of problems
existingMapper.merge(newMapper.mapping(), true, request.updateAllTypes());
} else {
// TODO: can we find a better place for this validation?
// The reason this validation is here is that the mapper service doesn't learn about

View file

@ -29,7 +29,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
@ -48,7 +47,13 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShadowIndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.Store;
@ -73,7 +78,7 @@ import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
/**
*
*/
public final class IndexService extends AbstractIndexComponent implements IndexComponent, Iterable<IndexShard>{
public final class IndexService extends AbstractIndexComponent implements IndexComponent, Iterable<IndexShard> {
private final IndexEventListener eventListener;
private final AnalysisService analysisService;
@ -93,7 +98,6 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
private final AtomicBoolean deleted = new AtomicBoolean(false);
private final IndexSettings indexSettings;
@Inject
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
SimilarityService similarityService,
ShardStoreDeleter shardStoreDeleter,
@ -146,7 +150,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
*/
@Nullable
public IndexShard getShardOrNull(int shardId) {
return shards.get(shardId);
return shards.get(shardId);
}
/**
@ -160,13 +164,17 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
return indexShard;
}
public Set<Integer> shardIds() { return shards.keySet(); }
public Set<Integer> shardIds() {
return shards.keySet();
}
public IndexCache cache() {
return indexCache;
}
public IndexFieldDataService fieldData() { return indexFieldData; }
public IndexFieldDataService fieldData() {
return indexFieldData;
}
public AnalysisService analysisService() {
return this.analysisService;
@ -207,7 +215,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
private long getAvgShardSizeInBytes() throws IOException {
long sum = 0;
int count = 0;
for(IndexShard indexShard : this) {
for (IndexShard indexShard : this) {
sum += indexShard.store().stats().sizeInBytes();
count++;
}
@ -254,17 +262,17 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
// TODO: we should, instead, hold a "bytes reserved" of how large we anticipate this shard will be, e.g. for a shard
// that's being relocated/replicated we know how large it will become once it's done copying:
// Count up how many shards are currently on each data path:
Map<Path,Integer> dataPathToShardCount = new HashMap<>();
for(IndexShard shard : this) {
Map<Path, Integer> dataPathToShardCount = new HashMap<>();
for (IndexShard shard : this) {
Path dataPath = shard.shardPath().getRootStatePath();
Integer curCount = dataPathToShardCount.get(dataPath);
if (curCount == null) {
curCount = 0;
}
dataPathToShardCount.put(dataPath, curCount+1);
dataPathToShardCount.put(dataPath, curCount + 1);
}
path = ShardPath.selectNewPathForShard(nodeEnv, shardId, this.indexSettings, routing.getExpectedShardSize() == ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE ? getAvgShardSizeInBytes() : routing.getExpectedShardSize(),
dataPathToShardCount);
dataPathToShardCount);
logger.debug("{} creating using a new path [{}]", shardId, path);
} else {
logger.debug("{} creating using an existing path [{}]", shardId, path);
@ -277,7 +285,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
logger.debug("creating shard_id {}", shardId);
// if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary.
final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false ||
(primary && IndexMetaData.isOnSharedFilesystem(indexSettings));
(primary && IndexMetaData.isOnSharedFilesystem(indexSettings));
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> nodeServicesProvider.getIndicesQueryCache().onClose(shardId)));
if (useShadowEngine(primary, indexSettings)) {
indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider);
@ -462,6 +470,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
}
}
}
/**
* Returns the filter associated with listed filtering aliases.
* <p>

View file

@ -781,10 +781,14 @@ public class InternalEngine extends Engine {
// we need to fail the engine. it might have already been failed before
// but we are double-checking it's failed and closed
if (indexWriter.isOpen() == false && indexWriter.getTragicException() != null) {
failEngine("already closed by tragic event", indexWriter.getTragicException());
failEngine("already closed by tragic event on the index writer", indexWriter.getTragicException());
} else if (translog.isOpen() == false && translog.getTragicException() != null) {
failEngine("already closed by tragic event on the translog", translog.getTragicException());
}
return true;
} else if (t != null && indexWriter.isOpen() == false && indexWriter.getTragicException() == t) {
} else if (t != null &&
((indexWriter.isOpen() == false && indexWriter.getTragicException() == t)
|| (translog.isOpen() == false && translog.getTragicException() == t))) {
// this spot on - we are handling the tragic event exception here so we have to fail the engine
// right away
failEngine(source, t);

View file

@ -19,16 +19,9 @@
package org.elasticsearch.index.mapper;
public class ContentPath {
public final class ContentPath {
public enum Type {
JUST_NAME,
FULL,
}
private Type pathType;
private final char delimiter;
private static final char DELIMITER = '.';
private final StringBuilder sb;
@ -47,7 +40,6 @@ public class ContentPath {
* number of path elements to not be included in {@link #pathAsText(String)}.
*/
public ContentPath(int offset) {
this.delimiter = '.';
this.sb = new StringBuilder();
this.offset = offset;
reset();
@ -71,26 +63,11 @@ public class ContentPath {
}
public String pathAsText(String name) {
if (pathType == Type.JUST_NAME) {
return name;
}
return fullPathAsText(name);
}
public String fullPathAsText(String name) {
sb.setLength(0);
for (int i = offset; i < index; i++) {
sb.append(path[i]).append(delimiter);
sb.append(path[i]).append(DELIMITER);
}
sb.append(name);
return sb.toString();
}
public Type pathType() {
return pathType;
}
public void pathType(Type type) {
this.pathType = type;
}
}

View file

@ -52,6 +52,7 @@ import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@ -117,7 +118,7 @@ public class DocumentMapper implements ToXContent {
private volatile CompressedXContent mappingSource;
private final Mapping mapping;
private volatile Mapping mapping;
private final DocumentParser documentParser;
@ -352,16 +353,19 @@ public class DocumentMapper implements ToXContent {
mapperService.addMappers(type, objectMappers, fieldMappers);
}
public MergeResult merge(Mapping mapping, boolean simulate, boolean updateAllTypes) {
public void merge(Mapping mapping, boolean simulate, boolean updateAllTypes) {
try (ReleasableLock lock = mappingWriteLock.acquire()) {
mapperService.checkMappersCompatibility(type, mapping, updateAllTypes);
final MergeResult mergeResult = new MergeResult(simulate, updateAllTypes);
this.mapping.merge(mapping, mergeResult);
// do the merge even if simulate == false so that we get exceptions
Mapping merged = this.mapping.merge(mapping, updateAllTypes);
if (simulate == false) {
addMappers(mergeResult.getNewObjectMappers(), mergeResult.getNewFieldMappers(), updateAllTypes);
this.mapping = merged;
Collection<ObjectMapper> objectMappers = new ArrayList<>();
Collection<FieldMapper> fieldMappers = new ArrayList<>(Arrays.asList(merged.metadataMappers));
MapperUtils.collect(merged.root, objectMappers, fieldMappers);
addMappers(objectMappers, fieldMappers, updateAllTypes);
refreshSource();
}
return mergeResult;
}
}

View file

@ -234,9 +234,6 @@ class DocumentParser implements Closeable {
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();
@ -267,12 +264,11 @@ class DocumentParser implements Closeable {
if (update == null) {
update = newUpdate;
} else {
MapperUtils.merge(update, newUpdate);
update = update.merge(newUpdate, false);
}
}
}
// restore the enable path flag
context.path().pathType(origPathType);
if (nested.isNested()) {
ParseContext.Document nestedDoc = context.doc();
ParseContext.Document parentDoc = nestedDoc.getParent();
@ -341,7 +337,7 @@ class DocumentParser implements Closeable {
context.path().remove();
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "object");
if (builder == null) {
builder = MapperBuilders.object(currentFieldName).enabled(true).pathType(mapper.pathType());
builder = MapperBuilders.object(currentFieldName).enabled(true);
// 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());
@ -610,7 +606,7 @@ class DocumentParser implements Closeable {
return null;
}
final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
final MappedFieldType existingFieldType = context.mapperService().fullName(context.path().fullPathAsText(currentFieldName));
final MappedFieldType existingFieldType = context.mapperService().fullName(context.path().pathAsText(currentFieldName));
Mapper.Builder builder = null;
if (existingFieldType != null) {
// create a builder of the same type
@ -695,7 +691,7 @@ class DocumentParser implements Closeable {
if (paths.length > 1) {
ObjectMapper parent = context.root();
for (int i = 0; i < paths.length-1; i++) {
mapper = context.docMapper().objectMappers().get(context.path().fullPathAsText(paths[i]));
mapper = context.docMapper().objectMappers().get(context.path().pathAsText(paths[i]));
if (mapper == null) {
// One mapping is missing, check if we are allowed to create a dynamic one.
ObjectMapper.Dynamic dynamic = parent.dynamic();
@ -713,12 +709,12 @@ class DocumentParser implements Closeable {
if (!(parent instanceof RootObjectMapper) && parent.dynamic() != ObjectMapper.Defaults.DYNAMIC) {
((ObjectMapper.Builder) builder).dynamic(parent.dynamic());
}
builder = MapperBuilders.object(paths[i]).enabled(true).pathType(parent.pathType());
builder = MapperBuilders.object(paths[i]).enabled(true);
}
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path());
mapper = (ObjectMapper) builder.build(builderContext);
if (mapper.nested() != ObjectMapper.Nested.NO) {
throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().fullPathAsText(paths[i]) + "]) through `copy_to`");
throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().pathAsText(paths[i]) + "]) through `copy_to`");
}
break;
case FALSE:
@ -759,7 +755,7 @@ class DocumentParser implements Closeable {
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);
mapper = (M) mapper.merge(update, false);
}
return mapper;
}

View file

@ -47,7 +47,7 @@ import java.util.List;
import java.util.Locale;
import java.util.stream.StreamSupport;
public abstract class FieldMapper extends Mapper {
public abstract class FieldMapper extends Mapper implements Cloneable {
public abstract static class Builder<T extends Builder, Y extends FieldMapper> extends Mapper.Builder<T, Y> {
@ -84,8 +84,13 @@ public abstract class FieldMapper extends Mapper {
* if the fieldType has a non-null option we are all good it might have been set through a different
* call.
*/
final IndexOptions options = getDefaultIndexOption();
assert options != IndexOptions.NONE : "default IndexOptions is NONE can't enable indexing";
IndexOptions options = getDefaultIndexOption();
if (options == IndexOptions.NONE) {
// can happen when an existing type on the same index has disabled indexing
// since we inherit the default field type from the first mapper that is
// created on an index
throw new IllegalArgumentException("mapper [" + name + "] has different [index] values from other types of the same index");
}
fieldType.setIndexOptions(options);
}
} else {
@ -202,11 +207,6 @@ public abstract class FieldMapper extends Mapper {
return this;
}
public T multiFieldPathType(ContentPath.Type pathType) {
multiFieldsBuilder.pathType(pathType);
return builder;
}
public T addMultiField(Mapper.Builder mapperBuilder) {
multiFieldsBuilder.add(mapperBuilder);
return builder;
@ -237,7 +237,7 @@ public abstract class FieldMapper extends Mapper {
}
protected String buildFullName(BuilderContext context) {
return context.path().fullPathAsText(name);
return context.path().pathAsText(name);
}
protected void setupFieldType(BuilderContext context) {
@ -270,7 +270,7 @@ public abstract class FieldMapper extends Mapper {
protected MappedFieldTypeReference fieldTypeRef;
protected final MappedFieldType defaultFieldType;
protected final MultiFields multiFields;
protected MultiFields multiFields;
protected CopyTo copyTo;
protected final boolean indexCreatedBefore2x;
@ -359,26 +359,41 @@ public abstract class FieldMapper extends Mapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected FieldMapper clone() {
try {
return (FieldMapper) super.clone();
} catch (CloneNotSupportedException e) {
throw new AssertionError(e);
}
}
@Override
public FieldMapper merge(Mapper mergeWith, boolean updateAllTypes) {
FieldMapper merged = clone();
merged.doMerge(mergeWith, updateAllTypes);
return merged;
}
/**
* Merge changes coming from {@code mergeWith} in place.
* @param updateAllTypes TODO
*/
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
if (!this.getClass().equals(mergeWith.getClass())) {
String mergedType = mergeWith.getClass().getSimpleName();
if (mergeWith instanceof FieldMapper) {
mergedType = ((FieldMapper) mergeWith).contentType();
}
mergeResult.addConflict("mapper [" + fieldType().names().fullName() + "] of different type, current_type [" + contentType() + "], merged_type [" + mergedType + "]");
// different types, return
return;
throw new IllegalArgumentException("mapper [" + fieldType().names().fullName() + "] of different type, current_type [" + contentType() + "], merged_type [" + mergedType + "]");
}
FieldMapper fieldMergeWith = (FieldMapper) mergeWith;
multiFields.merge(mergeWith, mergeResult);
multiFields = multiFields.merge(fieldMergeWith.multiFields);
if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
// apply changeable values
MappedFieldType fieldType = fieldMergeWith.fieldType().clone();
fieldType.freeze();
fieldTypeRef.set(fieldType);
this.copyTo = fieldMergeWith.copyTo;
}
// apply changeable values
MappedFieldType fieldType = fieldMergeWith.fieldType().clone();
fieldType.freeze();
fieldTypeRef.set(fieldType);
this.copyTo = fieldMergeWith.copyTo;
}
@Override
@ -520,18 +535,12 @@ public abstract class FieldMapper extends Mapper {
public static class MultiFields {
public static MultiFields empty() {
return new MultiFields(ContentPath.Type.FULL, ImmutableOpenMap.<String, FieldMapper>of());
return new MultiFields(ImmutableOpenMap.<String, FieldMapper>of());
}
public static class Builder {
private final ImmutableOpenMap.Builder<String, Mapper.Builder> mapperBuilders = ImmutableOpenMap.builder();
private ContentPath.Type pathType = ContentPath.Type.FULL;
public Builder pathType(ContentPath.Type pathType) {
this.pathType = pathType;
return this;
}
public Builder add(Mapper.Builder builder) {
mapperBuilders.put(builder.name(), builder);
@ -540,13 +549,9 @@ public abstract class FieldMapper extends Mapper {
@SuppressWarnings("unchecked")
public MultiFields build(FieldMapper.Builder mainFieldBuilder, BuilderContext context) {
if (pathType == ContentPath.Type.FULL && mapperBuilders.isEmpty()) {
if (mapperBuilders.isEmpty()) {
return empty();
} else if (mapperBuilders.isEmpty()) {
return new MultiFields(pathType, ImmutableOpenMap.<String, FieldMapper>of());
} else {
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(pathType);
context.path().add(mainFieldBuilder.name());
ImmutableOpenMap.Builder mapperBuilders = this.mapperBuilders;
for (ObjectObjectCursor<String, Mapper.Builder> cursor : this.mapperBuilders) {
@ -557,26 +562,25 @@ public abstract class FieldMapper extends Mapper {
mapperBuilders.put(key, mapper);
}
context.path().remove();
context.path().pathType(origPathType);
ImmutableOpenMap.Builder<String, FieldMapper> mappers = mapperBuilders.cast();
return new MultiFields(pathType, mappers.build());
return new MultiFields(mappers.build());
}
}
}
private final ContentPath.Type pathType;
private volatile ImmutableOpenMap<String, FieldMapper> mappers;
private final ImmutableOpenMap<String, FieldMapper> mappers;
public MultiFields(ContentPath.Type pathType, ImmutableOpenMap<String, FieldMapper> mappers) {
this.pathType = pathType;
this.mappers = mappers;
private MultiFields(ImmutableOpenMap<String, FieldMapper> mappers) {
ImmutableOpenMap.Builder<String, FieldMapper> builder = new ImmutableOpenMap.Builder<>();
// we disable the all in multi-field mappers
for (ObjectCursor<FieldMapper> cursor : mappers.values()) {
for (ObjectObjectCursor<String, FieldMapper> cursor : mappers) {
FieldMapper mapper = cursor.value;
if (mapper instanceof AllFieldMapper.IncludeInAll) {
((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll();
mapper = (FieldMapper) ((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll();
}
builder.put(cursor.key, mapper);
}
this.mappers = builder.build();
}
public void parse(FieldMapper mainField, ParseContext context) throws IOException {
@ -587,58 +591,33 @@ public abstract class FieldMapper extends Mapper {
context = context.createMultiFieldContext();
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(pathType);
context.path().add(mainField.simpleName());
for (ObjectCursor<FieldMapper> cursor : mappers.values()) {
cursor.value.parse(context);
}
context.path().remove();
context.path().pathType(origPathType);
}
// No need for locking, because locking is taken care of in ObjectMapper#merge and DocumentMapper#merge
public void merge(Mapper mergeWith, MergeResult mergeResult) {
FieldMapper mergeWithMultiField = (FieldMapper) mergeWith;
public MultiFields merge(MultiFields mergeWith) {
ImmutableOpenMap.Builder<String, FieldMapper> newMappersBuilder = ImmutableOpenMap.builder(mappers);
List<FieldMapper> newFieldMappers = null;
ImmutableOpenMap.Builder<String, FieldMapper> newMappersBuilder = null;
for (ObjectCursor<FieldMapper> cursor : mergeWithMultiField.multiFields.mappers.values()) {
for (ObjectCursor<FieldMapper> cursor : mergeWith.mappers.values()) {
FieldMapper mergeWithMapper = cursor.value;
Mapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName());
FieldMapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName());
if (mergeIntoMapper == null) {
// no mapping, simply add it if not simulating
if (!mergeResult.simulate()) {
// we disable the all in multi-field mappers
if (mergeWithMapper instanceof AllFieldMapper.IncludeInAll) {
((AllFieldMapper.IncludeInAll) mergeWithMapper).unsetIncludeInAll();
}
if (newMappersBuilder == null) {
newMappersBuilder = ImmutableOpenMap.builder(mappers);
}
newMappersBuilder.put(mergeWithMapper.simpleName(), mergeWithMapper);
if (mergeWithMapper instanceof FieldMapper) {
if (newFieldMappers == null) {
newFieldMappers = new ArrayList<>(2);
}
newFieldMappers.add(mergeWithMapper);
}
// we disable the all in multi-field mappers
if (mergeWithMapper instanceof AllFieldMapper.IncludeInAll) {
mergeWithMapper = (FieldMapper) ((AllFieldMapper.IncludeInAll) mergeWithMapper).unsetIncludeInAll();
}
newMappersBuilder.put(mergeWithMapper.simpleName(), mergeWithMapper);
} else {
mergeIntoMapper.merge(mergeWithMapper, mergeResult);
FieldMapper merged = mergeIntoMapper.merge(mergeWithMapper, false);
newMappersBuilder.put(merged.simpleName(), merged); // override previous definition
}
}
// first add all field mappers
if (newFieldMappers != null) {
mergeResult.addFieldMappers(newFieldMappers);
}
// now publish mappers
if (newMappersBuilder != null) {
mappers = newMappersBuilder.build();
}
ImmutableOpenMap<String, FieldMapper> mappers = newMappersBuilder.build();
return new MultiFields(mappers);
}
public Iterator<Mapper> iterator() {
@ -646,9 +625,6 @@ public abstract class FieldMapper extends Mapper {
}
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (pathType != ContentPath.Type.FULL) {
builder.field("path", pathType.name().toLowerCase(Locale.ROOT));
}
if (!mappers.isEmpty()) {
// sort the mappers so we get consistent serialization format
Mapper[] sortedMappers = mappers.values().toArray(Mapper.class);

View file

@ -174,5 +174,7 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
/** Returns the canonical name which uniquely identifies the mapper against other mappers in a type. */
public abstract String name();
public abstract void merge(Mapper mergeWith, MergeResult mergeResult);
/** Return the merge of {@code mergeWith} into this.
* Both {@code this} and {@code mergeWith} will be left unmodified. */
public abstract Mapper merge(Mapper mergeWith, boolean updateAllTypes);
}

View file

@ -32,7 +32,6 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.lucene.search.Queries;
@ -92,7 +91,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
private final ReleasableLock mappingWriteLock = new ReleasableLock(mappingLock.writeLock());
private volatile FieldTypeLookup fieldTypes;
private volatile ImmutableOpenMap<String, ObjectMapper> fullPathObjectMappers = ImmutableOpenMap.of();
private volatile Map<String, ObjectMapper> fullPathObjectMappers = new HashMap<>();
private boolean hasNested = false; // updated dynamically to true when a nested object is added
private final DocumentMapperParser documentParser;
@ -251,14 +250,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
DocumentMapper oldMapper = mappers.get(mapper.type());
if (oldMapper != null) {
// simulate first
MergeResult result = oldMapper.merge(mapper.mapping(), true, updateAllTypes);
if (result.hasConflicts()) {
throw new IllegalArgumentException("Merge failed with failures {" + Arrays.toString(result.buildConflicts()) + "}");
}
// then apply for real
result = oldMapper.merge(mapper.mapping(), false, updateAllTypes);
assert result.hasConflicts() == false; // we already simulated
oldMapper.merge(mapper.mapping(), false, updateAllTypes);
return oldMapper;
} else {
Tuple<Collection<ObjectMapper>, Collection<FieldMapper>> newMappers = checkMappersCompatibility(
@ -300,19 +292,56 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
return true;
}
private void checkFieldUniqueness(String type, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers) {
final Set<String> objectFullNames = new HashSet<>();
for (ObjectMapper objectMapper : objectMappers) {
final String fullPath = objectMapper.fullPath();
if (objectFullNames.add(fullPath) == false) {
throw new IllegalArgumentException("Object mapper [" + fullPath + "] is defined twice in mapping for type [" + type + "]");
}
}
if (indexSettings.getIndexVersionCreated().before(Version.V_3_0_0)) {
// Before 3.0 some metadata mappers are also registered under the root object mapper
// So we avoid false positives by deduplicating mappers
// given that we check exact equality, this would still catch the case that a mapper
// is defined under the root object
Collection<FieldMapper> uniqueFieldMappers = Collections.newSetFromMap(new IdentityHashMap<>());
uniqueFieldMappers.addAll(fieldMappers);
fieldMappers = uniqueFieldMappers;
}
final Set<String> fieldNames = new HashSet<>();
for (FieldMapper fieldMapper : fieldMappers) {
final String name = fieldMapper.name();
if (objectFullNames.contains(name)) {
throw new IllegalArgumentException("Field [" + name + "] is defined both as an object and a field in [" + type + "]");
} else if (fieldNames.add(name) == false) {
throw new IllegalArgumentException("Field [" + name + "] is defined twice in [" + type + "]");
}
}
}
protected void checkMappersCompatibility(String type, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers, boolean updateAllTypes) {
assert mappingLock.isWriteLockedByCurrentThread();
checkFieldUniqueness(type, objectMappers, fieldMappers);
for (ObjectMapper newObjectMapper : objectMappers) {
ObjectMapper existingObjectMapper = fullPathObjectMappers.get(newObjectMapper.fullPath());
if (existingObjectMapper != null) {
MergeResult result = new MergeResult(true, updateAllTypes);
existingObjectMapper.merge(newObjectMapper, result);
if (result.hasConflicts()) {
throw new IllegalArgumentException("Mapper for [" + newObjectMapper.fullPath() + "] conflicts with existing mapping in other types" +
Arrays.toString(result.buildConflicts()));
}
// simulate a merge and ignore the result, we are just interested
// in exceptions here
existingObjectMapper.merge(newObjectMapper, updateAllTypes);
}
}
for (FieldMapper fieldMapper : fieldMappers) {
if (fullPathObjectMappers.containsKey(fieldMapper.name())) {
throw new IllegalArgumentException("Field [" + fieldMapper.name() + "] is defined as a field in mapping [" + type + "] but this name is already used for an object in other types");
}
}
fieldTypes.checkCompatibility(type, fieldMappers, updateAllTypes);
}
@ -320,9 +349,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
String type, Mapping mapping, boolean updateAllTypes) {
List<ObjectMapper> objectMappers = new ArrayList<>();
List<FieldMapper> fieldMappers = new ArrayList<>();
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
fieldMappers.add(metadataMapper);
}
Collections.addAll(fieldMappers, mapping.metadataMappers);
MapperUtils.collect(mapping.root, objectMappers, fieldMappers);
checkMappersCompatibility(type, objectMappers, fieldMappers, updateAllTypes);
return new Tuple<>(objectMappers, fieldMappers);
@ -330,14 +357,14 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
protected void addMappers(String type, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers) {
assert mappingLock.isWriteLockedByCurrentThread();
ImmutableOpenMap.Builder<String, ObjectMapper> fullPathObjectMappers = ImmutableOpenMap.builder(this.fullPathObjectMappers);
Map<String, ObjectMapper> fullPathObjectMappers = new HashMap<>(this.fullPathObjectMappers);
for (ObjectMapper objectMapper : objectMappers) {
fullPathObjectMappers.put(objectMapper.fullPath(), objectMapper);
if (objectMapper.nested().isNested()) {
hasNested = true;
}
}
this.fullPathObjectMappers = fullPathObjectMappers.build();
this.fullPathObjectMappers = Collections.unmodifiableMap(fullPathObjectMappers);
this.fieldTypes = this.fieldTypes.copyAndAddAll(type, fieldMappers);
}

View file

@ -27,52 +27,6 @@ import java.util.Collection;
public enum MapperUtils {
;
private static MergeResult newStrictMergeResult() {
return new MergeResult(false, false) {
@Override
public void addFieldMappers(Collection<FieldMapper> fieldMappers) {
// no-op
}
@Override
public void addObjectMappers(Collection<ObjectMapper> objectMappers) {
// no-op
}
@Override
public Collection<FieldMapper> getNewFieldMappers() {
throw new UnsupportedOperationException("Strict merge result does not support new field mappers");
}
@Override
public Collection<ObjectMapper> getNewObjectMappers() {
throw new UnsupportedOperationException("Strict merge result does not support new object mappers");
}
@Override
public void addConflict(String mergeFailure) {
throw new MapperParsingException("Merging dynamic updates triggered a conflict: " + mergeFailure);
}
};
}
/**
* Merge {@code mergeWith} into {@code mergeTo}. Note: this method only
* merges mappings, not lookup structures. Conflicts are returned as exceptions.
*/
public static void merge(Mapper mergeInto, Mapper mergeWith) {
mergeInto.merge(mergeWith, newStrictMergeResult());
}
/**
* Merge {@code mergeWith} into {@code mergeTo}. Note: this method only
* merges mappings, not lookup structures. Conflicts are returned as exceptions.
*/
public static void merge(Mapping mergeInto, Mapping mergeWith) {
mergeInto.merge(mergeWith, newStrictMergeResult());
}
/** Split mapper and its descendants into object and field mappers. */
public static void collect(Mapper mapper, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers) {
if (mapper instanceof RootObjectMapper) {

View file

@ -27,10 +27,12 @@ import org.elasticsearch.index.mapper.object.RootObjectMapper;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
@ -41,25 +43,27 @@ import static java.util.Collections.unmodifiableMap;
*/
public final class Mapping implements ToXContent {
public static final List<String> LEGACY_INCLUDE_IN_OBJECT = Arrays.asList("_all", "_id", "_parent", "_routing", "_timestamp", "_ttl");
// Set of fields that were included into the root object mapper before 2.0
public static final Set<String> LEGACY_INCLUDE_IN_OBJECT = Collections.unmodifiableSet(new HashSet<>(
Arrays.asList("_all", "_id", "_parent", "_routing", "_timestamp", "_ttl")));
final Version indexCreated;
final RootObjectMapper root;
final MetadataFieldMapper[] metadataMappers;
final Map<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> metadataMappersMap;
volatile Map<String, Object> meta;
final Map<String, Object> meta;
public Mapping(Version indexCreated, RootObjectMapper rootObjectMapper, MetadataFieldMapper[] metadataMappers, Map<String, Object> meta) {
this.indexCreated = indexCreated;
this.root = rootObjectMapper;
this.metadataMappers = metadataMappers;
Map<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> metadataMappersMap = new HashMap<>();
for (MetadataFieldMapper metadataMapper : metadataMappers) {
if (indexCreated.before(Version.V_2_0_0_beta1) && LEGACY_INCLUDE_IN_OBJECT.contains(metadataMapper.name())) {
root.putMapper(metadataMapper);
rootObjectMapper = rootObjectMapper.copyAndPutMapper(metadataMapper);
}
metadataMappersMap.put(metadataMapper.getClass(), metadataMapper);
}
this.root = rootObjectMapper;
// keep root mappers sorted for consistent serialization
Arrays.sort(metadataMappers, new Comparator<Mapper>() {
@Override
@ -90,21 +94,20 @@ public final class Mapping implements ToXContent {
}
/** @see DocumentMapper#merge(Mapping, boolean, boolean) */
public void merge(Mapping mergeWith, MergeResult mergeResult) {
assert metadataMappers.length == mergeWith.metadataMappers.length;
root.merge(mergeWith.root, mergeResult);
for (MetadataFieldMapper metadataMapper : metadataMappers) {
MetadataFieldMapper mergeWithMetadataMapper = mergeWith.metadataMapper(metadataMapper.getClass());
if (mergeWithMetadataMapper != null) {
metadataMapper.merge(mergeWithMetadataMapper, mergeResult);
public Mapping merge(Mapping mergeWith, boolean updateAllTypes) {
RootObjectMapper mergedRoot = root.merge(mergeWith.root, updateAllTypes);
Map<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> mergedMetaDataMappers = new HashMap<>(metadataMappersMap);
for (MetadataFieldMapper metaMergeWith : mergeWith.metadataMappers) {
MetadataFieldMapper mergeInto = mergedMetaDataMappers.get(metaMergeWith.getClass());
MetadataFieldMapper merged;
if (mergeInto == null) {
merged = metaMergeWith;
} else {
merged = mergeInto.merge(metaMergeWith, updateAllTypes);
}
mergedMetaDataMappers.put(merged.getClass(), merged);
}
if (mergeResult.simulate() == false) {
// let the merge with attributes to override the attributes
meta = mergeWith.meta;
}
return new Mapping(indexCreated, mergedRoot, mergedMetaDataMappers.values().toArray(new MetadataFieldMapper[0]), mergeWith.meta);
}
@Override

View file

@ -1,81 +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.mapper;
import org.elasticsearch.common.Strings;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
/** A container for tracking results of a mapping merge. */
public class MergeResult {
private final boolean simulate;
private final boolean updateAllTypes;
private final List<String> conflicts = new ArrayList<>();
private final List<FieldMapper> newFieldMappers = new ArrayList<>();
private final List<ObjectMapper> newObjectMappers = new ArrayList<>();
public MergeResult(boolean simulate, boolean updateAllTypes) {
this.simulate = simulate;
this.updateAllTypes = updateAllTypes;
}
public void addFieldMappers(Collection<FieldMapper> fieldMappers) {
assert simulate() == false;
newFieldMappers.addAll(fieldMappers);
}
public void addObjectMappers(Collection<ObjectMapper> objectMappers) {
assert simulate() == false;
newObjectMappers.addAll(objectMappers);
}
public Collection<FieldMapper> getNewFieldMappers() {
return newFieldMappers;
}
public Collection<ObjectMapper> getNewObjectMappers() {
return newObjectMappers;
}
public boolean simulate() {
return simulate;
}
public boolean updateAllTypes() {
return updateAllTypes;
}
public void addConflict(String mergeFailure) {
conflicts.add(mergeFailure);
}
public boolean hasConflicts() {
return conflicts.isEmpty() == false;
}
public String[] buildConflicts() {
return conflicts.toArray(Strings.EMPTY_ARRAY);
}
}

View file

@ -70,4 +70,8 @@ public abstract class MetadataFieldMapper extends FieldMapper {
*/
public abstract void postParse(ParseContext context) throws IOException;
@Override
public MetadataFieldMapper merge(Mapper mergeWith, boolean updateAllTypes) {
return (MetadataFieldMapper) super.merge(mergeWith, updateAllTypes);
}
}

View file

@ -595,7 +595,7 @@ public abstract class ParseContext {
if (dynamicMappingsUpdate == null) {
dynamicMappingsUpdate = mapper;
} else {
MapperUtils.merge(dynamicMappingsUpdate, mapper);
dynamicMappingsUpdate = dynamicMappingsUpdate.merge(mapper, false);
}
}

View file

@ -128,7 +128,7 @@ public class ParsedDocument {
if (dynamicMappingsUpdate == null) {
dynamicMappingsUpdate = update;
} else {
MapperUtils.merge(dynamicMappingsUpdate, update);
dynamicMappingsUpdate = dynamicMappingsUpdate.merge(update, false);
}
}

View file

@ -77,8 +77,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
setupFieldType(context);
ByteFieldMapper fieldMapper = new ByteFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context),
coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (ByteFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -605,11 +605,9 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
CompletionFieldMapper fieldMergeWith = (CompletionFieldMapper) mergeWith;
if (!mergeResult.simulate()) {
this.maxInputLength = fieldMergeWith.maxInputLength;
}
this.maxInputLength = fieldMergeWith.maxInputLength;
}
}

View file

@ -123,8 +123,7 @@ public class DateFieldMapper extends NumberFieldMapper {
fieldType.setNullValue(nullValue);
DateFieldMapper fieldMapper = new DateFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context),
coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (DateFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -80,8 +80,7 @@ public class DoubleFieldMapper extends NumberFieldMapper {
setupFieldType(context);
DoubleFieldMapper fieldMapper = new DoubleFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context),
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (DoubleFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -81,8 +81,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
setupFieldType(context);
FloatFieldMapper fieldMapper = new FloatFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context),
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (FloatFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -85,8 +85,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
IntegerFieldMapper fieldMapper = new IntegerFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context),
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (IntegerFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -84,8 +84,7 @@ public class LongFieldMapper extends NumberFieldMapper {
setupFieldType(context);
LongFieldMapper fieldMapper = new LongFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (LongFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -183,22 +183,41 @@ public abstract class NumberFieldMapper extends FieldMapper implements AllFieldM
}
@Override
public void includeInAll(Boolean includeInAll) {
protected NumberFieldMapper clone() {
return (NumberFieldMapper) super.clone();
}
@Override
public Mapper includeInAll(Boolean includeInAll) {
if (includeInAll != null) {
this.includeInAll = includeInAll;
NumberFieldMapper clone = clone();
clone.includeInAll = includeInAll;
return clone;
} else {
return this;
}
}
@Override
public void includeInAllIfNotSet(Boolean includeInAll) {
public Mapper includeInAllIfNotSet(Boolean includeInAll) {
if (includeInAll != null && this.includeInAll == null) {
this.includeInAll = includeInAll;
NumberFieldMapper clone = clone();
clone.includeInAll = includeInAll;
return clone;
} else {
return this;
}
}
@Override
public void unsetIncludeInAll() {
includeInAll = null;
public Mapper unsetIncludeInAll() {
if (includeInAll != null) {
NumberFieldMapper clone = clone();
clone.includeInAll = null;
return clone;
} else {
return this;
}
}
@Override
@ -254,21 +273,16 @@ public abstract class NumberFieldMapper extends FieldMapper implements AllFieldM
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;
}
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
NumberFieldMapper nfmMergeWith = (NumberFieldMapper) mergeWith;
if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
this.includeInAll = nfmMergeWith.includeInAll;
if (nfmMergeWith.ignoreMalformed.explicit()) {
this.ignoreMalformed = nfmMergeWith.ignoreMalformed;
}
if (nfmMergeWith.coerce.explicit()) {
this.coerce = nfmMergeWith.coerce;
}
this.includeInAll = nfmMergeWith.includeInAll;
if (nfmMergeWith.ignoreMalformed.explicit()) {
this.ignoreMalformed = nfmMergeWith.ignoreMalformed;
}
if (nfmMergeWith.coerce.explicit()) {
this.coerce = nfmMergeWith.coerce;
}
}

View file

@ -81,8 +81,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
ShortFieldMapper fieldMapper = new ShortFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context),
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (ShortFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -35,7 +35,6 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
@ -150,8 +149,7 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
StringFieldMapper fieldMapper = new StringFieldMapper(
name, fieldType, defaultFieldType, positionIncrementGap, ignoreAbove,
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return fieldMapper.includeInAll(includeInAll);
}
}
@ -257,22 +255,41 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
}
@Override
public void includeInAll(Boolean includeInAll) {
protected StringFieldMapper clone() {
return (StringFieldMapper) super.clone();
}
@Override
public StringFieldMapper includeInAll(Boolean includeInAll) {
if (includeInAll != null) {
this.includeInAll = includeInAll;
StringFieldMapper clone = clone();
clone.includeInAll = includeInAll;
return clone;
} else {
return this;
}
}
@Override
public void includeInAllIfNotSet(Boolean includeInAll) {
public StringFieldMapper includeInAllIfNotSet(Boolean includeInAll) {
if (includeInAll != null && this.includeInAll == null) {
this.includeInAll = includeInAll;
StringFieldMapper clone = clone();
clone.includeInAll = includeInAll;
return clone;
} else {
return this;
}
}
@Override
public void unsetIncludeInAll() {
includeInAll = null;
public StringFieldMapper unsetIncludeInAll() {
if (includeInAll != null) {
StringFieldMapper clone = clone();
clone.includeInAll = null;
return clone;
} else {
return this;
}
}
@Override
@ -359,15 +376,10 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;
}
if (!mergeResult.simulate()) {
this.includeInAll = ((StringFieldMapper) mergeWith).includeInAll;
this.ignoreAbove = ((StringFieldMapper) mergeWith).ignoreAbove;
}
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
this.includeInAll = ((StringFieldMapper) mergeWith).includeInAll;
this.ignoreAbove = ((StringFieldMapper) mergeWith).ignoreAbove;
}
@Override

View file

@ -33,7 +33,6 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.StringFieldMapper.ValueAndBoost;
@ -81,8 +80,7 @@ public class TokenCountFieldMapper extends IntegerFieldMapper {
TokenCountFieldMapper fieldMapper = new TokenCountFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context), context.indexSettings(),
analyzer, multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (TokenCountFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override
@ -190,14 +188,9 @@ public class TokenCountFieldMapper extends IntegerFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;
}
if (!mergeResult.simulate()) {
this.analyzer = ((TokenCountFieldMapper) mergeWith).analyzer;
}
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
this.analyzer = ((TokenCountFieldMapper) mergeWith).analyzer;
}
@Override

View file

@ -61,7 +61,6 @@ public class TypeParsers {
@Override
public Mapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
ContentPath.Type pathType = null;
FieldMapper.Builder mainFieldBuilder = null;
List<FieldMapper.Builder> fields = null;
String firstType = null;
@ -70,10 +69,7 @@ public class TypeParsers {
Map.Entry<String, Object> entry = iterator.next();
String fieldName = Strings.toUnderscoreCase(entry.getKey());
Object fieldNode = entry.getValue();
if (fieldName.equals("path") && parserContext.indexVersionCreated().before(Version.V_2_0_0_beta1)) {
pathType = parsePathType(name, fieldNode.toString());
iterator.remove();
} else if (fieldName.equals("fields")) {
if (fieldName.equals("fields")) {
Map<String, Object> fieldsNode = (Map<String, Object>) fieldNode;
for (Iterator<Map.Entry<String, Object>> fieldsIterator = fieldsNode.entrySet().iterator(); fieldsIterator.hasNext();) {
Map.Entry<String, Object> entry1 = fieldsIterator.next();
@ -132,17 +128,10 @@ public class TypeParsers {
}
}
if (fields != null && pathType != null) {
if (fields != null) {
for (Mapper.Builder field : fields) {
mainFieldBuilder.addMultiField(field);
}
mainFieldBuilder.multiFieldPathType(pathType);
} else if (fields != null) {
for (Mapper.Builder field : fields) {
mainFieldBuilder.addMultiField(field);
}
} else if (pathType != null) {
mainFieldBuilder.multiFieldPathType(pathType);
}
return mainFieldBuilder;
}
@ -337,10 +326,7 @@ public class TypeParsers {
public static boolean parseMultiField(FieldMapper.Builder builder, String name, Mapper.TypeParser.ParserContext parserContext, String propName, Object propNode) {
parserContext = parserContext.createMultiFieldContext(parserContext);
if (propName.equals("path") && parserContext.indexVersionCreated().before(Version.V_2_0_0_beta1)) {
builder.multiFieldPathType(parsePathType(name, propNode.toString()));
return true;
} else if (propName.equals("fields")) {
if (propName.equals("fields")) {
final Map<String, Object> multiFieldsPropNodes;
@ -457,17 +443,6 @@ public class TypeParsers {
}
}
public static ContentPath.Type parsePathType(String name, String path) throws MapperParsingException {
path = Strings.toUnderscoreCase(path);
if ("just_name".equals(path)) {
return ContentPath.Type.JUST_NAME;
} else if ("full".equals(path)) {
return ContentPath.Type.FULL;
} else {
throw new MapperParsingException("wrong value for pathType [" + path + "] for object [" + name + "]");
}
}
@SuppressWarnings("unchecked")
public static void parseCopyFields(Object propNode, FieldMapper.Builder builder) {
FieldMapper.CopyTo.Builder copyToBuilder = new FieldMapper.CopyTo.Builder();

View file

@ -33,12 +33,10 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.DoubleFieldMapper;
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
@ -74,7 +72,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
public static class Defaults {
public static final ContentPath.Type PATH_TYPE = ContentPath.Type.FULL;
public static final boolean ENABLE_LATLON = false;
public static final boolean ENABLE_GEOHASH = false;
public static final boolean ENABLE_GEOHASH_PREFIX = false;
@ -83,7 +80,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
public abstract static class Builder<T extends Builder, Y extends BaseGeoPointFieldMapper> extends FieldMapper.Builder<T, Y> {
protected ContentPath.Type pathType = Defaults.PATH_TYPE;
protected boolean enableLatLon = Defaults.ENABLE_LATLON;
@ -106,12 +102,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
return (GeoPointFieldType)fieldType;
}
@Override
public T multiFieldPathType(ContentPath.Type pathType) {
this.pathType = pathType;
return builder;
}
@Override
public T fieldDataSettings(Settings settings) {
this.fieldDataSettings = settings;
@ -159,13 +149,10 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
public abstract Y build(BuilderContext context, String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Settings indexSettings, ContentPath.Type pathType, DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper,
Settings indexSettings, DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper,
StringFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed, CopyTo copyTo);
public Y build(Mapper.BuilderContext context) {
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(pathType);
GeoPointFieldType geoPointFieldType = (GeoPointFieldType)fieldType;
DoubleFieldMapper latMapper = null;
@ -191,9 +178,8 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
geoPointFieldType.setGeoHashEnabled(geoHashMapper.fieldType(), geoHashPrecision, enableGeoHashPrefix);
}
context.path().remove();
context.path().pathType(origPathType);
return build(context, name, fieldType, defaultFieldType, context.indexSettings(), origPathType,
return build(context, name, fieldType, defaultFieldType, context.indexSettings(),
latMapper, lonMapper, geoHashMapper, multiFieldsBuilder.build(this, context), ignoreMalformed(context), copyTo);
}
}
@ -365,17 +351,14 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
protected final DoubleFieldMapper lonMapper;
protected final ContentPath.Type pathType;
protected final StringFieldMapper geoHashMapper;
protected Explicit<Boolean> ignoreMalformed;
protected BaseGeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings,
ContentPath.Type pathType, DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper, StringFieldMapper geoHashMapper,
DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper, StringFieldMapper geoHashMapper,
MultiFields multiFields, Explicit<Boolean> ignoreMalformed, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
this.pathType = pathType;
this.latMapper = latMapper;
this.lonMapper = lonMapper;
this.geoHashMapper = geoHashMapper;
@ -388,17 +371,11 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;
}
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
BaseGeoPointFieldMapper gpfmMergeWith = (BaseGeoPointFieldMapper) mergeWith;
if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
if (gpfmMergeWith.ignoreMalformed.explicit()) {
this.ignoreMalformed = gpfmMergeWith.ignoreMalformed;
}
if (gpfmMergeWith.ignoreMalformed.explicit()) {
this.ignoreMalformed = gpfmMergeWith.ignoreMalformed;
}
}
@ -441,8 +418,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
@Override
public Mapper parse(ParseContext context) throws IOException {
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(pathType);
context.path().add(simpleName());
GeoPoint sparse = context.parseExternalValue(GeoPoint.class);
@ -487,7 +462,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
context.path().remove();
context.path().pathType(origPathType);
return null;
}
@ -512,9 +486,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || pathType != Defaults.PATH_TYPE) {
builder.field("path", pathType.name().toLowerCase(Locale.ROOT));
}
if (includeDefaults || fieldType().isLatLonEnabled() != GeoPointFieldMapper.Defaults.ENABLE_LATLON) {
builder.field("lat_lon", fieldType().isLatLonEnabled());
}

View file

@ -27,7 +27,6 @@ import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -81,12 +80,12 @@ public class GeoPointFieldMapper extends BaseGeoPointFieldMapper {
@Override
public GeoPointFieldMapper build(BuilderContext context, String simpleName, MappedFieldType fieldType,
MappedFieldType defaultFieldType, Settings indexSettings, ContentPath.Type pathType, DoubleFieldMapper latMapper,
MappedFieldType defaultFieldType, Settings indexSettings, DoubleFieldMapper latMapper,
DoubleFieldMapper lonMapper, StringFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed,
CopyTo copyTo) {
fieldType.setTokenized(false);
setupFieldType(context);
return new GeoPointFieldMapper(simpleName, fieldType, defaultFieldType, indexSettings, pathType, latMapper, lonMapper,
return new GeoPointFieldMapper(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper,
geoHashMapper, multiFields, ignoreMalformed, copyTo);
}
@ -104,9 +103,9 @@ public class GeoPointFieldMapper extends BaseGeoPointFieldMapper {
}
public GeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings,
ContentPath.Type pathType, DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper,
DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper,
StringFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, pathType, latMapper, lonMapper, geoHashMapper, multiFields,
super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields,
ignoreMalformed, copyTo);
}

View file

@ -35,11 +35,9 @@ import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.util.ByteUtils;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.DoubleFieldMapper;
import org.elasticsearch.index.mapper.core.NumberFieldMapper.CustomNumericDocValuesField;
@ -111,14 +109,14 @@ public class GeoPointFieldMapperLegacy extends BaseGeoPointFieldMapper implement
@Override
public GeoPointFieldMapperLegacy build(BuilderContext context, String simpleName, MappedFieldType fieldType,
MappedFieldType defaultFieldType, Settings indexSettings, ContentPath.Type pathType, DoubleFieldMapper latMapper,
MappedFieldType defaultFieldType, Settings indexSettings, DoubleFieldMapper latMapper,
DoubleFieldMapper lonMapper, StringFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed,
CopyTo copyTo) {
fieldType.setTokenized(false);
setupFieldType(context);
fieldType.setHasDocValues(false);
defaultFieldType.setHasDocValues(false);
return new GeoPointFieldMapperLegacy(simpleName, fieldType, defaultFieldType, indexSettings, pathType, latMapper, lonMapper,
return new GeoPointFieldMapperLegacy(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper,
geoHashMapper, multiFields, ignoreMalformed, coerce(context), copyTo);
}
@ -288,32 +286,27 @@ public class GeoPointFieldMapperLegacy extends BaseGeoPointFieldMapper implement
protected Explicit<Boolean> coerce;
public GeoPointFieldMapperLegacy(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings,
ContentPath.Type pathType, DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper,
DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper,
StringFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed,
Explicit<Boolean> coerce, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, pathType, latMapper, lonMapper, geoHashMapper, multiFields,
super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields,
ignoreMalformed, copyTo);
this.coerce = coerce;
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;
}
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
GeoPointFieldMapperLegacy gpfmMergeWith = (GeoPointFieldMapperLegacy) mergeWith;
if (gpfmMergeWith.coerce.explicit()) {
if (coerce.explicit() && coerce.value() != gpfmMergeWith.coerce.value()) {
mergeResult.addConflict("mapper [" + fieldType().names().fullName() + "] has different [coerce]");
throw new IllegalArgumentException("mapper [" + fieldType().names().fullName() + "] has different [coerce]");
}
}
if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
if (gpfmMergeWith.coerce.explicit()) {
this.coerce = gpfmMergeWith.coerce;
}
if (gpfmMergeWith.coerce.explicit()) {
this.coerce = gpfmMergeWith.coerce;
}
}

View file

@ -45,7 +45,6 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import java.io.IOException;
@ -475,17 +474,12 @@ public class GeoShapeFieldMapper extends FieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;
}
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
GeoShapeFieldMapper gsfm = (GeoShapeFieldMapper)mergeWith;
if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
if (gsfm.coerce.explicit()) {
this.coerce = gsfm.coerce;
}
if (gsfm.coerce.explicit()) {
this.coerce = gsfm.coerce;
}
}

View file

@ -36,7 +36,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryShardContext;
@ -58,11 +57,24 @@ public class AllFieldMapper extends MetadataFieldMapper {
public interface IncludeInAll {
void includeInAll(Boolean includeInAll);
/**
* If {@code includeInAll} is not null then return a copy of this mapper
* that will include values in the _all field according to {@code includeInAll}.
*/
Mapper includeInAll(Boolean includeInAll);
void includeInAllIfNotSet(Boolean includeInAll);
/**
* If {@code includeInAll} is not null and not set on this mapper yet, then
* return a copy of this mapper that will include values in the _all field
* according to {@code includeInAll}.
*/
Mapper includeInAllIfNotSet(Boolean includeInAll);
void unsetIncludeInAll();
/**
* If {@code includeInAll} was already set on this mapper then return a copy
* of this mapper that has {@code includeInAll} not set.
*/
Mapper unsetIncludeInAll();
}
public static final String NAME = "_all";
@ -309,11 +321,11 @@ public class AllFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
if (((AllFieldMapper)mergeWith).enabled() != this.enabled() && ((AllFieldMapper)mergeWith).enabledState != Defaults.ENABLED) {
mergeResult.addConflict("mapper [" + fieldType().names().fullName() + "] enabled is " + this.enabled() + " now encountering "+ ((AllFieldMapper)mergeWith).enabled());
throw new IllegalArgumentException("mapper [" + fieldType().names().fullName() + "] enabled is " + this.enabled() + " now encountering "+ ((AllFieldMapper)mergeWith).enabled());
}
super.merge(mergeWith, mergeResult);
super.doMerge(mergeWith, updateAllTypes);
}
@Override

View file

@ -44,7 +44,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.Uid;
@ -331,7 +330,7 @@ public class IdFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// do nothing here, no merging, but also no exception
}
}

View file

@ -34,7 +34,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryShardContext;
@ -279,12 +278,10 @@ public class IndexFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
IndexFieldMapper indexFieldMapperMergeWith = (IndexFieldMapper) mergeWith;
if (!mergeResult.simulate()) {
if (indexFieldMapperMergeWith.enabledState != enabledState && !indexFieldMapperMergeWith.enabledState.unset()) {
this.enabledState = indexFieldMapperMergeWith.enabledState;
}
if (indexFieldMapperMergeWith.enabledState != enabledState && !indexFieldMapperMergeWith.enabledState.unset()) {
this.enabledState = indexFieldMapperMergeWith.enabledState;
}
}

View file

@ -38,7 +38,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.Uid;
@ -371,11 +370,11 @@ public class ParentFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
ParentFieldMapper fieldMergeWith = (ParentFieldMapper) mergeWith;
if (Objects.equals(parentType, fieldMergeWith.parentType) == false) {
mergeResult.addConflict("The _parent field's type option can't be changed: [" + parentType + "]->[" + fieldMergeWith.parentType + "]");
throw new IllegalArgumentException("The _parent field's type option can't be changed: [" + parentType + "]->[" + fieldMergeWith.parentType + "]");
}
List<String> conflicts = new ArrayList<>();
@ -383,13 +382,13 @@ public class ParentFieldMapper extends MetadataFieldMapper {
parentJoinFieldType.checkCompatibility(fieldMergeWith.parentJoinFieldType, conflicts, true); // same here
if (childJoinFieldType != null) {
// TODO: this can be set to false when the old parent/child impl is removed, we can do eager global ordinals loading per type.
childJoinFieldType.checkCompatibility(fieldMergeWith.childJoinFieldType, conflicts, mergeResult.updateAllTypes() == false);
childJoinFieldType.checkCompatibility(fieldMergeWith.childJoinFieldType, conflicts, updateAllTypes == false);
}
for (String conflict : conflicts) {
mergeResult.addConflict(conflict);
if (conflicts.isEmpty() == false) {
throw new IllegalArgumentException("Merge conflicts: " + conflicts);
}
if (active() && mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
if (active()) {
childJoinFieldType = fieldMergeWith.childJoinFieldType.clone();
}
}

View file

@ -31,7 +31,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -249,7 +248,7 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// do nothing here, no merging, but also no exception
}
}

View file

@ -41,11 +41,11 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
@ -310,18 +310,20 @@ public class SourceFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
SourceFieldMapper sourceMergeWith = (SourceFieldMapper) mergeWith;
if (mergeResult.simulate()) {
if (this.enabled != sourceMergeWith.enabled) {
mergeResult.addConflict("Cannot update enabled setting for [_source]");
}
if (Arrays.equals(includes(), sourceMergeWith.includes()) == false) {
mergeResult.addConflict("Cannot update includes setting for [_source]");
}
if (Arrays.equals(excludes(), sourceMergeWith.excludes()) == false) {
mergeResult.addConflict("Cannot update excludes setting for [_source]");
}
List<String> conflicts = new ArrayList<>();
if (this.enabled != sourceMergeWith.enabled) {
conflicts.add("Cannot update enabled setting for [_source]");
}
if (Arrays.equals(includes(), sourceMergeWith.includes()) == false) {
conflicts.add("Cannot update includes setting for [_source]");
}
if (Arrays.equals(excludes(), sourceMergeWith.excludes()) == false) {
conflicts.add("Cannot update excludes setting for [_source]");
}
if (conflicts.isEmpty() == false) {
throw new IllegalArgumentException("Can't merge because of conflicts: " + conflicts);
}
}
}

View file

@ -32,7 +32,6 @@ import org.elasticsearch.index.analysis.NumericLongAnalyzer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.SourceToParse;
@ -258,21 +257,19 @@ public class TTLFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
TTLFieldMapper ttlMergeWith = (TTLFieldMapper) mergeWith;
if (((TTLFieldMapper) mergeWith).enabledState != Defaults.ENABLED_STATE) {//only do something if actually something was set for the document mapper that we merge with
if (this.enabledState == EnabledAttributeMapper.ENABLED && ((TTLFieldMapper) mergeWith).enabledState == EnabledAttributeMapper.DISABLED) {
mergeResult.addConflict("_ttl cannot be disabled once it was enabled.");
if (ttlMergeWith.enabledState != Defaults.ENABLED_STATE) {//only do something if actually something was set for the document mapper that we merge with
if (this.enabledState == EnabledAttributeMapper.ENABLED && ttlMergeWith.enabledState == EnabledAttributeMapper.DISABLED) {
throw new IllegalArgumentException("_ttl cannot be disabled once it was enabled.");
} else {
if (!mergeResult.simulate()) {
this.enabledState = ttlMergeWith.enabledState;
}
this.enabledState = ttlMergeWith.enabledState;
}
}
if (ttlMergeWith.defaultTTL != -1) {
// we never build the default when the field is disabled so we should also not set it
// (it does not make a difference though as everything that is not build in toXContent will also not be set in the cluster)
if (!mergeResult.simulate() && (enabledState == EnabledAttributeMapper.ENABLED)) {
if (enabledState == EnabledAttributeMapper.ENABLED) {
this.defaultTTL = ttlMergeWith.defaultTTL;
}
}

View file

@ -33,13 +33,13 @@ import org.elasticsearch.index.analysis.NumericDateAnalyzer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.core.DateFieldMapper;
import org.elasticsearch.index.mapper.core.LongFieldMapper;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -379,31 +379,32 @@ public class TimestampFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
TimestampFieldMapper timestampFieldMapperMergeWith = (TimestampFieldMapper) mergeWith;
super.merge(mergeWith, mergeResult);
if (!mergeResult.simulate()) {
if (timestampFieldMapperMergeWith.enabledState != enabledState && !timestampFieldMapperMergeWith.enabledState.unset()) {
this.enabledState = timestampFieldMapperMergeWith.enabledState;
}
} else {
if (timestampFieldMapperMergeWith.defaultTimestamp() == null && defaultTimestamp == null) {
return;
}
if (defaultTimestamp == null) {
mergeResult.addConflict("Cannot update default in _timestamp value. Value is null now encountering " + timestampFieldMapperMergeWith.defaultTimestamp());
} else if (timestampFieldMapperMergeWith.defaultTimestamp() == null) {
mergeResult.addConflict("Cannot update default in _timestamp value. Value is \" + defaultTimestamp.toString() + \" now encountering null");
} else if (!timestampFieldMapperMergeWith.defaultTimestamp().equals(defaultTimestamp)) {
mergeResult.addConflict("Cannot update default in _timestamp value. Value is " + defaultTimestamp.toString() + " now encountering " + timestampFieldMapperMergeWith.defaultTimestamp());
}
if (this.path != null) {
if (path.equals(timestampFieldMapperMergeWith.path()) == false) {
mergeResult.addConflict("Cannot update path in _timestamp value. Value is " + path + " path in merged mapping is " + (timestampFieldMapperMergeWith.path() == null ? "missing" : timestampFieldMapperMergeWith.path()));
}
} else if (timestampFieldMapperMergeWith.path() != null) {
mergeResult.addConflict("Cannot update path in _timestamp value. Value is " + path + " path in merged mapping is missing");
super.doMerge(mergeWith, updateAllTypes);
if (timestampFieldMapperMergeWith.enabledState != enabledState && !timestampFieldMapperMergeWith.enabledState.unset()) {
this.enabledState = timestampFieldMapperMergeWith.enabledState;
}
if (timestampFieldMapperMergeWith.defaultTimestamp() == null && defaultTimestamp == null) {
return;
}
List<String> conflicts = new ArrayList<>();
if (defaultTimestamp == null) {
conflicts.add("Cannot update default in _timestamp value. Value is null now encountering " + timestampFieldMapperMergeWith.defaultTimestamp());
} else if (timestampFieldMapperMergeWith.defaultTimestamp() == null) {
conflicts.add("Cannot update default in _timestamp value. Value is \" + defaultTimestamp.toString() + \" now encountering null");
} else if (!timestampFieldMapperMergeWith.defaultTimestamp().equals(defaultTimestamp)) {
conflicts.add("Cannot update default in _timestamp value. Value is " + defaultTimestamp.toString() + " now encountering " + timestampFieldMapperMergeWith.defaultTimestamp());
}
if (this.path != null) {
if (path.equals(timestampFieldMapperMergeWith.path()) == false) {
conflicts.add("Cannot update path in _timestamp value. Value is " + path + " path in merged mapping is " + (timestampFieldMapperMergeWith.path() == null ? "missing" : timestampFieldMapperMergeWith.path()));
}
} else if (timestampFieldMapperMergeWith.path() != null) {
conflicts.add("Cannot update path in _timestamp value. Value is " + path + " path in merged mapping is missing");
}
if (conflicts.isEmpty() == false) {
throw new IllegalArgumentException("Conflicts: " + conflicts);
}
}
}

View file

@ -40,7 +40,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.Uid;
@ -225,7 +224,7 @@ public class TypeFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// do nothing here, no merging, but also no exception
}
}

View file

@ -33,7 +33,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
@ -225,7 +224,7 @@ public class UidFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// do nothing here, no merging, but also no exception
}
}

View file

@ -30,7 +30,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
@ -166,7 +165,7 @@ public class VersionFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// nothing to do
}
}

View file

@ -122,8 +122,7 @@ public class IpFieldMapper extends NumberFieldMapper {
setupFieldType(context);
IpFieldMapper fieldMapper = new IpFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context),
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (IpFieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -125,13 +125,13 @@ public class DynamicTemplate {
}
public boolean match(ContentPath path, String name, String dynamicType) {
if (pathMatch != null && !patternMatch(pathMatch, path.fullPathAsText(name))) {
if (pathMatch != null && !patternMatch(pathMatch, path.pathAsText(name))) {
return false;
}
if (match != null && !patternMatch(match, name)) {
return false;
}
if (pathUnmatch != null && patternMatch(pathUnmatch, path.fullPathAsText(name))) {
if (pathUnmatch != null && patternMatch(pathUnmatch, path.pathAsText(name))) {
return false;
}
if (unmatch != null && patternMatch(unmatch, name)) {

View file

@ -24,7 +24,6 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.CopyOnWriteHashMap;
@ -40,7 +39,6 @@ import java.util.*;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeBooleanValue;
import static org.elasticsearch.index.mapper.MapperBuilders.object;
import static org.elasticsearch.index.mapper.core.TypeParsers.parsePathType;
/**
*
@ -54,7 +52,6 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
public static final boolean ENABLED = true;
public static final Nested NESTED = Nested.NO;
public static final Dynamic DYNAMIC = null; // not set, inherited from root
public static final ContentPath.Type PATH_TYPE = ContentPath.Type.FULL;
}
public static enum Dynamic {
@ -104,8 +101,6 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
protected Dynamic dynamic = Defaults.DYNAMIC;
protected ContentPath.Type pathType = Defaults.PATH_TYPE;
protected Boolean includeInAll;
protected final List<Mapper.Builder> mappersBuilders = new ArrayList<>();
@ -130,11 +125,6 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
return builder;
}
public T pathType(ContentPath.Type pathType) {
this.pathType = pathType;
return builder;
}
public T includeInAll(boolean includeInAll) {
this.includeInAll = includeInAll;
return builder;
@ -147,8 +137,6 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
@Override
public Y build(BuilderContext context) {
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(pathType);
context.path().add(name);
Map<String, Mapper> mappers = new HashMap<>();
@ -156,17 +144,16 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
Mapper mapper = builder.build(context);
mappers.put(mapper.simpleName(), mapper);
}
context.path().pathType(origPathType);
context.path().remove();
ObjectMapper objectMapper = createMapper(name, context.path().fullPathAsText(name), enabled, nested, dynamic, pathType, mappers, context.indexSettings());
objectMapper.includeInAllIfNotSet(includeInAll);
ObjectMapper objectMapper = createMapper(name, context.path().pathAsText(name), enabled, nested, dynamic, mappers, context.indexSettings());
objectMapper = objectMapper.includeInAllIfNotSet(includeInAll);
return (Y) objectMapper;
}
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers, @Nullable Settings settings) {
return new ObjectMapper(name, fullPath, enabled, nested, dynamic, pathType, mappers);
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map<String, Mapper> mappers, @Nullable Settings settings) {
return new ObjectMapper(name, fullPath, enabled, nested, dynamic, mappers);
}
}
@ -179,7 +166,7 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
Map.Entry<String, Object> entry = iterator.next();
String fieldName = Strings.toUnderscoreCase(entry.getKey());
Object fieldNode = entry.getValue();
if (parseObjectOrDocumentTypeProperties(fieldName, fieldNode, parserContext, builder) || parseObjectProperties(name, fieldName, fieldNode, parserContext, builder)) {
if (parseObjectOrDocumentTypeProperties(fieldName, fieldNode, parserContext, builder)) {
iterator.remove();
}
}
@ -214,14 +201,6 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
return false;
}
protected static boolean parseObjectProperties(String name, String fieldName, Object fieldNode, ParserContext parserContext, ObjectMapper.Builder builder) {
if (fieldName.equals("path") && parserContext.indexVersionCreated().before(Version.V_2_0_0_beta1)) {
builder.pathType(parsePathType(name, fieldNode.toString()));
return true;
}
return false;
}
protected static void parseNested(String name, Map<String, Object> node, ObjectMapper.Builder builder) {
boolean nested = false;
boolean nestedIncludeInParent = false;
@ -326,19 +305,16 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
private volatile Dynamic dynamic;
private final ContentPath.Type pathType;
private Boolean includeInAll;
private volatile CopyOnWriteHashMap<String, Mapper> mappers;
ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers) {
ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map<String, Mapper> mappers) {
super(name);
this.fullPath = fullPath;
this.enabled = enabled;
this.nested = nested;
this.dynamic = dynamic;
this.pathType = pathType;
if (mappers == null) {
this.mappers = new CopyOnWriteHashMap<>();
} else {
@ -380,50 +356,58 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
return this.enabled;
}
public ContentPath.Type pathType() {
return pathType;
}
public Mapper getMapper(String field) {
return mappers.get(field);
}
@Override
public void includeInAll(Boolean includeInAll) {
public ObjectMapper includeInAll(Boolean includeInAll) {
if (includeInAll == null) {
return;
return this;
}
this.includeInAll = includeInAll;
ObjectMapper clone = clone();
clone.includeInAll = includeInAll;
// when called from outside, apply this on all the inner mappers
for (Mapper mapper : mappers.values()) {
for (Mapper mapper : clone.mappers.values()) {
if (mapper instanceof AllFieldMapper.IncludeInAll) {
((AllFieldMapper.IncludeInAll) mapper).includeInAll(includeInAll);
clone.putMapper(((AllFieldMapper.IncludeInAll) mapper).includeInAll(includeInAll));
}
}
return clone;
}
@Override
public void includeInAllIfNotSet(Boolean includeInAll) {
if (this.includeInAll == null) {
this.includeInAll = includeInAll;
public ObjectMapper includeInAllIfNotSet(Boolean includeInAll) {
if (includeInAll == null || this.includeInAll != null) {
return this;
}
ObjectMapper clone = clone();
clone.includeInAll = includeInAll;
// when called from outside, apply this on all the inner mappers
for (Mapper mapper : mappers.values()) {
for (Mapper mapper : clone.mappers.values()) {
if (mapper instanceof AllFieldMapper.IncludeInAll) {
((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll);
clone.putMapper(((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll));
}
}
return clone;
}
@Override
public void unsetIncludeInAll() {
includeInAll = null;
public ObjectMapper unsetIncludeInAll() {
if (includeInAll == null) {
return this;
}
ObjectMapper clone = clone();
clone.includeInAll = null;
// when called from outside, apply this on all the inner mappers
for (Mapper mapper : mappers.values()) {
if (mapper instanceof AllFieldMapper.IncludeInAll) {
((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll();
clone.putMapper(((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll());
}
}
return clone;
}
public Nested nested() {
@ -434,14 +418,9 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
return this.nestedTypeFilter;
}
/**
* Put a new mapper.
* NOTE: this method must be called under the current {@link DocumentMapper}
* lock if concurrent updates are expected.
*/
public void putMapper(Mapper mapper) {
protected void putMapper(Mapper mapper) {
if (mapper instanceof AllFieldMapper.IncludeInAll) {
((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll);
mapper = ((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll);
}
mappers = mappers.copyAndPut(mapper.simpleName(), mapper);
}
@ -464,64 +443,43 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
}
@Override
public void merge(final Mapper mergeWith, final MergeResult mergeResult) {
public ObjectMapper merge(Mapper mergeWith, boolean updateAllTypes) {
if (!(mergeWith instanceof ObjectMapper)) {
mergeResult.addConflict("Can't merge a non object mapping [" + mergeWith.name() + "] with an object mapping [" + name() + "]");
return;
throw new IllegalArgumentException("Can't merge a non object mapping [" + mergeWith.name() + "] with an object mapping [" + name() + "]");
}
ObjectMapper mergeWithObject = (ObjectMapper) mergeWith;
if (nested().isNested()) {
if (!mergeWithObject.nested().isNested()) {
mergeResult.addConflict("object mapping [" + name() + "] can't be changed from nested to non-nested");
return;
}
} else {
if (mergeWithObject.nested().isNested()) {
mergeResult.addConflict("object mapping [" + name() + "] can't be changed from non-nested to nested");
return;
}
}
if (!mergeResult.simulate()) {
if (mergeWithObject.dynamic != null) {
this.dynamic = mergeWithObject.dynamic;
}
}
doMerge(mergeWithObject, mergeResult);
List<Mapper> mappersToPut = new ArrayList<>();
List<ObjectMapper> newObjectMappers = new ArrayList<>();
List<FieldMapper> newFieldMappers = new ArrayList<>();
for (Mapper mapper : mergeWithObject) {
Mapper mergeWithMapper = mapper;
Mapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName());
if (mergeIntoMapper == null) {
// no mapping, simply add it if not simulating
if (!mergeResult.simulate()) {
mappersToPut.add(mergeWithMapper);
MapperUtils.collect(mergeWithMapper, newObjectMappers, newFieldMappers);
}
} else if (mergeIntoMapper instanceof MetadataFieldMapper == false) {
// root mappers can only exist here for backcompat, and are merged in Mapping
mergeIntoMapper.merge(mergeWithMapper, mergeResult);
}
}
if (!newFieldMappers.isEmpty()) {
mergeResult.addFieldMappers(newFieldMappers);
}
if (!newObjectMappers.isEmpty()) {
mergeResult.addObjectMappers(newObjectMappers);
}
// add the mappers only after the administration have been done, so it will not be visible to parser (which first try to read with no lock)
for (Mapper mapper : mappersToPut) {
putMapper(mapper);
}
ObjectMapper merged = clone();
merged.doMerge(mergeWithObject, updateAllTypes);
return merged;
}
protected void doMerge(ObjectMapper mergeWith, MergeResult mergeResult) {
protected void doMerge(final ObjectMapper mergeWith, boolean updateAllTypes) {
if (nested().isNested()) {
if (!mergeWith.nested().isNested()) {
throw new IllegalArgumentException("object mapping [" + name() + "] can't be changed from nested to non-nested");
}
} else {
if (mergeWith.nested().isNested()) {
throw new IllegalArgumentException("object mapping [" + name() + "] can't be changed from non-nested to nested");
}
}
if (mergeWith.dynamic != null) {
this.dynamic = mergeWith.dynamic;
}
for (Mapper mergeWithMapper : mergeWith) {
Mapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName());
Mapper merged;
if (mergeIntoMapper == null) {
// no mapping, simply add it
merged = mergeWithMapper;
} else {
// root mappers can only exist here for backcompat, and are merged in Mapping
merged = mergeIntoMapper.merge(mergeWithMapper, updateAllTypes);
}
putMapper(merged);
}
}
@Override
@ -549,9 +507,6 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
if (enabled != Defaults.ENABLED) {
builder.field("enabled", enabled);
}
if (pathType != Defaults.PATH_TYPE) {
builder.field("path", pathType.name().toLowerCase(Locale.ROOT));
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
}

View file

@ -95,7 +95,7 @@ public class RootObjectMapper extends ObjectMapper {
@Override
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers, @Nullable Settings settings) {
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map<String, Mapper> mappers, @Nullable Settings settings) {
assert !nested.isNested();
FormatDateTimeFormatter[] dates = null;
if (dynamicDateTimeFormatters == null) {
@ -106,7 +106,7 @@ public class RootObjectMapper extends ObjectMapper {
} else {
dates = dynamicDateTimeFormatters.toArray(new FormatDateTimeFormatter[dynamicDateTimeFormatters.size()]);
}
return new RootObjectMapper(name, enabled, dynamic, pathType, mappers,
return new RootObjectMapper(name, enabled, dynamic, mappers,
dates,
dynamicTemplates.toArray(new DynamicTemplate[dynamicTemplates.size()]),
dateDetection, numericDetection);
@ -196,15 +196,23 @@ public class RootObjectMapper extends ObjectMapper {
private volatile DynamicTemplate dynamicTemplates[];
RootObjectMapper(String name, boolean enabled, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers,
RootObjectMapper(String name, boolean enabled, Dynamic dynamic, Map<String, Mapper> mappers,
FormatDateTimeFormatter[] dynamicDateTimeFormatters, DynamicTemplate dynamicTemplates[], boolean dateDetection, boolean numericDetection) {
super(name, name, enabled, Nested.NO, dynamic, pathType, mappers);
super(name, name, enabled, Nested.NO, dynamic, mappers);
this.dynamicTemplates = dynamicTemplates;
this.dynamicDateTimeFormatters = dynamicDateTimeFormatters;
this.dateDetection = dateDetection;
this.numericDetection = numericDetection;
}
/** Return a copy of this mapper that has the given {@code mapper} as a
* sub mapper. */
public RootObjectMapper copyAndPutMapper(Mapper mapper) {
RootObjectMapper clone = (RootObjectMapper) clone();
clone.putMapper(mapper);
return clone;
}
@Override
public ObjectMapper mappingUpdate(Mapper mapper) {
RootObjectMapper update = (RootObjectMapper) super.mappingUpdate(mapper);
@ -253,25 +261,29 @@ public class RootObjectMapper extends ObjectMapper {
}
@Override
protected void doMerge(ObjectMapper mergeWith, MergeResult mergeResult) {
public RootObjectMapper merge(Mapper mergeWith, boolean updateAllTypes) {
return (RootObjectMapper) super.merge(mergeWith, updateAllTypes);
}
@Override
protected void doMerge(ObjectMapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
RootObjectMapper mergeWithObject = (RootObjectMapper) mergeWith;
if (!mergeResult.simulate()) {
// merge them
List<DynamicTemplate> mergedTemplates = new ArrayList<>(Arrays.asList(this.dynamicTemplates));
for (DynamicTemplate template : mergeWithObject.dynamicTemplates) {
boolean replaced = false;
for (int i = 0; i < mergedTemplates.size(); i++) {
if (mergedTemplates.get(i).name().equals(template.name())) {
mergedTemplates.set(i, template);
replaced = true;
}
}
if (!replaced) {
mergedTemplates.add(template);
// merge them
List<DynamicTemplate> mergedTemplates = new ArrayList<>(Arrays.asList(this.dynamicTemplates));
for (DynamicTemplate template : mergeWithObject.dynamicTemplates) {
boolean replaced = false;
for (int i = 0; i < mergedTemplates.size(); i++) {
if (mergedTemplates.get(i).name().equals(template.name())) {
mergedTemplates.set(i, template);
replaced = true;
}
}
this.dynamicTemplates = mergedTemplates.toArray(new DynamicTemplate[mergedTemplates.size()]);
if (!replaced) {
mergedTemplates.add(template);
}
}
this.dynamicTemplates = mergedTemplates.toArray(new DynamicTemplate[mergedTemplates.size()]);
}
@Override

View file

@ -19,13 +19,11 @@
package org.elasticsearch.index.query.functionscore;
import java.util.Map;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.functionscore.exp.ExponentialDecayFunctionParser;
import org.elasticsearch.index.query.functionscore.fieldvaluefactor.FieldValueFactorFunctionParser;
import org.elasticsearch.index.query.functionscore.gauss.GaussDecayFunctionParser;
@ -74,11 +72,12 @@ public class ScoreFunctionParserMapper {
return functionParsers.get(parserName);
}
private static void addParser(ScoreFunctionParser<?> scoreFunctionParser, Map<String, ScoreFunctionParser<?>> map, NamedWriteableRegistry namedWriteableRegistry) {
private static void addParser(ScoreFunctionParser<? extends ScoreFunctionBuilder> scoreFunctionParser, Map<String, ScoreFunctionParser<?>> map, NamedWriteableRegistry namedWriteableRegistry) {
for (String name : scoreFunctionParser.getNames()) {
map.put(name, scoreFunctionParser);
}
namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, scoreFunctionParser.getBuilderPrototype());
@SuppressWarnings("unchecked") NamedWriteable<? extends ScoreFunctionBuilder> sfb = scoreFunctionParser.getBuilderPrototype();
namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, sfb);
}
}

View file

@ -110,7 +110,7 @@ public class TranslogRecoveryPerformer {
if (currentUpdate == null) {
recoveredTypes.put(type, update);
} else {
MapperUtils.merge(currentUpdate, update);
currentUpdate = currentUpdate.merge(update, false);
}
}

View file

@ -48,22 +48,27 @@ public final class BufferingTranslogWriter extends TranslogWriter {
public Translog.Location add(BytesReference data) throws IOException {
try (ReleasableLock lock = writeLock.acquire()) {
ensureOpen();
operationCounter++;
final long offset = totalOffset;
if (data.length() >= buffer.length) {
flush();
// we use the channel to write, since on windows, writing to the RAF might not be reflected
// when reading through the channel
data.writeTo(channel);
try {
data.writeTo(channel);
} catch (Throwable ex) {
closeWithTragicEvent(ex);
throw ex;
}
writtenOffset += data.length();
totalOffset += data.length();
return new Translog.Location(generation, offset, data.length());
} else {
if (data.length() > buffer.length - bufferCount) {
flush();
}
data.writeTo(bufferOs);
totalOffset += data.length();
}
if (data.length() > buffer.length - bufferCount) {
flush();
}
data.writeTo(bufferOs);
totalOffset += data.length();
operationCounter++;
return new Translog.Location(generation, offset, data.length());
}
}
@ -71,10 +76,17 @@ public final class BufferingTranslogWriter extends TranslogWriter {
protected final void flush() throws IOException {
assert writeLock.isHeldByCurrentThread();
if (bufferCount > 0) {
ensureOpen();
// we use the channel to write, since on windows, writing to the RAF might not be reflected
// when reading through the channel
Channels.writeToChannel(buffer, 0, bufferCount, channel);
writtenOffset += bufferCount;
final int bufferSize = bufferCount;
try {
Channels.writeToChannel(buffer, 0, bufferSize, channel);
} catch (Throwable ex) {
closeWithTragicEvent(ex);
throw ex;
}
writtenOffset += bufferSize;
bufferCount = 0;
}
}
@ -102,20 +114,28 @@ public final class BufferingTranslogWriter extends TranslogWriter {
}
@Override
public void sync() throws IOException {
if (!syncNeeded()) {
return;
}
synchronized (this) {
public synchronized void sync() throws IOException {
if (syncNeeded()) {
ensureOpen(); // this call gives a better exception that the incRef if we are closed by a tragic event
channelReference.incRef();
try {
final long offsetToSync;
final int opsCounter;
try (ReleasableLock lock = writeLock.acquire()) {
flush();
lastSyncedOffset = totalOffset;
offsetToSync = totalOffset;
opsCounter = operationCounter;
}
// we can do this outside of the write lock but we have to protect from
// concurrent syncs
checkpoint(lastSyncedOffset, operationCounter, channelReference);
ensureOpen(); // just for kicks - the checkpoint happens or not either way
try {
checkpoint(offsetToSync, opsCounter, channelReference);
} catch (Throwable ex) {
closeWithTragicEvent(ex);
throw ex;
}
lastSyncedOffset = offsetToSync;
} finally {
channelReference.decRef();
}

View file

@ -115,7 +115,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
private final Path location;
private TranslogWriter current;
private volatile ImmutableTranslogReader currentCommittingTranslog;
private long lastCommittedTranslogFileGeneration = -1; // -1 is safe as it will not cause an translog deletion.
private volatile long lastCommittedTranslogFileGeneration = -1; // -1 is safe as it will not cause an translog deletion.
private final AtomicBoolean closed = new AtomicBoolean();
private final TranslogConfig config;
private final String translogUUID;
@ -279,7 +279,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
}
}
boolean isOpen() {
/** Returns {@code true} if this {@code Translog} is still open. */
public boolean isOpen() {
return closed.get() == false;
}
@ -288,10 +289,14 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
if (closed.compareAndSet(false, true)) {
try (ReleasableLock lock = writeLock.acquire()) {
try {
IOUtils.close(current, currentCommittingTranslog);
current.sync();
} finally {
IOUtils.close(recoveredTranslogs);
recoveredTranslogs.clear();
try {
IOUtils.close(current, currentCommittingTranslog);
} finally {
IOUtils.close(recoveredTranslogs);
recoveredTranslogs.clear();
}
}
} finally {
FutureUtils.cancel(syncScheduler);
@ -354,7 +359,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
TranslogWriter createWriter(long fileGeneration) throws IOException {
TranslogWriter newFile;
try {
newFile = TranslogWriter.create(config.getType(), shardId, translogUUID, fileGeneration, location.resolve(getFilename(fileGeneration)), new OnCloseRunnable(), config.getBufferSize());
newFile = TranslogWriter.create(config.getType(), shardId, translogUUID, fileGeneration, location.resolve(getFilename(fileGeneration)), new OnCloseRunnable(), config.getBufferSize(), getChannelFactory());
} catch (IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
}
@ -393,7 +398,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
* @see Index
* @see org.elasticsearch.index.translog.Translog.Delete
*/
public Location add(Operation operation) throws TranslogException {
public Location add(Operation operation) throws IOException {
final ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays);
try {
final BufferedChecksumStreamOutput checksumStreamOutput = new BufferedChecksumStreamOutput(out);
@ -415,7 +420,14 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
assert current.assertBytesAtLocation(location, bytes);
return location;
}
} catch (AlreadyClosedException ex) {
} catch (AlreadyClosedException | IOException ex) {
if (current.getTragicException() != null) {
try {
close();
} catch (Exception inner) {
ex.addSuppressed(inner);
}
}
throw ex;
} catch (Throwable e) {
throw new TranslogException(shardId, "Failed to write operation [" + operation + "]", e);
@ -429,6 +441,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
* Snapshots are fixed in time and will not be updated with future operations.
*/
public Snapshot newSnapshot() {
ensureOpen();
try (ReleasableLock lock = readLock.acquire()) {
ArrayList<TranslogReader> toOpen = new ArrayList<>();
toOpen.addAll(recoveredTranslogs);
@ -493,6 +506,15 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
if (closed.get() == false) {
current.sync();
}
} catch (AlreadyClosedException | IOException ex) {
if (current.getTragicException() != null) {
try {
close();
} catch (Exception inner) {
ex.addSuppressed(inner);
}
}
throw ex;
}
}
@ -520,6 +542,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
public boolean ensureSynced(Location location) throws IOException {
try (ReleasableLock lock = readLock.acquire()) {
if (location.generation == current.generation) { // if we have a new one it's already synced
ensureOpen();
return current.syncUpTo(location.translogLocation + location.size);
}
}
@ -548,31 +571,29 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
private final class OnCloseRunnable implements Callback<ChannelReference> {
@Override
public void handle(ChannelReference channelReference) {
try (ReleasableLock lock = writeLock.acquire()) {
if (isReferencedGeneration(channelReference.getGeneration()) == false) {
Path translogPath = channelReference.getPath();
assert channelReference.getPath().getParent().equals(location) : "translog files must be in the location folder: " + location + " but was: " + translogPath;
// if the given translogPath is not the current we can safely delete the file since all references are released
logger.trace("delete translog file - not referenced and not current anymore {}", translogPath);
IOUtils.deleteFilesIgnoringExceptions(translogPath);
IOUtils.deleteFilesIgnoringExceptions(translogPath.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration())));
if (isReferencedGeneration(channelReference.getGeneration()) == false) {
Path translogPath = channelReference.getPath();
assert channelReference.getPath().getParent().equals(location) : "translog files must be in the location folder: " + location + " but was: " + translogPath;
// if the given translogPath is not the current we can safely delete the file since all references are released
logger.trace("delete translog file - not referenced and not current anymore {}", translogPath);
IOUtils.deleteFilesIgnoringExceptions(translogPath);
IOUtils.deleteFilesIgnoringExceptions(translogPath.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration())));
}
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location)) {
for (Path path : stream) {
Matcher matcher = PARSE_STRICT_ID_PATTERN.matcher(path.getFileName().toString());
if (matcher.matches()) {
long generation = Long.parseLong(matcher.group(1));
if (isReferencedGeneration(generation) == false) {
logger.trace("delete translog file - not referenced and not current anymore {}", path);
IOUtils.deleteFilesIgnoringExceptions(path);
IOUtils.deleteFilesIgnoringExceptions(path.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration())));
}
}
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location)) {
for (Path path : stream) {
Matcher matcher = PARSE_STRICT_ID_PATTERN.matcher(path.getFileName().toString());
if (matcher.matches()) {
long generation = Long.parseLong(matcher.group(1));
if (isReferencedGeneration(generation) == false) {
logger.trace("delete translog file - not referenced and not current anymore {}", path);
IOUtils.deleteFilesIgnoringExceptions(path);
IOUtils.deleteFilesIgnoringExceptions(path.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration())));
}
}
} catch (IOException e) {
logger.warn("failed to delete unreferenced translog files", e);
}
} catch (IOException e) {
logger.warn("failed to delete unreferenced translog files", e);
}
}
}
@ -1294,6 +1315,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
throw new IllegalStateException("already committing a translog with generation: " + currentCommittingTranslog.getGeneration());
}
final TranslogWriter oldCurrent = current;
oldCurrent.ensureOpen();
oldCurrent.sync();
currentCommittingTranslog = current.immutableReader();
Path checkpoint = location.resolve(CHECKPOINT_FILE_NAME);
@ -1389,7 +1411,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
private void ensureOpen() {
if (closed.get()) {
throw new AlreadyClosedException("translog is already closed");
throw new AlreadyClosedException("translog is already closed", current.getTragicException());
}
}
@ -1400,4 +1422,15 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
return outstandingViews.size();
}
TranslogWriter.ChannelFactory getChannelFactory() {
return TranslogWriter.ChannelFactory.DEFAULT;
}
/** If this {@code Translog} was closed as a side-effect of a tragic exception,
* e.g. disk full while flushing a new segment, this returns the root cause exception.
* Otherwise (no tragic exception has occurred) it returns null. */
public Throwable getTragicException() {
return current.getTragicException();
}
}

View file

@ -140,16 +140,16 @@ public abstract class TranslogReader implements Closeable, Comparable<TranslogRe
@Override
public void close() throws IOException {
if (closed.compareAndSet(false, true)) {
doClose();
channelReference.decRef();
}
}
protected void doClose() throws IOException {
channelReference.decRef();
protected final boolean isClosed() {
return closed.get();
}
protected void ensureOpen() {
if (closed.get()) {
if (isClosed()) {
throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed");
}
}

View file

@ -20,6 +20,7 @@
package org.elasticsearch.index.translog;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -54,6 +55,9 @@ public class TranslogWriter extends TranslogReader {
protected volatile int operationCounter;
/* the offset in bytes written to the file */
protected volatile long writtenOffset;
/* if we hit an exception that we can't recover from we assign it to this var and ship it with every AlreadyClosedException we throw */
private volatile Throwable tragedy;
public TranslogWriter(ShardId shardId, long generation, ChannelReference channelReference) throws IOException {
super(generation, channelReference, channelReference.getChannel().position());
@ -65,10 +69,10 @@ public class TranslogWriter extends TranslogReader {
this.lastSyncedOffset = channelReference.getChannel().position();;
}
public static TranslogWriter create(Type type, ShardId shardId, String translogUUID, long fileGeneration, Path file, Callback<ChannelReference> onClose, int bufferSize) throws IOException {
public static TranslogWriter create(Type type, ShardId shardId, String translogUUID, long fileGeneration, Path file, Callback<ChannelReference> onClose, int bufferSize, ChannelFactory channelFactory) throws IOException {
final BytesRef ref = new BytesRef(translogUUID);
final int headerLength = CodecUtil.headerLength(TRANSLOG_CODEC) + ref.length + RamUsageEstimator.NUM_BYTES_INT;
final FileChannel channel = FileChannel.open(file, StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.CREATE_NEW);
final FileChannel channel = channelFactory.open(file);
try {
// This OutputStreamDataOutput is intentionally not closed because
// closing it will close the FileChannel
@ -90,6 +94,12 @@ public class TranslogWriter extends TranslogReader {
throw throwable;
}
}
/** If this {@code TranslogWriter} was closed as a side-effect of a tragic exception,
* e.g. disk full while flushing a new segment, this returns the root cause exception.
* Otherwise (no tragic exception has occurred) it returns null. */
public Throwable getTragicException() {
return tragedy;
}
public enum Type {
@ -118,6 +128,16 @@ public class TranslogWriter extends TranslogReader {
}
}
protected final void closeWithTragicEvent(Throwable throwable) throws IOException {
try (ReleasableLock lock = writeLock.acquire()) {
if (tragedy == null) {
tragedy = throwable;
} else {
tragedy.addSuppressed(throwable);
}
close();
}
}
/**
* add the given bytes to the translog and return the location they were written at
@ -127,9 +147,14 @@ public class TranslogWriter extends TranslogReader {
try (ReleasableLock lock = writeLock.acquire()) {
ensureOpen();
position = writtenOffset;
data.writeTo(channel);
try {
data.writeTo(channel);
} catch (Throwable e) {
closeWithTragicEvent(e);
throw e;
}
writtenOffset = writtenOffset + data.length();
operationCounter = operationCounter + 1;
operationCounter++;;
}
return new Translog.Location(generation, position, data.length());
}
@ -143,12 +168,13 @@ public class TranslogWriter extends TranslogReader {
/**
* write all buffered ops to disk and fsync file
*/
public void sync() throws IOException {
public synchronized void sync() throws IOException { // synchronized to ensure only one sync happens a time
// check if we really need to sync here...
if (syncNeeded()) {
try (ReleasableLock lock = writeLock.acquire()) {
ensureOpen();
checkpoint(writtenOffset, operationCounter, channelReference);
lastSyncedOffset = writtenOffset;
checkpoint(lastSyncedOffset, operationCounter, channelReference);
}
}
}
@ -262,15 +288,6 @@ public class TranslogWriter extends TranslogReader {
return false;
}
@Override
protected final void doClose() throws IOException {
try (ReleasableLock lock = writeLock.acquire()) {
sync();
} finally {
super.doClose();
}
}
@Override
protected void readBytes(ByteBuffer buffer, long position) throws IOException {
try (ReleasableLock lock = readLock.acquire()) {
@ -288,4 +305,20 @@ public class TranslogWriter extends TranslogReader {
Checkpoint checkpoint = new Checkpoint(syncPosition, numOperations, generation);
Checkpoint.write(checkpointFile, checkpoint, options);
}
static class ChannelFactory {
static final ChannelFactory DEFAULT = new ChannelFactory();
// only for testing until we have a disk-full FileSystemt
public FileChannel open(Path file) throws IOException {
return FileChannel.open(file, StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.CREATE_NEW);
}
}
protected final void ensureOpen() {
if (isClosed()) {
throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed", tragedy);
}
}
}

View file

@ -21,6 +21,7 @@ package org.elasticsearch.indices.query;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.query.EmptyQueryBuilder;
@ -40,11 +41,12 @@ public class IndicesQueriesRegistry extends AbstractComponent {
public IndicesQueriesRegistry(Settings settings, Set<QueryParser> injectedQueryParsers, NamedWriteableRegistry namedWriteableRegistry) {
super(settings);
Map<String, QueryParser<?>> queryParsers = new HashMap<>();
for (QueryParser<?> queryParser : injectedQueryParsers) {
for (@SuppressWarnings("unchecked") QueryParser<? extends QueryBuilder> queryParser : injectedQueryParsers) {
for (String name : queryParser.names()) {
queryParsers.put(name, queryParser);
}
namedWriteableRegistry.registerPrototype(QueryBuilder.class, queryParser.getBuilderPrototype());
@SuppressWarnings("unchecked") NamedWriteable<? extends QueryBuilder> qb = queryParser.getBuilderPrototype();
namedWriteableRegistry.registerPrototype(QueryBuilder.class, qb);
}
// EmptyQueryBuilder is not registered as query parser but used internally.
// We need to register it with the NamedWriteableRegistry in order to serialize it

View file

@ -90,6 +90,7 @@ public class PluginManager {
"mapper-murmur3",
"mapper-size",
"repository-azure",
"repository-hdfs",
"repository-s3",
"store-smb"));

View file

@ -84,6 +84,7 @@ import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsContext;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsContext.FieldDataField;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext.ScriptField;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.*;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import org.elasticsearch.search.profile.Profilers;
@ -91,6 +92,7 @@ import org.elasticsearch.search.query.*;
import org.elasticsearch.search.warmer.IndexWarmersMetaData;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
@ -658,7 +660,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
}
}
private void parseSource(DefaultSearchContext context, SearchSourceBuilder source) throws SearchParseException {
private void parseSource(DefaultSearchContext context, SearchSourceBuilder source) throws SearchContextException {
// nothing to parse...
if (source == null) {
return;
@ -812,19 +814,11 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
fieldDataFieldsContext.setHitExecutionNeeded(true);
}
if (source.highlighter() != null) {
XContentParser highlighterParser = null;
HighlightBuilder highlightBuilder = source.highlighter();
try {
highlighterParser = XContentFactory.xContent(source.highlighter()).createParser(source.highlighter());
this.elementParsers.get("highlight").parse(highlighterParser, context);
} catch (Exception e) {
String sSource = "_na_";
try {
sSource = source.toString();
} catch (Throwable e1) {
// ignore
}
XContentLocation location = highlighterParser != null ? highlighterParser.getTokenLocation() : null;
throw new SearchParseException(context, "failed to parse suggest source [" + sSource + "]", location, e);
context.highlight(highlightBuilder.build(context.indexShard().getQueryShardContext()));
} catch (IOException e) {
throw new SearchContextException(context, "failed to create SearchContextHighlighter", e);
}
}
if (source.innerHits() != null) {

View file

@ -91,7 +91,7 @@ public class LongTerms extends InternalTerms<LongTerms, LongTerms.Bucket> {
@Override
public String getKeyAsString() {
return String.valueOf(term);
return formatter.format(term);
}
@Override

View file

@ -19,7 +19,6 @@
package org.elasticsearch.search.aggregations.metrics.tophits;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
@ -194,7 +193,7 @@ public class TopHitsBuilder extends AbstractAggregationBuilder {
return sourceBuilder;
}
public BytesReference highlighter() {
public HighlightBuilder highlighter() {
return sourceBuilder().highlighter();
}

View file

@ -146,7 +146,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
private List<BytesReference> aggregations;
private BytesReference highlightBuilder;
private HighlightBuilder highlightBuilder;
private BytesReference suggestBuilder;
@ -410,22 +410,14 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
* Adds highlight to perform as part of the search.
*/
public SearchSourceBuilder highlighter(HighlightBuilder highlightBuilder) {
try {
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
highlightBuilder.innerXContent(builder);
builder.endObject();
this.highlightBuilder = builder.bytes();
return this;
} catch (IOException e) {
throw new RuntimeException(e);
}
this.highlightBuilder = highlightBuilder;
return this;
}
/**
* Gets the bytes representing the hightlighter builder for this request.
* Gets the hightlighter builder for this request.
*/
public BytesReference highlighter() {
public HighlightBuilder highlighter() {
return highlightBuilder;
}
@ -836,8 +828,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
}
builder.aggregations = aggregations;
} else if (context.parseFieldMatcher().match(currentFieldName, HIGHLIGHT_FIELD)) {
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
builder.highlightBuilder = xContentBuilder.bytes();
builder.highlightBuilder = HighlightBuilder.PROTOTYPE.fromXContent(context);
} else if (context.parseFieldMatcher().match(currentFieldName, INNER_HITS_FIELD)) {
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
builder.innerHitsBuilder = xContentBuilder.bytes();
@ -1039,10 +1030,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
}
if (highlightBuilder != null) {
builder.field(HIGHLIGHT_FIELD.getPreferredName());
XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(highlightBuilder);
parser.nextToken();
builder.copyCurrentStructure(parser);
this.highlightBuilder.toXContent(builder, params);
}
if (innerHitsBuilder != null) {
@ -1185,7 +1173,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
}
builder.from = in.readVInt();
if (in.readBoolean()) {
builder.highlightBuilder = in.readBytesReference();
builder.highlightBuilder = HighlightBuilder.PROTOTYPE.readFrom(in);
}
boolean hasIndexBoost = in.readBoolean();
if (hasIndexBoost) {
@ -1291,7 +1279,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
boolean hasHighlightBuilder = highlightBuilder != null;
out.writeBoolean(hasHighlightBuilder);
if (hasHighlightBuilder) {
out.writeBytesReference(highlightBuilder);
highlightBuilder.writeTo(out);
}
boolean hasIndexBoost = indexBoost != null;
out.writeBoolean(hasIndexBoost);

View file

@ -20,7 +20,6 @@
package org.elasticsearch.search.fetch.innerhits;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryBuilder;
@ -266,7 +265,7 @@ public class InnerHitsBuilder implements ToXContent {
return this;
}
public BytesReference highlighter() {
public HighlightBuilder highlighter() {
return sourceBuilder().highlighter();
}

View file

@ -29,6 +29,8 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.highlight.HighlightBuilder.Order;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -78,7 +80,7 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
protected QueryBuilder<?> highlightQuery;
protected String order;
protected Order order;
protected Boolean highlightFilter;
@ -217,18 +219,26 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
/**
* The order of fragments per field. By default, ordered by the order in the
* highlighted text. Can be <tt>score</tt>, which then it will be ordered
* by score of the fragments.
* by score of the fragments, or <tt>none</TT>.
*/
public HB order(String order) {
return order(Order.fromString(order));
}
/**
* By default, fragments of a field are ordered by the order in the highlighted text.
* If set to {@link Order#SCORE}, this changes order to score of the fragments.
*/
@SuppressWarnings("unchecked")
public HB order(String order) {
this.order = order;
public HB order(Order scoreOrdered) {
this.order = scoreOrdered;
return (HB) this;
}
/**
* @return the value set by {@link #order(String)}
* @return the value set by {@link #order(Order)}
*/
public String order() {
public Order order() {
return this.order;
}
@ -395,7 +405,7 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
builder.field(HIGHLIGHT_QUERY_FIELD.getPreferredName(), highlightQuery);
}
if (order != null) {
builder.field(ORDER_FIELD.getPreferredName(), order);
builder.field(ORDER_FIELD.getPreferredName(), order.toString());
}
if (highlightFilter != null) {
builder.field(HIGHLIGHT_FILTER_FIELD.getPreferredName(), highlightFilter);
@ -458,7 +468,7 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
}
} else if (token.isValue()) {
if (parseContext.parseFieldMatcher().match(currentFieldName, ORDER_FIELD)) {
highlightBuilder.order(parser.text());
highlightBuilder.order(Order.fromString(parser.text()));
} else if (parseContext.parseFieldMatcher().match(currentFieldName, HIGHLIGHT_FILTER_FIELD)) {
highlightBuilder.highlightFilter(parser.booleanValue());
} else if (parseContext.parseFieldMatcher().match(currentFieldName, FRAGMENT_SIZE_FIELD)) {
@ -578,7 +588,9 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
if (in.readBoolean()) {
highlightQuery(in.readQuery());
}
order(in.readOptionalString());
if (in.readBoolean()) {
order(Order.PROTOTYPE.readFrom(in));
}
highlightFilter(in.readOptionalBoolean());
forceSource(in.readOptionalBoolean());
boundaryMaxScan(in.readOptionalVInt());
@ -609,7 +621,11 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
if (hasQuery) {
out.writeQuery(highlightQuery);
}
out.writeOptionalString(order);
boolean hasSetOrder = order != null;
out.writeBoolean(hasSetOrder);
if (hasSetOrder) {
order.writeTo(out);
}
out.writeOptionalBoolean(highlightFilter);
out.writeOptionalBoolean(forceSource);
out.writeOptionalVInt(boundaryMaxScan);

View file

@ -44,6 +44,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Objects;
import java.util.Set;
@ -308,9 +309,15 @@ public class HighlightBuilder extends AbstractHighlighterBuilder<HighlightBuilde
*/
@SuppressWarnings({ "rawtypes", "unchecked" })
private static void transferOptions(AbstractHighlighterBuilder highlighterBuilder, SearchContextHighlight.FieldOptions.Builder targetOptionsBuilder, QueryShardContext context) throws IOException {
targetOptionsBuilder.preTags(highlighterBuilder.preTags);
targetOptionsBuilder.postTags(highlighterBuilder.postTags);
targetOptionsBuilder.scoreOrdered("score".equals(highlighterBuilder.order));
if (highlighterBuilder.preTags != null) {
targetOptionsBuilder.preTags(highlighterBuilder.preTags);
}
if (highlighterBuilder.postTags != null) {
targetOptionsBuilder.postTags(highlighterBuilder.postTags);
}
if (highlighterBuilder.order != null) {
targetOptionsBuilder.scoreOrdered(highlighterBuilder.order == Order.SCORE);
}
if (highlighterBuilder.highlightFilter != null) {
targetOptionsBuilder.highlightFilter(highlighterBuilder.highlightFilter);
}
@ -326,9 +333,15 @@ public class HighlightBuilder extends AbstractHighlighterBuilder<HighlightBuilde
if (highlighterBuilder.boundaryMaxScan != null) {
targetOptionsBuilder.boundaryMaxScan(highlighterBuilder.boundaryMaxScan);
}
targetOptionsBuilder.boundaryChars(convertCharArray(highlighterBuilder.boundaryChars));
targetOptionsBuilder.highlighterType(highlighterBuilder.highlighterType);
targetOptionsBuilder.fragmenter(highlighterBuilder.fragmenter);
if (highlighterBuilder.boundaryChars != null) {
targetOptionsBuilder.boundaryChars(convertCharArray(highlighterBuilder.boundaryChars));
}
if (highlighterBuilder.highlighterType != null) {
targetOptionsBuilder.highlighterType(highlighterBuilder.highlighterType);
}
if (highlighterBuilder.fragmenter != null) {
targetOptionsBuilder.fragmenter(highlighterBuilder.fragmenter);
}
if (highlighterBuilder.noMatchSize != null) {
targetOptionsBuilder.noMatchSize(highlighterBuilder.noMatchSize);
}
@ -338,7 +351,9 @@ public class HighlightBuilder extends AbstractHighlighterBuilder<HighlightBuilde
if (highlighterBuilder.phraseLimit != null) {
targetOptionsBuilder.phraseLimit(highlighterBuilder.phraseLimit);
}
targetOptionsBuilder.options(highlighterBuilder.options);
if (highlighterBuilder.options != null) {
targetOptionsBuilder.options(highlighterBuilder.options);
}
if (highlighterBuilder.highlightQuery != null) {
targetOptionsBuilder.highlightQuery(highlighterBuilder.highlightQuery.toQuery(context));
}
@ -545,4 +560,36 @@ public class HighlightBuilder extends AbstractHighlighterBuilder<HighlightBuilde
writeOptionsTo(out);
}
}
public enum Order implements Writeable<Order> {
NONE, SCORE;
static Order PROTOTYPE = NONE;
@Override
public Order readFrom(StreamInput in) throws IOException {
int ordinal = in.readVInt();
if (ordinal < 0 || ordinal >= values().length) {
throw new IOException("Unknown Order ordinal [" + ordinal + "]");
}
return values()[ordinal];
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(this.ordinal());
}
public static Order fromString(String order) {
if (order.toUpperCase(Locale.ROOT).equals(SCORE.name())) {
return Order.SCORE;
}
return NONE;
}
@Override
public String toString() {
return name().toLowerCase(Locale.ROOT);
}
}
}

View file

@ -0,0 +1,76 @@
/*
* 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.transport;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
/**
* Base class for delegating transport response to a transport channel
*/
public abstract class TransportChannelResponseHandler<T extends TransportResponse> implements TransportResponseHandler<T> {
/**
* Convenience method for delegating an empty response to the provided changed
*/
public static TransportChannelResponseHandler<TransportResponse.Empty> emptyResponseHandler(ESLogger logger, TransportChannel channel, String extraInfoOnError) {
return new TransportChannelResponseHandler<TransportResponse.Empty>(logger, channel, extraInfoOnError) {
@Override
public TransportResponse.Empty newInstance() {
return TransportResponse.Empty.INSTANCE;
}
};
}
private final ESLogger logger;
private final TransportChannel channel;
private final String extraInfoOnError;
protected TransportChannelResponseHandler(ESLogger logger, TransportChannel channel, String extraInfoOnError) {
this.logger = logger;
this.channel = channel;
this.extraInfoOnError = extraInfoOnError;
}
@Override
public void handleResponse(T response) {
try {
channel.sendResponse(response);
} catch (IOException e) {
handleException(new TransportException(e));
}
}
@Override
public void handleException(TransportException exp) {
try {
channel.sendResponse(exp);
} catch (IOException e) {
logger.debug("failed to send failure {}", e, extraInfoOnError == null ? "" : "(" + extraInfoOnError + ")");
}
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
}

View file

@ -50,6 +50,7 @@ OFFICIAL PLUGINS
- mapper-murmur3
- mapper-size
- repository-azure
- repository-hdfs
- repository-s3
- store-smb

View file

@ -31,7 +31,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
@ -39,7 +38,6 @@ import org.elasticsearch.index.mapper.core.LongFieldMapper;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@ -321,9 +319,7 @@ public class CopyToMapperTests extends ESSingleNodeTestCase {
DocumentMapper docMapperAfter = parser.parse(mappingAfter);
MergeResult mergeResult = docMapperBefore.merge(docMapperAfter.mapping(), true, false);
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(false));
docMapperBefore.merge(docMapperAfter.mapping(), true, false);
docMapperBefore.merge(docMapperAfter.mapping(), false, false);

View file

@ -23,7 +23,6 @@ import org.apache.lucene.analysis.*;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.io.IOException;
@ -60,13 +59,11 @@ public class TokenCountFieldMapperTests extends ESSingleNodeTestCase {
.endObject().endObject().string();
DocumentMapper stage2 = parser.parse(stage2Mapping);
MergeResult mergeResult = stage1.merge(stage2.mapping(), true, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
stage1.merge(stage2.mapping(), true, false);
// Just simulated so merge hasn't happened yet
assertThat(((TokenCountFieldMapper) stage1.mappers().smartNameFieldMapper("tc")).analyzer(), equalTo("keyword"));
mergeResult = stage1.merge(stage2.mapping(), false, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
stage1.merge(stage2.mapping(), false, false);
// Just simulated so merge hasn't happened yet
assertThat(((TokenCountFieldMapper) stage1.mappers().smartNameFieldMapper("tc")).analyzer(), equalTo("standard"));
}

View file

@ -371,9 +371,8 @@ public class SimpleDateMappingTests extends ESSingleNodeTestCase {
Map<String, String> config = getConfigurationViaXContent(initialDateFieldMapper);
assertThat(config.get("format"), is("EEE MMM dd HH:mm:ss.S Z yyyy||EEE MMM dd HH:mm:ss.SSS Z yyyy"));
MergeResult mergeResult = defaultMapper.merge(mergeMapper.mapping(), false, false);
defaultMapper.merge(mergeMapper.mapping(), false, false);
assertThat("Merging resulting in conflicts: " + Arrays.asList(mergeResult.buildConflicts()), mergeResult.hasConflicts(), is(false));
assertThat(defaultMapper.mappers().getMapper("field"), is(instanceOf(DateFieldMapper.class)));
DateFieldMapper mergedFieldMapper = (DateFieldMapper) defaultMapper.mappers().getMapper("field");

View file

@ -34,7 +34,6 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.BinaryFieldMapper;
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
@ -96,9 +95,6 @@ public class ExternalMapper extends FieldMapper {
@Override
public ExternalMapper build(BuilderContext context) {
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(ContentPath.Type.FULL);
context.path().add(name);
BinaryFieldMapper binMapper = binBuilder.build(context);
BooleanFieldMapper boolMapper = boolBuilder.build(context);
@ -108,7 +104,6 @@ public class ExternalMapper extends FieldMapper {
FieldMapper stringMapper = (FieldMapper)stringBuilder.build(context);
context.path().remove();
context.path().pathType(origPathType);
setupFieldType(context);
return new ExternalMapper(name, fieldType, generatedValue, mapperName, binMapper, boolMapper, pointMapper, shapeMapper, stringMapper,
@ -219,7 +214,7 @@ public class ExternalMapper extends FieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// ignore this for now
}

View file

@ -28,7 +28,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
@ -66,9 +65,9 @@ public class ExternalMetadataMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
public void doMerge(Mapper mergeWith, boolean updateAllTypes) {
if (!(mergeWith instanceof ExternalMetadataMapper)) {
mergeResult.addConflict("Trying to merge " + mergeWith + " with " + this);
throw new IllegalArgumentException("Trying to merge " + mergeWith + " with " + this);
}
}

View file

@ -87,7 +87,7 @@ public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase {
.startObject("f")
.field("type", ExternalMapperPlugin.EXTERNAL_UPPER)
.startObject("fields")
.startObject("f")
.startObject("g")
.field("type", "string")
.field("store", "yes")
.startObject("fields")
@ -107,7 +107,7 @@ public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase {
refresh();
SearchResponse response = client().prepareSearch("test-idx")
.setQuery(QueryBuilders.termQuery("f.f.raw", "FOO BAR"))
.setQuery(QueryBuilders.termQuery("f.g.raw", "FOO BAR"))
.execute().actionGet();
assertThat(response.getHits().totalHits(), equalTo((long) 1));

View file

@ -33,7 +33,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.test.ESSingleNodeTestCase;

View file

@ -30,17 +30,13 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.isIn;
public class GeoShapeFieldMapperTests extends ESSingleNodeTestCase {
public void testDefaultConfiguration() throws IOException {

View file

@ -29,7 +29,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.mapper.object.ObjectMapper;
@ -39,6 +38,7 @@ import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
@ -59,15 +59,12 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase {
.endObject().endObject().endObject().string();
DocumentMapper stage2 = parser.parse(stage2Mapping);
MergeResult mergeResult = stage1.merge(stage2.mapping(), true, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
stage1.merge(stage2.mapping(), true, false);
// since we are simulating, we should not have the age mapping
assertThat(stage1.mappers().smartNameFieldMapper("age"), nullValue());
assertThat(stage1.mappers().smartNameFieldMapper("obj1.prop1"), nullValue());
// now merge, don't simulate
mergeResult = stage1.merge(stage2.mapping(), false, false);
// there is still merge failures
assertThat(mergeResult.hasConflicts(), equalTo(false));
stage1.merge(stage2.mapping(), false, false);
// but we have the age in
assertThat(stage1.mappers().smartNameFieldMapper("age"), notNullValue());
assertThat(stage1.mappers().smartNameFieldMapper("obj1.prop1"), notNullValue());
@ -83,8 +80,7 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase {
DocumentMapper withDynamicMapper = parser.parse(withDynamicMapping);
assertThat(withDynamicMapper.root().dynamic(), equalTo(ObjectMapper.Dynamic.FALSE));
MergeResult mergeResult = mapper.merge(withDynamicMapper.mapping(), false, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
mapper.merge(withDynamicMapper.mapping(), false, false);
assertThat(mapper.root().dynamic(), equalTo(ObjectMapper.Dynamic.FALSE));
}
@ -99,14 +95,19 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase {
.endObject().endObject().endObject().string();
DocumentMapper nestedMapper = parser.parse(nestedMapping);
MergeResult mergeResult = objectMapper.merge(nestedMapper.mapping(), true, false);
assertThat(mergeResult.hasConflicts(), equalTo(true));
assertThat(mergeResult.buildConflicts().length, equalTo(1));
assertThat(mergeResult.buildConflicts()[0], equalTo("object mapping [obj] can't be changed from non-nested to nested"));
try {
objectMapper.merge(nestedMapper.mapping(), true, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("object mapping [obj] can't be changed from non-nested to nested"));
}
mergeResult = nestedMapper.merge(objectMapper.mapping(), true, false);
assertThat(mergeResult.buildConflicts().length, equalTo(1));
assertThat(mergeResult.buildConflicts()[0], equalTo("object mapping [obj] can't be changed from nested to non-nested"));
try {
nestedMapper.merge(objectMapper.mapping(), true, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("object mapping [obj] can't be changed from nested to non-nested"));
}
}
public void testMergeSearchAnalyzer() throws Exception {
@ -122,9 +123,8 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase {
DocumentMapper changed = parser.parse(mapping2);
assertThat(((NamedAnalyzer) existing.mappers().getMapper("field").fieldType().searchAnalyzer()).name(), equalTo("whitespace"));
MergeResult mergeResult = existing.merge(changed.mapping(), false, false);
existing.merge(changed.mapping(), false, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
assertThat(((NamedAnalyzer) existing.mappers().getMapper("field").fieldType().searchAnalyzer()).name(), equalTo("keyword"));
}
@ -141,9 +141,8 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase {
DocumentMapper changed = parser.parse(mapping2);
assertThat(((NamedAnalyzer) existing.mappers().getMapper("field").fieldType().searchAnalyzer()).name(), equalTo("whitespace"));
MergeResult mergeResult = existing.merge(changed.mapping(), false, false);
existing.merge(changed.mapping(), false, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
assertThat(((NamedAnalyzer) existing.mappers().getMapper("field").fieldType().searchAnalyzer()).name(), equalTo("standard"));
assertThat(((StringFieldMapper) (existing.mappers().getMapper("field"))).getIgnoreAbove(), equalTo(14));
}

View file

@ -27,15 +27,11 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.util.Arrays;
import static org.elasticsearch.test.StreamsUtils.copyToStringFromClasspath;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
@ -62,8 +58,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase {
mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping2.json");
DocumentMapper docMapper2 = parser.parse(mapping);
MergeResult mergeResult = docMapper.merge(docMapper2.mapping(), true, false);
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(false));
docMapper.merge(docMapper2.mapping(), true, false);
docMapper.merge(docMapper2.mapping(), false, false);
@ -84,8 +79,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase {
mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping3.json");
DocumentMapper docMapper3 = parser.parse(mapping);
mergeResult = docMapper.merge(docMapper3.mapping(), true, false);
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(false));
docMapper.merge(docMapper3.mapping(), true, false);
docMapper.merge(docMapper3.mapping(), false, false);
@ -100,8 +94,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase {
mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping4.json");
DocumentMapper docMapper4 = parser.parse(mapping);
mergeResult = docMapper.merge(docMapper4.mapping(), true, false);
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(false));
docMapper.merge(docMapper4.mapping(), true, false);
docMapper.merge(docMapper4.mapping(), false, false);

View file

@ -34,11 +34,9 @@ import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.VersionUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
public class DefaultSourceMappingTests extends ESSingleNodeTestCase {
@ -194,13 +192,18 @@ public class DefaultSourceMappingTests extends ESSingleNodeTestCase {
void assertConflicts(String mapping1, String mapping2, DocumentMapperParser parser, String... conflicts) throws IOException {
DocumentMapper docMapper = parser.parse(mapping1);
docMapper = parser.parse(docMapper.mappingSource().string());
MergeResult mergeResult = docMapper.merge(parser.parse(mapping2).mapping(), true, false);
List<String> expectedConflicts = new ArrayList<>(Arrays.asList(conflicts));
for (String conflict : mergeResult.buildConflicts()) {
assertTrue("found unexpected conflict [" + conflict + "]", expectedConflicts.remove(conflict));
if (conflicts.length == 0) {
docMapper.merge(parser.parse(mapping2).mapping(), true, false);
} else {
try {
docMapper.merge(parser.parse(mapping2).mapping(), true, false);
fail();
} catch (IllegalArgumentException e) {
for (String conflict : conflicts) {
assertThat(e.getMessage(), containsString(conflict));
}
}
}
assertTrue("missing conflicts: " + Arrays.toString(expectedConflicts.toArray()), expectedConflicts.isEmpty());
}
public void testEnabledNotUpdateable() throws Exception {

View file

@ -40,7 +40,6 @@ import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
@ -493,8 +492,7 @@ public class SimpleStringMappingTests extends ESSingleNodeTestCase {
String updatedMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string").startObject("norms").field("enabled", false).endObject()
.endObject().endObject().endObject().endObject().string();
MergeResult mergeResult = defaultMapper.merge(parser.parse(updatedMapping).mapping(), false, false);
assertFalse(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts());
defaultMapper.merge(parser.parse(updatedMapping).mapping(), false, false);
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()

View file

@ -42,7 +42,6 @@ import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
@ -515,8 +514,7 @@ public class TimestampMappingTests extends ESSingleNodeTestCase {
.startObject("_timestamp").field("enabled", randomBoolean()).startObject("fielddata").field("loading", "eager").field("format", "array").endObject().field("store", "yes").endObject()
.endObject().endObject().string();
MergeResult mergeResult = docMapper.merge(parser.parse(mapping).mapping(), false, false);
assertThat(mergeResult.buildConflicts().length, equalTo(0));
docMapper.merge(parser.parse(mapping).mapping(), false, false);
assertThat(docMapper.timestampFieldMapper().fieldType().fieldDataType().getLoading(), equalTo(MappedFieldType.Loading.EAGER));
assertThat(docMapper.timestampFieldMapper().fieldType().fieldDataType().getFormat(indexSettings), equalTo("array"));
}
@ -618,9 +616,9 @@ public class TimestampMappingTests extends ESSingleNodeTestCase {
.field("index", indexValues.remove(randomInt(2)))
.endObject()
.endObject().endObject().string();
DocumentMapperParser parser = createIndex("test", BWC_SETTINGS).mapperService().documentMapperParser();
MapperService mapperService = createIndex("test", BWC_SETTINGS).mapperService();
DocumentMapper docMapper = parser.parse(mapping);
mapperService.merge("type", new CompressedXContent(mapping), true, false);
mapping = XContentFactory.jsonBuilder().startObject()
.startObject("type")
.startObject("_timestamp")
@ -628,18 +626,11 @@ public class TimestampMappingTests extends ESSingleNodeTestCase {
.endObject()
.endObject().endObject().string();
MergeResult mergeResult = docMapper.merge(parser.parse(mapping).mapping(), true, false);
List<String> expectedConflicts = new ArrayList<>();
expectedConflicts.add("mapper [_timestamp] has different [index] values");
expectedConflicts.add("mapper [_timestamp] has different [tokenize] values");
if (indexValues.get(0).equals("not_analyzed") == false) {
// if the only index value left is not_analyzed, then the doc values setting will be the same, but in the
// other two cases, it will change
expectedConflicts.add("mapper [_timestamp] has different [doc_values] values");
}
for (String conflict : mergeResult.buildConflicts()) {
assertThat(conflict, isIn(expectedConflicts));
try {
mapperService.merge("type", new CompressedXContent(mapping), false, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [_timestamp] has different [index] values"));
}
}
@ -686,10 +677,15 @@ public class TimestampMappingTests extends ESSingleNodeTestCase {
void assertConflict(String mapping1, String mapping2, DocumentMapperParser parser, String conflict) throws IOException {
DocumentMapper docMapper = parser.parse(mapping1);
docMapper = parser.parse(docMapper.mappingSource().string());
MergeResult mergeResult = docMapper.merge(parser.parse(mapping2).mapping(), true, false);
assertThat(mergeResult.buildConflicts().length, equalTo(conflict == null ? 0 : 1));
if (conflict != null) {
assertThat(mergeResult.buildConflicts()[0], containsString(conflict));
if (conflict == null) {
docMapper.merge(parser.parse(mapping2).mapping(), true, false);
} else {
try {
docMapper.merge(parser.parse(mapping2).mapping(), true, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString(conflict));
}
}
}

View file

@ -35,7 +35,6 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
@ -116,9 +115,8 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
DocumentMapper mapperWithoutTtl = parser.parse(mappingWithoutTtl);
DocumentMapper mapperWithTtl = parser.parse(mappingWithTtl);
MergeResult mergeResult = mapperWithoutTtl.merge(mapperWithTtl.mapping(), false, false);
mapperWithoutTtl.merge(mapperWithTtl.mapping(), false, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
assertThat(mapperWithoutTtl.TTLFieldMapper().enabled(), equalTo(true));
}
@ -141,9 +139,8 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
DocumentMapper initialMapper = parser.parse(mappingWithTtl);
DocumentMapper updatedMapper = parser.parse(updatedMapping);
MergeResult mergeResult = initialMapper.merge(updatedMapper.mapping(), true, false);
initialMapper.merge(updatedMapper.mapping(), true, false);
assertThat(mergeResult.hasConflicts(), equalTo(false));
assertThat(initialMapper.TTLFieldMapper().enabled(), equalTo(true));
}
@ -154,9 +151,13 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
DocumentMapper initialMapper = parser.parse(mappingWithTtl);
DocumentMapper updatedMapper = parser.parse(mappingWithTtlDisabled);
MergeResult mergeResult = initialMapper.merge(updatedMapper.mapping(), true, false);
try {
initialMapper.merge(updatedMapper.mapping(), true, false);
fail();
} catch (IllegalArgumentException e) {
// expected
}
assertThat(mergeResult.hasConflicts(), equalTo(true));
assertThat(initialMapper.TTLFieldMapper().enabled(), equalTo(true));
}
@ -189,23 +190,20 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
public void testNoConflictIfNothingSetAndDisabledLater() throws Exception {
IndexService indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type");
XContentBuilder mappingWithTtlDisabled = getMappingWithTtlDisabled("7d");
MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlDisabled.string()), true).mapping(), randomBoolean(), false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlDisabled.string()), true).mapping(), randomBoolean(), false);
}
public void testNoConflictIfNothingSetAndEnabledLater() throws Exception {
IndexService indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type");
XContentBuilder mappingWithTtlEnabled = getMappingWithTtlEnabled("7d");
MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), randomBoolean(), false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), randomBoolean(), false);
}
public void testMergeWithOnlyDefaultSet() throws Exception {
XContentBuilder mappingWithTtlEnabled = getMappingWithTtlEnabled("7d");
IndexService indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type", mappingWithTtlEnabled);
XContentBuilder mappingWithOnlyDefaultSet = getMappingWithOnlyTtlDefaultSet("6m");
MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), true).mapping(), false, false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), true).mapping(), false, false);
CompressedXContent mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterMerge, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":true,\"default\":360000},\"properties\":{\"field\":{\"type\":\"string\"}}}}")));
}
@ -216,8 +214,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
CompressedXContent mappingAfterCreation = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterCreation, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":false},\"properties\":{\"field\":{\"type\":\"string\"}}}}")));
XContentBuilder mappingWithOnlyDefaultSet = getMappingWithOnlyTtlDefaultSet("6m");
MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), true).mapping(), false, false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), true).mapping(), false, false);
CompressedXContent mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterMerge, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":false},\"properties\":{\"field\":{\"type\":\"string\"}}}}")));
}
@ -228,8 +225,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
IndexService indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type", mappingWithTtl);
CompressedXContent mappingBeforeMerge = indexService.mapperService().documentMapper("type").mappingSource();
XContentBuilder mappingWithTtlDifferentDefault = getMappingWithTtlEnabled("7d");
MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlDifferentDefault.string()), true).mapping(), true, false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlDifferentDefault.string()), true).mapping(), true, false);
// make sure simulate flag actually worked - no mappings applied
CompressedXContent mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterMerge, equalTo(mappingBeforeMerge));
@ -240,8 +236,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type", mappingWithoutTtl);
mappingBeforeMerge = indexService.mapperService().documentMapper("type").mappingSource();
XContentBuilder mappingWithTtlEnabled = getMappingWithTtlEnabled();
mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), true, false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), true, false);
// make sure simulate flag actually worked - no mappings applied
mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterMerge, equalTo(mappingBeforeMerge));
@ -252,8 +247,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type", mappingWithoutTtl);
mappingBeforeMerge = indexService.mapperService().documentMapper("type").mappingSource();
mappingWithTtlEnabled = getMappingWithTtlEnabled("7d");
mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), true, false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), true, false);
// make sure simulate flag actually worked - no mappings applied
mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterMerge, equalTo(mappingBeforeMerge));
@ -263,8 +257,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
mappingWithoutTtl = getMappingWithTtlDisabled("6d");
indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type", mappingWithoutTtl);
mappingWithTtlEnabled = getMappingWithTtlEnabled("7d");
mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), false, false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), false, false);
// make sure simulate flag actually worked - mappings applied
mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterMerge, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":true,\"default\":604800000},\"properties\":{\"field\":{\"type\":\"string\"}}}}")));
@ -273,8 +266,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
// check if switching simulate flag off works if nothing was applied in the beginning
indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type");
mappingWithTtlEnabled = getMappingWithTtlEnabled("7d");
mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), false, false);
assertFalse(mergeResult.hasConflicts());
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingWithTtlEnabled.string()), true).mapping(), false, false);
// make sure simulate flag actually worked - mappings applied
mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterMerge, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":true,\"default\":604800000},\"properties\":{\"field\":{\"type\":\"string\"}}}}")));

View file

@ -29,7 +29,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.core.LongFieldMapper;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -77,9 +76,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
private void testNoConflictWhileMergingAndMappingChanged(XContentBuilder mapping, XContentBuilder mappingUpdate, XContentBuilder expectedMapping) throws IOException {
IndexService indexService = createIndex("test", Settings.settingsBuilder().build(), "type", mapping);
// simulate like in MetaDataMappingService#putMapping
MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingUpdate.bytes()), true).mapping(), false, false);
// assure we have no conflicts
assertThat(mergeResult.buildConflicts().length, equalTo(0));
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingUpdate.bytes()), true).mapping(), false, false);
// make sure mappings applied
CompressedXContent mappingAfterUpdate = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterUpdate.toString(), equalTo(expectedMapping.string()));
@ -101,9 +98,12 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
IndexService indexService = createIndex("test", Settings.settingsBuilder().build(), "type", mapping);
CompressedXContent mappingBeforeUpdate = indexService.mapperService().documentMapper("type").mappingSource();
// simulate like in MetaDataMappingService#putMapping
MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingUpdate.bytes()), true).mapping(), true, false);
// assure we have conflicts
assertThat(mergeResult.buildConflicts().length, equalTo(1));
try {
indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedXContent(mappingUpdate.bytes()), true).mapping(), true, false);
fail();
} catch (IllegalArgumentException e) {
// expected
}
// make sure simulate flag actually worked - no mappings applied
CompressedXContent mappingAfterUpdate = indexService.mapperService().documentMapper("type").mappingSource();
assertThat(mappingAfterUpdate, equalTo(mappingBeforeUpdate));
@ -202,6 +202,51 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
assertNull(mapperService.documentMapper("type2").mapping().root().getMapper("foo"));
}
public void testReuseMetaField() throws IOException {
XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("_id").field("type", "string").endObject()
.endObject().endObject().endObject();
MapperService mapperService = createIndex("test", Settings.settingsBuilder().build()).mapperService();
try {
mapperService.merge("type", new CompressedXContent(mapping.string()), false, false);
fail();
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]"));
}
try {
mapperService.merge("type", new CompressedXContent(mapping.string()), false, false);
fail();
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]"));
}
}
public void testReuseMetaFieldBackCompat() throws IOException {
XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("_id").field("type", "string").endObject()
.endObject().endObject().endObject();
// the logic is different for 2.x indices since they record some meta mappers (including _id)
// in the root object
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_2_1_0).build();
MapperService mapperService = createIndex("test", settings).mapperService();
try {
mapperService.merge("type", new CompressedXContent(mapping.string()), false, false);
fail();
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]"));
}
try {
mapperService.merge("type", new CompressedXContent(mapping.string()), false, false);
fail();
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]"));
}
}
public void testIndexFieldParsingBackcompat() throws IOException {
IndexService indexService = createIndex("test", Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build());
XContentBuilder indexMapping = XContentFactory.jsonBuilder();

View file

@ -34,13 +34,12 @@ import java.nio.file.Path;
public class BufferedTranslogTests extends TranslogTests {
@Override
protected Translog create(Path path) throws IOException {
protected TranslogConfig getTranslogConfig(Path path) {
Settings build = Settings.settingsBuilder()
.put("index.translog.fs.type", TranslogWriter.Type.BUFFERED.name())
.put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024), ByteSizeUnit.BYTES)
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.build();
TranslogConfig translogConfig = new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
return new Translog(translogConfig);
return new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
}
}

View file

@ -22,9 +22,11 @@ package org.elasticsearch.index.translog;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.Term;
import org.apache.lucene.mockfile.FilterFileChannel;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -110,16 +112,19 @@ public class TranslogTests extends ESTestCase {
}
}
protected Translog create(Path path) throws IOException {
private Translog create(Path path) throws IOException {
return new Translog(getTranslogConfig(path));
}
protected TranslogConfig getTranslogConfig(Path path) {
Settings build = Settings.settingsBuilder()
.put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.SIMPLE.name())
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.build();
TranslogConfig translogConfig = new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
return new Translog(translogConfig);
return new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
}
protected void addToTranslogAndList(Translog translog, ArrayList<Translog.Operation> list, Translog.Operation op) {
protected void addToTranslogAndList(Translog translog, ArrayList<Translog.Operation> list, Translog.Operation op) throws IOException {
list.add(op);
translog.add(op);
}
@ -330,7 +335,7 @@ public class TranslogTests extends ESTestCase {
}
}
public void testSnapshot() {
public void testSnapshot() throws IOException {
ArrayList<Translog.Operation> ops = new ArrayList<>();
Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
@ -389,7 +394,7 @@ public class TranslogTests extends ESTestCase {
Translog.Snapshot snapshot = translog.newSnapshot();
fail("translog is closed");
} catch (AlreadyClosedException ex) {
assertThat(ex.getMessage(), containsString("translog-1.tlog is already closed can't increment"));
assertEquals(ex.getMessage(), "translog is already closed");
}
}
@ -634,7 +639,7 @@ public class TranslogTests extends ESTestCase {
final String threadId = "writer_" + i;
writers[i] = new Thread(new AbstractRunnable() {
@Override
public void doRun() throws BrokenBarrierException, InterruptedException {
public void doRun() throws BrokenBarrierException, InterruptedException, IOException {
barrier.await();
int counter = 0;
while (run.get()) {
@ -1279,4 +1284,122 @@ public class TranslogTests extends ESTestCase {
}
}
}
public void testFailFlush() throws IOException {
Path tempDir = createTempDir();
final AtomicBoolean simulateDiskFull = new AtomicBoolean();
TranslogConfig config = getTranslogConfig(tempDir);
Translog translog = new Translog(config) {
@Override
TranslogWriter.ChannelFactory getChannelFactory() {
final TranslogWriter.ChannelFactory factory = super.getChannelFactory();
return new TranslogWriter.ChannelFactory() {
@Override
public FileChannel open(Path file) throws IOException {
FileChannel channel = factory.open(file);
return new FilterFileChannel(channel) {
@Override
public int write(ByteBuffer src) throws IOException {
if (simulateDiskFull.get()) {
if (src.limit() > 1) {
final int pos = src.position();
final int limit = src.limit();
src.limit(limit / 2);
super.write(src);
src.position(pos);
src.limit(limit);
throw new IOException("__FAKE__ no space left on device");
}
}
return super.write(src);
}
};
}
};
}
};
List<Translog.Location> locations = new ArrayList<>();
int opsSynced = 0;
int opsAdded = 0;
boolean failed = false;
while(failed == false) {
try {
locations.add(translog.add(new Translog.Index("test", "" + opsSynced, Integer.toString(opsSynced).getBytes(Charset.forName("UTF-8")))));
opsAdded++;
translog.sync();
opsSynced++;
} catch (IOException ex) {
failed = true;
assertFalse(translog.isOpen());
assertEquals("__FAKE__ no space left on device", ex.getMessage());
}
simulateDiskFull.set(randomBoolean());
}
simulateDiskFull.set(false);
if (randomBoolean()) {
try {
locations.add(translog.add(new Translog.Index("test", "" + opsSynced, Integer.toString(opsSynced).getBytes(Charset.forName("UTF-8")))));
fail("we are already closed");
} catch (AlreadyClosedException ex) {
assertNotNull(ex.getCause());
assertEquals(ex.getCause().getMessage(), "__FAKE__ no space left on device");
}
}
Translog.TranslogGeneration translogGeneration = translog.getGeneration();
try {
translog.newSnapshot();
fail("already closed");
} catch (AlreadyClosedException ex) {
// all is well
assertNotNull(ex.getCause());
assertSame(translog.getTragicException(), ex.getCause());
}
try {
translog.commit();
fail("already closed");
} catch (AlreadyClosedException ex) {
assertNotNull(ex.getCause());
assertSame(translog.getTragicException(), ex.getCause());
}
assertFalse(translog.isOpen());
translog.close(); // we are closed
config.setTranslogGeneration(translogGeneration);
try (Translog tlog = new Translog(config)){
assertEquals("lastCommitted must be 1 less than current", translogGeneration.translogFileGeneration + 1, tlog.currentFileGeneration());
assertFalse(tlog.syncNeeded());
try (Translog.Snapshot snapshot = tlog.newSnapshot()) {
assertEquals(opsSynced, snapshot.estimatedTotalOperations());
for (int i = 0; i < opsSynced; i++) {
assertEquals("expected operation" + i + " to be in the previous translog but wasn't", tlog.currentFileGeneration() - 1, locations.get(i).generation);
Translog.Operation next = snapshot.next();
assertNotNull("operation " + i + " must be non-null", next);
assertEquals(i, Integer.parseInt(next.getSource().source.toUtf8()));
}
}
}
}
public void testTranslogOpsCountIsCorrect() throws IOException {
List<Translog.Location> locations = new ArrayList<>();
int numOps = randomIntBetween(100, 200);
LineFileDocs lineFileDocs = new LineFileDocs(random()); // writes pretty big docs so we cross buffer boarders regularly
for (int opsAdded = 0; opsAdded < numOps; opsAdded++) {
locations.add(translog.add(new Translog.Index("test", "" + opsAdded, lineFileDocs.nextDoc().toString().getBytes(Charset.forName("UTF-8")))));
try (Translog.Snapshot snapshot = translog.newSnapshot()) {
assertEquals(opsAdded+1, snapshot.estimatedTotalOperations());
for (int i = 0; i < opsAdded; i++) {
assertEquals("expected operation" + i + " to be in the current translog but wasn't", translog.currentFileGeneration(), locations.get(i).generation);
Translog.Operation next = snapshot.next();
assertNotNull("operation " + i + " must be non-null", next);
}
}
}
}
}

View file

@ -99,20 +99,22 @@ public class BooleanTermsIT extends ESIntegTestCase {
final int bucketCount = numSingleFalses > 0 && numSingleTrues > 0 ? 2 : numSingleFalses + numSingleTrues > 0 ? 1 : 0;
assertThat(terms.getBuckets().size(), equalTo(bucketCount));
Terms.Bucket bucket = terms.getBucketByKey("0");
Terms.Bucket bucket = terms.getBucketByKey("false");
if (numSingleFalses == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numSingleFalses, bucket.getDocCount());
assertEquals("false", bucket.getKeyAsString());
}
bucket = terms.getBucketByKey("1");
bucket = terms.getBucketByKey("true");
if (numSingleTrues == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numSingleTrues, bucket.getDocCount());
assertEquals("true", bucket.getKeyAsString());
}
}
@ -131,20 +133,22 @@ public class BooleanTermsIT extends ESIntegTestCase {
final int bucketCount = numMultiFalses > 0 && numMultiTrues > 0 ? 2 : numMultiFalses + numMultiTrues > 0 ? 1 : 0;
assertThat(terms.getBuckets().size(), equalTo(bucketCount));
Terms.Bucket bucket = terms.getBucketByKey("0");
Terms.Bucket bucket = terms.getBucketByKey("false");
if (numMultiFalses == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numMultiFalses, bucket.getDocCount());
assertEquals("false", bucket.getKeyAsString());
}
bucket = terms.getBucketByKey("1");
bucket = terms.getBucketByKey("true");
if (numMultiTrues == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numMultiTrues, bucket.getDocCount());
assertEquals("true", bucket.getKeyAsString());
}
}

View file

@ -423,10 +423,10 @@ public class NestedIT extends ESIntegTestCase {
Terms startDate = response.getAggregations().get("startDate");
assertThat(startDate.getBuckets().size(), equalTo(2));
Terms.Bucket bucket = startDate.getBucketByKey("1414800000000"); // 2014-11-01T00:00:00.000Z
Terms.Bucket bucket = startDate.getBucketByKey("2014-11-01T00:00:00.000Z");
assertThat(bucket.getDocCount(), equalTo(1l));
Terms endDate = bucket.getAggregations().get("endDate");
bucket = endDate.getBucketByKey("1417305600000"); // 2014-11-30T00:00:00.000Z
bucket = endDate.getBucketByKey("2014-11-30T00:00:00.000Z");
assertThat(bucket.getDocCount(), equalTo(1l));
Terms period = bucket.getAggregations().get("period");
bucket = period.getBucketByKey("2014-11");
@ -440,10 +440,10 @@ public class NestedIT extends ESIntegTestCase {
Terms tags = nestedTags.getAggregations().get("tag");
assertThat(tags.getBuckets().size(), equalTo(0)); // and this must be empty
bucket = startDate.getBucketByKey("1417392000000"); // 2014-12-01T00:00:00.000Z
bucket = startDate.getBucketByKey("2014-12-01T00:00:00.000Z");
assertThat(bucket.getDocCount(), equalTo(1l));
endDate = bucket.getAggregations().get("endDate");
bucket = endDate.getBucketByKey("1419984000000"); // 2014-12-31T00:00:00.000Z
bucket = endDate.getBucketByKey("2014-12-31T00:00:00.000Z");
assertThat(bucket.getDocCount(), equalTo(1l));
period = bucket.getAggregations().get("period");
bucket = period.getBucketByKey("2014-12");

View file

@ -61,7 +61,7 @@ public class SearchWhileRelocatingIT extends ESIntegTestCase {
final int numShards = between(1, 20);
client().admin().indices().prepareCreate("test")
.setSettings(settingsBuilder().put("index.number_of_shards", numShards).put("index.number_of_replicas", numberOfReplicas))
.addMapping("type1", "loc", "type=geo_point", "test", "type=string").execute().actionGet();
.addMapping("type", "loc", "type=geo_point", "test", "type=string").execute().actionGet();
ensureGreen();
List<IndexRequestBuilder> indexBuilders = new ArrayList<>();
final int numDocs = between(10, 20);

View file

@ -34,7 +34,11 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.*;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.AbstractQueryTestCase;
import org.elasticsearch.index.query.EmptyQueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
@ -47,7 +51,7 @@ import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder;
import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder.InnerHit;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.highlight.HighlightBuilderTests;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
@ -251,8 +255,7 @@ public class SearchSourceBuilderTests extends ESTestCase {
}
}
if (randomBoolean()) {
// NORELEASE need a random highlight builder method
builder.highlighter(new HighlightBuilder().field(randomAsciiOfLengthBetween(5, 20)));
builder.highlighter(HighlightBuilderTests.randomHighlighterBuilder());
}
if (randomBoolean()) {
// NORELEASE need a random suggest builder method

View file

@ -1176,7 +1176,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
.endObject().endObject()).get();
fail();
} catch (IllegalArgumentException e) {
assertThat(e.toString(), containsString("Merge failed with failures {[The _parent field's type option can't be changed: [null]->[parent]"));
assertThat(e.toString(), containsString("The _parent field's type option can't be changed: [null]->[parent]"));
}
}

View file

@ -53,6 +53,7 @@ import org.elasticsearch.index.query.TermQueryBuilder;
import org.elasticsearch.index.query.TermQueryParser;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.highlight.HighlightBuilder.Field;
import org.elasticsearch.search.highlight.HighlightBuilder.Order;
import org.elasticsearch.search.highlight.SearchContextHighlight.FieldOptions;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
@ -148,7 +149,6 @@ public class HighlightBuilderTests extends ESTestCase {
context.parseFieldMatcher(new ParseFieldMatcher(Settings.EMPTY));
for (int runs = 0; runs < NUMBER_OF_TESTBUILDERS; runs++) {
HighlightBuilder highlightBuilder = randomHighlighterBuilder();
System.out.println(highlightBuilder);
XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values()));
if (randomBoolean()) {
builder.prettyPrint();
@ -416,6 +416,30 @@ public class HighlightBuilderTests extends ESTestCase {
System.out.println(Math.log(1/(double)(1+1)) + 1.0);
}
/**
* test ordinals of {@link Order}, since serialization depends on it
*/
public void testValidOrderOrdinals() {
assertThat(Order.NONE.ordinal(), equalTo(0));
assertThat(Order.SCORE.ordinal(), equalTo(1));
}
public void testOrderSerialization() throws Exception {
try (BytesStreamOutput out = new BytesStreamOutput()) {
Order.NONE.writeTo(out);
try (StreamInput in = StreamInput.wrap(out.bytes())) {
assertThat(in.readVInt(), equalTo(0));
}
}
try (BytesStreamOutput out = new BytesStreamOutput()) {
Order.SCORE.writeTo(out);
try (StreamInput in = StreamInput.wrap(out.bytes())) {
assertThat(in.readVInt(), equalTo(1));
}
}
}
protected static XContentBuilder toXContent(HighlightBuilder highlight, XContentType contentType) throws IOException {
XContentBuilder builder = XContentFactory.contentBuilder(contentType);
if (randomBoolean()) {
@ -426,9 +450,9 @@ public class HighlightBuilderTests extends ESTestCase {
}
/**
* create random shape that is put under test
* create random highlight builder that is put under test
*/
private static HighlightBuilder randomHighlighterBuilder() {
public static HighlightBuilder randomHighlighterBuilder() {
HighlightBuilder testHighlighter = new HighlightBuilder();
setRandomCommonOptions(testHighlighter);
testHighlighter.useExplicitFieldOrder(randomBoolean());
@ -487,7 +511,12 @@ public class HighlightBuilderTests extends ESTestCase {
highlightBuilder.highlightQuery(highlightQuery);
}
if (randomBoolean()) {
highlightBuilder.order(randomAsciiOfLengthBetween(1, 10));
if (randomBoolean()) {
highlightBuilder.order(randomFrom(Order.values()));
} else {
// also test the string setter
highlightBuilder.order(randomFrom(Order.values()).toString());
}
}
if (randomBoolean()) {
highlightBuilder.highlightFilter(randomBoolean());
@ -556,7 +585,11 @@ public class HighlightBuilderTests extends ESTestCase {
highlightBuilder.highlightQuery(new TermQueryBuilder(randomAsciiOfLengthBetween(11, 20), randomAsciiOfLengthBetween(11, 20)));
break;
case 8:
highlightBuilder.order(randomAsciiOfLengthBetween(11, 20));
if (highlightBuilder.order() == Order.NONE) {
highlightBuilder.order(Order.SCORE);
} else {
highlightBuilder.order(Order.NONE);
}
break;
case 9:
highlightBuilder.highlightFilter(toggleOrSet(highlightBuilder.highlightFilter()));

View file

@ -17,7 +17,7 @@ listed in this documentation for inspiration.
====================================
The example site plugin mentioned above contains all of the scaffolding needed
for integrating with Maven builds. If you don't plan on using Maven, then all
for integrating with Gradle builds. If you don't plan on using Gradle, then all
you really need in your plugin is:
* The `plugin-descriptor.properties` file
@ -33,14 +33,14 @@ All plugins, be they site or Java plugins, must contain a file called
`plugin-descriptor.properties` in the root directory. The format for this file
is described in detail here:
https://github.com/elastic/elasticsearch/blob/master/dev-tools/src/main/resources/plugin-metadata/plugin-descriptor.properties[`dev-tools/src/main/resources/plugin-metadata/plugin-descriptor.properties`].
https://github.com/elastic/elasticsearch/blob/master/buildSrc/src/main/resources/plugin-descriptor.properties[`/buildSrc/src/main/resources/plugin-descriptor.properties`].
Either fill in this template yourself (see
https://github.com/lmenezes/elasticsearch-kopf/blob/master/plugin-descriptor.properties[elasticsearch-kopf]
as an example) or, if you are using Elasticsearch's Maven build system, you
can fill in the necessary values in the `pom.xml` for your plugin. For
as an example) or, if you are using Elasticsearch's Gradle build system, you
can fill in the necessary values in the `build.gradle` file for your plugin. For
instance, see
https://github.com/elastic/elasticsearch/blob/master/plugins/site-example/pom.xml[`plugins/site-example/pom.xml`].
https://github.com/elastic/elasticsearch/blob/master/plugins/site-example/build.gradle[`/plugins/site-example/build.gradle`].
[float]
==== Mandatory elements for all plugins
@ -129,7 +129,7 @@ with a large warning, and they will have to confirm them when installing the
plugin interactively. So if possible, it is best to avoid requesting any
spurious permissions!
If you are using the elasticsearch Maven build system, place this file in
If you are using the elasticsearch Gradle build system, place this file in
`src/main/plugin-metadata` and it will be applied during unit tests as well.
Keep in mind that the Java security model is stack-based, and the additional

View file

@ -0,0 +1,118 @@
[[repository-hdfs]]
=== Hadoop HDFS Repository Plugin
The HDFS repository plugin adds support for using HDFS File System as a repository for
{ref}/modules-snapshots.html[Snapshot/Restore].
[[repository-hdfs-install]]
[float]
==== Installation
This plugin can be installed using the plugin manager using _one_ of the following packages:
[source,sh]
----------------------------------------------------------------
sudo bin/plugin install repository-hdfs
sudo bin/plugin install repository-hdfs-hadoop2
sudo bin/plugin install repository-hdfs-lite
----------------------------------------------------------------
The chosen plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
[[repository-hdfs-remove]]
[float]
==== Removal
The plugin can be removed by specifying the _installed_ package using _one_ of the following commands:
[source,sh]
----------------------------------------------------------------
sudo bin/plugin remove repository-hdfs
sudo bin/plugin remove repository-hdfs-hadoop2
sudo bin/plugin remove repository-hdfs-lite
----------------------------------------------------------------
The node must be stopped before removing the plugin.
[[repository-hdfs-usage]]
==== Getting started with HDFS
The HDFS snapshot/restore plugin comes in three _flavors_:
* Default / Hadoop 1.x::
The default version contains the plugin jar alongside Apache Hadoop 1.x (stable) dependencies.
* YARN / Hadoop 2.x::
The `hadoop2` version contains the plugin jar plus the Apache Hadoop 2.x (also known as YARN) dependencies.
* Lite::
The `lite` version contains just the plugin jar, without any Hadoop dependencies. The user should provide these (read below).
[[repository-hdfs-flavor]]
===== What version to use?
It depends on whether Hadoop is locally installed or not and if not, whether it is compatible with Apache Hadoop clients.
* Are you using Apache Hadoop (or a _compatible_ distro) and do not have installed on the Elasticsearch nodes?::
+
If the answer is yes, for Apache Hadoop 1 use the default `repository-hdfs` or `repository-hdfs-hadoop2` for Apache Hadoop 2.
+
* If you are have Hadoop installed locally on the Elasticsearch nodes or are using a certain distro::
+
Use the `lite` version and place your Hadoop _client_ jars and their dependencies in the plugin folder under `hadoop-libs`.
For large deployments, it is recommended to package the libraries in the plugin zip and deploy it manually across nodes
(and thus avoiding having to do the libraries setup on each node).
[[repository-hdfs-security]]
==== Handling JVM Security and Permissions
Out of the box, Elasticsearch runs in a JVM with the security manager turned _on_ to make sure that unsafe or sensitive actions
are allowed only from trusted code. Hadoop however is not really designed to run under one; it does not rely on privileged blocks
to execute sensitive code, of which it uses plenty.
The `repository-hdfs` plugin provides the necessary permissions for both Apache Hadoop 1.x and 2.x (latest versions) to successfully
run in a secured JVM as one can tell from the number of permissions required when installing the plugin.
However using a certain Hadoop File-System (outside DFS), a certain distro or operating system (in particular Windows), might require
additional permissions which are not provided by the plugin.
In this case there are several workarounds:
* add the permission into `plugin-security.policy` (available in the plugin folder)
* disable the security manager through `es.security.manager.enabled=false` configurations setting - NOT RECOMMENDED
If you find yourself in such a situation, please let us know what Hadoop distro version and OS you are using and what permission is missing
by raising an issue. Thank you!
[[repository-hdfs-config]]
==== Configuration Properties
Once installed, define the configuration for the `hdfs` repository through `elasticsearch.yml` or the
{ref}/modules-snapshots.html[REST API]:
[source,yaml]
----
repositories
hdfs:
uri: "hdfs://<host>:<port>/" \# optional - Hadoop file-system URI
path: "some/path" \# required - path with the file-system where data is stored/loaded
load_defaults: "true" \# optional - whether to load the default Hadoop configuration (default) or not
conf_location: "extra-cfg.xml" \# optional - Hadoop configuration XML to be loaded (use commas for multi values)
conf.<key> : "<value>" \# optional - 'inlined' key=value added to the Hadoop configuration
concurrent_streams: 5 \# optional - the number of concurrent streams (defaults to 5)
compress: "false" \# optional - whether to compress the metadata or not (default)
chunk_size: "10mb" \# optional - chunk size (disabled by default)
----
NOTE: Be careful when including a paths within the `uri` setting; Some implementations ignore them completely while
others consider them. In general, we recommend keeping the `uri` to a minimum and using the `path` element instead.
[[repository-hdfs-other-fs]]
==== Plugging other file-systems
Any HDFS-compatible file-systems (like Amazon `s3://` or Google `gs://`) can be used as long as the proper Hadoop
configuration is passed to the Elasticsearch plugin. In practice, this means making sure the correct Hadoop configuration
files (`core-site.xml` and `hdfs-site.xml`) and its jars are available in plugin classpath, just as you would with any
other Hadoop client or job.
Otherwise, the plugin will only read the _default_, vanilla configuration of Hadoop and will not be able to recognized
the plugged-in file-system.

View file

@ -18,10 +18,9 @@ The S3 repository plugin adds support for using S3 as a repository.
The Azure repository plugin adds support for using Azure as a repository.
https://github.com/elastic/elasticsearch-hadoop/tree/master/repository-hdfs[Hadoop HDFS Repository]::
<<repository-hdfs,HDFS Repository>>::
The Hadoop HDFS Repository plugin adds support for using an HDFS file system
as a repository.
The Hadoop HDFS Repository plugin adds support for using HDFS as a repository.
[float]
@ -40,3 +39,5 @@ include::repository-azure.asciidoc[]
include::repository-s3.asciidoc[]
include::repository-hdfs.asciidoc[]

View file

@ -66,7 +66,7 @@ GET my_index/_search
==== Using `nested` fields for arrays of objects
If you need to index arrays of objects and to maintain the independence of
each object in the array, you should used the `nested` datatype instead of the
each object in the array, you should use the `nested` datatype instead of the
<<object,`object`>> datatype. Internally, nested objects index each object in
the array as a separate hidden document, meaning that each nested object can be
queried independently of the others, with the <<query-dsl-nested-query,`nested` query>>:
@ -110,7 +110,7 @@ GET my_index/_search
"bool": {
"must": [
{ "match": { "user.first": "Alice" }},
{ "match": { "user.last": "White" }} <2>
{ "match": { "user.last": "Smith" }} <2>
]
}
}
@ -127,7 +127,7 @@ GET my_index/_search
"bool": {
"must": [
{ "match": { "user.first": "Alice" }},
{ "match": { "user.last": "Smith" }} <3>
{ "match": { "user.last": "White" }} <3>
]
}
},
@ -137,14 +137,14 @@ GET my_index/_search
"user.first": {}
}
}
}
}
}
}
--------------------------------------------------
// AUTOSENSE
<1> The `user` field is mapped as type `nested` instead of type `object`.
<2> This query doesn't match because `Alice` and `White` are not in the same nested object.
<2> This query doesn't match because `Alice` and `Smith` are not in the same nested object.
<3> This query matches because `Alice` and `White` are in the same nested object.
<4> `inner_hits` allow us to highlight the matching nested documents.

View file

@ -392,8 +392,7 @@ public class SearchFieldsTests extends ESIntegTestCase {
createIndex("test");
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().execute().actionGet();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
.startObject("_source").field("enabled", false).endObject()
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_source").field("enabled", false).endObject().startObject("properties")
.startObject("byte_field").field("type", "byte").field("store", "yes").endObject()
.startObject("short_field").field("type", "short").field("store", "yes").endObject()
.startObject("integer_field").field("type", "integer").field("store", "yes").endObject()
@ -556,8 +555,7 @@ public class SearchFieldsTests extends ESIntegTestCase {
createIndex("test");
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().execute().actionGet();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
.startObject("_source").field("enabled", false).endObject()
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_source").field("enabled", false).endObject().startObject("properties")
.startObject("string_field").field("type", "string").endObject()
.startObject("byte_field").field("type", "byte").endObject()
.startObject("short_field").field("type", "short").endObject()

View file

@ -37,7 +37,6 @@ import java.util.*;
import static org.elasticsearch.index.mapper.MapperBuilders.*;
import static org.elasticsearch.index.mapper.core.TypeParsers.parseMultiField;
import static org.elasticsearch.index.mapper.core.TypeParsers.parsePathType;
/**
* <pre>
@ -65,7 +64,6 @@ public class AttachmentMapper extends FieldMapper {
public static final String CONTENT_TYPE = "attachment";
public static class Defaults {
public static final ContentPath.Type PATH_TYPE = ContentPath.Type.FULL;
public static final AttachmentFieldType FIELD_TYPE = new AttachmentFieldType();
static {
@ -108,8 +106,6 @@ public class AttachmentMapper extends FieldMapper {
public static class Builder extends FieldMapper.Builder<Builder, AttachmentMapper> {
private ContentPath.Type pathType = Defaults.PATH_TYPE;
private Boolean ignoreErrors = null;
private Integer defaultIndexedChars = null;
@ -140,11 +136,6 @@ public class AttachmentMapper extends FieldMapper {
this.contentBuilder = stringField(FieldNames.CONTENT);
}
public Builder pathType(ContentPath.Type pathType) {
this.pathType = pathType;
return this;
}
public Builder content(Mapper.Builder content) {
this.contentBuilder = content;
return this;
@ -192,8 +183,6 @@ public class AttachmentMapper extends FieldMapper {
@Override
public AttachmentMapper build(BuilderContext context) {
ContentPath.Type origPathType = context.path().pathType();
context.path().pathType(pathType);
FieldMapper contentMapper;
if (context.indexCreatedVersion().before(Version.V_2_0_0_beta1)) {
@ -220,8 +209,6 @@ public class AttachmentMapper extends FieldMapper {
FieldMapper language = (FieldMapper) languageBuilder.build(context);
context.path().remove();
context.path().pathType(origPathType);
if (defaultIndexedChars == null && context.indexSettings() != null) {
defaultIndexedChars = context.indexSettings().getAsInt("index.mapping.attachment.indexed_chars", 100000);
}
@ -257,7 +244,7 @@ public class AttachmentMapper extends FieldMapper {
defaultFieldType.freeze();
this.setupFieldType(context);
return new AttachmentMapper(name, fieldType, defaultFieldType, pathType, defaultIndexedChars, ignoreErrors, langDetect, contentMapper,
return new AttachmentMapper(name, fieldType, defaultFieldType, defaultIndexedChars, ignoreErrors, langDetect, contentMapper,
dateMapper, titleMapper, nameMapper, authorMapper, keywordsMapper, contentTypeMapper, contentLength,
language, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@ -309,10 +296,7 @@ public class AttachmentMapper extends FieldMapper {
Map.Entry<String, Object> entry = iterator.next();
String fieldName = entry.getKey();
Object fieldNode = entry.getValue();
if (fieldName.equals("path") && parserContext.indexVersionCreated().before(Version.V_2_0_0_beta1)) {
builder.pathType(parsePathType(name, fieldNode.toString()));
iterator.remove();
} else if (fieldName.equals("fields")) {
if (fieldName.equals("fields")) {
Map<String, Object> fieldsNode = (Map<String, Object>) fieldNode;
for (Iterator<Map.Entry<String, Object>> fieldsIterator = fieldsNode.entrySet().iterator(); fieldsIterator.hasNext();) {
Map.Entry<String, Object> entry1 = fieldsIterator.next();
@ -375,8 +359,6 @@ public class AttachmentMapper extends FieldMapper {
}
}
private final ContentPath.Type pathType;
private final int defaultIndexedChars;
private final boolean ignoreErrors;
@ -401,13 +383,12 @@ public class AttachmentMapper extends FieldMapper {
private final FieldMapper languageMapper;
public AttachmentMapper(String simpleName, MappedFieldType type, MappedFieldType defaultFieldType, ContentPath.Type pathType, int defaultIndexedChars, Boolean ignoreErrors,
public AttachmentMapper(String simpleName, MappedFieldType type, MappedFieldType defaultFieldType, int defaultIndexedChars, Boolean ignoreErrors,
Boolean defaultLangDetect, FieldMapper contentMapper,
FieldMapper dateMapper, FieldMapper titleMapper, FieldMapper nameMapper, FieldMapper authorMapper,
FieldMapper keywordsMapper, FieldMapper contentTypeMapper, FieldMapper contentLengthMapper,
FieldMapper languageMapper, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, type, defaultFieldType, indexSettings, multiFields, copyTo);
this.pathType = pathType;
this.defaultIndexedChars = defaultIndexedChars;
this.ignoreErrors = ignoreErrors;
this.defaultLangDetect = defaultLangDetect;
@ -602,7 +583,7 @@ public class AttachmentMapper extends FieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// ignore this for now
}
@ -626,9 +607,6 @@ public class AttachmentMapper extends FieldMapper {
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(simpleName());
builder.field("type", CONTENT_TYPE);
if (indexCreatedBefore2x) {
builder.field("path", pathType.name().toLowerCase(Locale.ROOT));
}
builder.startObject("fields");
contentMapper.toXContent(builder, params);

View file

@ -66,8 +66,7 @@ public class Murmur3FieldMapper extends LongFieldMapper {
Murmur3FieldMapper fieldMapper = new Murmur3FieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context),
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
fieldMapper.includeInAll(includeInAll);
return fieldMapper;
return (Murmur3FieldMapper) fieldMapper.includeInAll(includeInAll);
}
@Override

View file

@ -28,7 +28,6 @@ import org.elasticsearch.index.analysis.NumericIntegerAnalyzer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.IntegerFieldMapper;
@ -177,12 +176,10 @@ public class SizeFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) {
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
SizeFieldMapper sizeFieldMapperMergeWith = (SizeFieldMapper) mergeWith;
if (!mergeResult.simulate()) {
if (sizeFieldMapperMergeWith.enabledState != enabledState && !sizeFieldMapperMergeWith.enabledState.unset()) {
this.enabledState = sizeFieldMapperMergeWith.enabledState;
}
if (sizeFieldMapperMergeWith.enabledState != enabledState && !sizeFieldMapperMergeWith.enabledState.unset()) {
this.enabledState = sizeFieldMapperMergeWith.enabledState;
}
}
}

View file

@ -0,0 +1,203 @@
/*
* 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.
*/
//apply plugin: 'nebula.provided-base'
esplugin {
description 'The HDFS repository plugin adds support for Hadoop Distributed File-System (HDFS) repositories.'
classname 'org.elasticsearch.plugin.hadoop.hdfs.HdfsPlugin'
}
configurations {
hadoop1
hadoop2
}
versions << [
'hadoop1': '1.2.1',
'hadoop2': '2.7.1'
]
dependencies {
provided "org.elasticsearch:elasticsearch:${versions.elasticsearch}"
provided "org.apache.hadoop:hadoop-core:${versions.hadoop1}"
// use Hadoop1 to compile and test things (a subset of Hadoop2)
testCompile "org.apache.hadoop:hadoop-core:${versions.hadoop1}"
testCompile "org.apache.hadoop:hadoop-test:${versions.hadoop1}"
// Hadoop dependencies
testCompile "commons-configuration:commons-configuration:1.6"
testCompile "commons-lang:commons-lang:${versions.commonslang}"
testCompile "commons-collections:commons-collections:3.2.2"
testCompile "commons-net:commons-net:1.4.1"
testCompile "org.mortbay.jetty:jetty:6.1.26"
testCompile "org.mortbay.jetty:jetty-util:6.1.26"
testCompile "org.mortbay.jetty:servlet-api:2.5-20081211"
testCompile "com.sun.jersey:jersey-core:1.8"
hadoop1("org.apache.hadoop:hadoop-core:${versions.hadoop1}") {
exclude module: "commons-cli"
exclude group: "com.sun.jersey"
exclude group: "org.mortbay.jetty"
exclude group: "tomcat"
exclude module: "commons-el"
exclude module: "hsqldb"
exclude group: "org.eclipse.jdt"
exclude module: "commons-beanutils"
exclude module: "commons-beanutils-core"
exclude module: "junit"
// provided by ES itself
exclude group: "log4j"
}
hadoop2("org.apache.hadoop:hadoop-client:${versions.hadoop2}") {
exclude module: "commons-cli"
exclude group: "com.sun.jersey"
exclude group: "com.sun.jersey.contribs"
exclude group: "com.sun.jersey.jersey-test-framework"
exclude module: "guice"
exclude group: "org.mortbay.jetty"
exclude group: "tomcat"
exclude module: "commons-el"
exclude module: "hsqldb"
exclude group: "org.eclipse.jdt"
exclude module: "commons-beanutils"
exclude module: "commons-beanutils-core"
exclude module: "javax.servlet"
exclude module: "junit"
// provided by ES itself
exclude group: "log4j"
}
hadoop2("org.apache.hadoop:hadoop-hdfs:${versions.hadoop2}") {
exclude module: "guava"
exclude module: "junit"
// provided by ES itself
exclude group: "log4j"
}
}
configurations.all {
resolutionStrategy {
force "commons-codec:commons-codec:${versions.commonscodec}"
force "commons-logging:commons-logging:${versions.commonslogging}"
force "commons-lang:commons-lang:2.6"
force "commons-httpclient:commons-httpclient:3.0.1"
force "org.codehaus.jackson:jackson-core-asl:1.8.8"
force "org.codehaus.jackson:jackson-mapper-asl:1.8.8"
force "com.google.code.findbugs:jsr305:3.0.0"
force "com.google.guava:guava:16.0.1"
force "org.slf4j:slf4j-api:1.7.10"
force "org.slf4j:slf4j-log4j12:1.7.10"
}
}
dependencyLicenses {
mapping from: /hadoop-core.*/, to: 'hadoop-1'
mapping from: /hadoop-.*/, to: 'hadoop-2'
}
compileJava.options.compilerArgs << '-Xlint:-deprecation,-rawtypes'
// main jar includes just the plugin classes
jar {
include "org/elasticsearch/plugin/hadoop/hdfs/*"
}
// hadoop jar (which actually depend on Hadoop)
task hadoopLinkedJar(type: Jar, dependsOn:jar) {
appendix "internal"
from sourceSets.main.output.classesDir
// exclude plugin
exclude "org/elasticsearch/plugin/hadoop/hdfs/*"
}
bundlePlugin.dependsOn hadoopLinkedJar
// configure 'bundle' as being w/o Hadoop deps
bundlePlugin {
into ("internal-libs") {
from hadoopLinkedJar.archivePath
}
into ("hadoop-libs") {
from configurations.hadoop2.allArtifacts.files
from configurations.hadoop2
}
}
task distZipHadoop1(type: Zip, dependsOn: [hadoopLinkedJar, jar]) { zipTask ->
from (zipTree(bundlePlugin.archivePath)) {
include "*"
include "internal-libs/**"
}
description = "Builds archive (with Hadoop1 dependencies) suitable for download page."
classifier = "hadoop1"
into ("hadoop-libs") {
from configurations.hadoop1.allArtifacts.files
from configurations.hadoop1
}
}
task distZipHadoop2(type: Zip, dependsOn: [hadoopLinkedJar, jar]) { zipTask ->
from (zipTree(bundlePlugin.archivePath)) {
include "*"
include "internal-libs/**"
}
description = "Builds archive (with Hadoop2/YARN dependencies) suitable for download page."
classifier = "hadoop2"
into ("hadoop-libs") {
from configurations.hadoop2.allArtifacts.files
from configurations.hadoop2
}
}
task distZipNoHadoop(type: Zip, dependsOn: [hadoopLinkedJar, jar]) { zipTask ->
from (zipTree(bundlePlugin.archivePath)) {
exclude "hadoop-libs/**"
}
from sourceSets.main.output.resourcesDir
description = "Builds archive (without any Hadoop dependencies) suitable for download page."
classifier = "lite"
}
artifacts {
archives bundlePlugin
'default' bundlePlugin
archives distZipHadoop1
archives distZipHadoop2
archives distZipNoHadoop
}
integTest {
cluster {
plugin(pluginProperties.extension.name, zipTree(distZipHadoop2.archivePath))
}
}

View file

@ -0,0 +1,173 @@
/*
* 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.plugin.hadoop.hdfs;
import java.io.IOException;
import java.lang.reflect.Method;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
import java.net.URLClassLoader;
import java.nio.file.Path;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardRepository;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.repositories.RepositoriesModule;
import org.elasticsearch.repositories.Repository;
//
// Note this plugin is somewhat special as Hadoop itself loads a number of libraries and thus requires a number of permissions to run even in client mode.
// This poses two problems:
// - Hadoop itself comes with tons of jars, many providing the same classes across packages. In particular Hadoop 2 provides package annotations in the same
// package across jars which trips JarHell. Thus, to allow Hadoop jars to load, the plugin uses a dedicated CL which picks them up from the hadoop-libs folder.
// - The issue though with using a different CL is that it picks up the jars from a different location / codeBase and thus it does not fall under the plugin
// permissions. In other words, the plugin permissions don't apply to the hadoop libraries.
// There are different approaches here:
// - implement a custom classloader that loads the jars but 'lies' about the codesource. It is doable but since URLClassLoader is locked down, one would
// would have to implement the whole jar opening and loading from it. Not impossible but still fairly low-level.
// Further more, even if the code has the proper credentials, it needs to use the proper Privileged blocks to use its full permissions which does not
// happen in the Hadoop code base.
// - use a different Policy. Works but the Policy is JVM wide and thus the code needs to be quite efficient - quite a bit impact to cover just some plugin
// libraries
// - use a DomainCombiner. This doesn't change the semantics (it's clear where the code is loaded from, etc..) however it gives us a scoped, fine-grained
// callback on handling the permission intersection for secured calls. Note that DC works only in the current PAC call - the moment another PA is used,
// the domain combiner is going to be ignored (unless the caller specifically uses it). Due to its scoped impact and official Java support, this approach
// was used.
// ClassLoading info
// - package plugin.hadoop.hdfs is part of the plugin
// - all the other packages are assumed to be in the nested Hadoop CL.
// Code
public class HdfsPlugin extends Plugin {
@Override
public String name() {
return "repository-hdfs";
}
@Override
public String description() {
return "HDFS Repository Plugin";
}
@SuppressWarnings("unchecked")
public void onModule(RepositoriesModule repositoriesModule) {
String baseLib = Utils.detectLibFolder();
List<URL> cp = getHadoopClassLoaderPath(baseLib);
ClassLoader hadoopCL = URLClassLoader.newInstance(cp.toArray(new URL[cp.size()]), getClass().getClassLoader());
Class<? extends Repository> repository = null;
try {
repository = (Class<? extends Repository>) hadoopCL.loadClass("org.elasticsearch.repositories.hdfs.HdfsRepository");
} catch (ClassNotFoundException cnfe) {
throw new IllegalStateException("Cannot load plugin class; is the plugin class setup correctly?", cnfe);
}
repositoriesModule.registerRepository("hdfs", repository, BlobStoreIndexShardRepository.class);
Loggers.getLogger(HdfsPlugin.class).info("Loaded Hadoop [{}] libraries from {}", getHadoopVersion(hadoopCL), baseLib);
}
protected List<URL> getHadoopClassLoaderPath(String baseLib) {
List<URL> cp = new ArrayList<>();
// add plugin internal jar
discoverJars(createURI(baseLib, "internal-libs"), cp, false);
// add Hadoop jars
discoverJars(createURI(baseLib, "hadoop-libs"), cp, true);
return cp;
}
private String getHadoopVersion(ClassLoader hadoopCL) {
SecurityManager sm = System.getSecurityManager();
if (sm != null) {
// unprivileged code such as scripts do not have SpecialPermission
sm.checkPermission(new SpecialPermission());
}
return AccessController.doPrivileged(new PrivilegedAction<String>() {
@Override
public String run() {
// Hadoop 2 relies on TCCL to determine the version
ClassLoader tccl = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(hadoopCL);
return doGetHadoopVersion(hadoopCL);
} finally {
Thread.currentThread().setContextClassLoader(tccl);
}
}
}, Utils.hadoopACC());
}
private String doGetHadoopVersion(ClassLoader hadoopCL) {
String version = "Unknown";
Class<?> clz = null;
try {
clz = hadoopCL.loadClass("org.apache.hadoop.util.VersionInfo");
} catch (ClassNotFoundException cnfe) {
// unknown
}
if (clz != null) {
try {
Method method = clz.getMethod("getVersion");
version = method.invoke(null).toString();
} catch (Exception ex) {
// class has changed, ignore
}
}
return version;
}
private URI createURI(String base, String suffix) {
String location = base + suffix;
try {
return new URI(location);
} catch (URISyntaxException ex) {
throw new IllegalStateException(String.format(Locale.ROOT, "Cannot detect plugin folder; [%s] seems invalid", location), ex);
}
}
@SuppressForbidden(reason = "discover nested jar")
private void discoverJars(URI libPath, List<URL> cp, boolean optional) {
try {
Path[] jars = FileSystemUtils.files(PathUtils.get(libPath), "*.jar");
for (Path path : jars) {
cp.add(path.toUri().toURL());
}
} catch (IOException ex) {
if (!optional) {
throw new IllegalStateException("Cannot compute plugin classpath", ex);
}
}
}
}

View file

@ -0,0 +1,84 @@
package org.elasticsearch.plugin.hadoop.hdfs;
import java.net.URL;
import java.security.AccessControlContext;
import java.security.AccessController;
import java.security.DomainCombiner;
import java.security.PrivilegedAction;
import java.security.ProtectionDomain;
import org.elasticsearch.SpecialPermission;
public abstract class Utils {
protected static AccessControlContext hadoopACC() {
SecurityManager sm = System.getSecurityManager();
if (sm != null) {
// unprivileged code such as scripts do not have SpecialPermission
sm.checkPermission(new SpecialPermission());
}
return AccessController.doPrivileged(new PrivilegedAction<AccessControlContext>() {
@Override
public AccessControlContext run() {
return new AccessControlContext(AccessController.getContext(), new HadoopDomainCombiner());
}
});
}
private static class HadoopDomainCombiner implements DomainCombiner {
private static String BASE_LIB = detectLibFolder();
@Override
public ProtectionDomain[] combine(ProtectionDomain[] currentDomains, ProtectionDomain[] assignedDomains) {
for (ProtectionDomain pd : assignedDomains) {
if (pd.getCodeSource().getLocation().toString().startsWith(BASE_LIB)) {
return assignedDomains;
}
}
return currentDomains;
}
}
static String detectLibFolder() {
ClassLoader cl = Utils.class.getClassLoader();
// we could get the URL from the URLClassloader directly
// but that can create issues when running the tests from the IDE
// we could detect that by loading resources but that as well relies on
// the JAR URL
String classToLookFor = HdfsPlugin.class.getName().replace(".", "/").concat(".class");
URL classURL = cl.getResource(classToLookFor);
if (classURL == null) {
throw new IllegalStateException("Cannot detect itself; something is wrong with this ClassLoader " + cl);
}
String base = classURL.toString();
// extract root
// typically a JAR URL
int index = base.indexOf("!/");
if (index > 0) {
base = base.substring(0, index);
// remove its prefix (jar:)
base = base.substring(4);
// remove the trailing jar
index = base.lastIndexOf("/");
base = base.substring(0, index + 1);
}
// not a jar - something else, do a best effort here
else {
// remove the class searched
base = base.substring(0, base.length() - classToLookFor.length());
}
// append /
if (!base.endsWith("/")) {
base = base.concat("/");
}
return base;
}
}

Some files were not shown because too many files have changed in this diff Show more