Merge branch 'master' into jigsaw

This commit is contained in:
Ryan Ernst 2015-12-04 11:29:49 -08:00
commit 01d48e2062
101 changed files with 2376 additions and 887 deletions
buildSrc/src/main/groovy/org/elasticsearch/gradle
core/src
main/java/org/elasticsearch
test/java/org/elasticsearch
distribution
docs
plugins

@ -23,19 +23,19 @@ import org.elasticsearch.gradle.test.RestIntegTestTask
import org.elasticsearch.gradle.test.RunTask
import org.gradle.api.Project
import org.gradle.api.Task
import org.gradle.api.file.FileCollection
import org.gradle.api.tasks.SourceSet
import org.gradle.api.tasks.bundling.Zip
/**
* Encapsulates build configuration for an Elasticsearch plugin.
*/
class PluginBuildPlugin extends BuildPlugin {
public class PluginBuildPlugin extends BuildPlugin {
@Override
void apply(Project project) {
public void apply(Project project) {
super.apply(project)
configureDependencies(project)
// this afterEvaluate must happen before the afterEvaluate added by integTest configure,
// this afterEvaluate must happen before the afterEvaluate added by integTest creation,
// so that the file name resolution for installing the plugin will be setup
project.afterEvaluate {
String name = project.pluginProperties.extension.name
@ -52,18 +52,12 @@ class PluginBuildPlugin extends BuildPlugin {
project.tasks.run.clusterConfig.plugin(name, project.bundlePlugin.outputs.files)
}
}
RestIntegTestTask.configure(project)
RunTask.configure(project)
Task bundle = configureBundleTask(project)
project.configurations.archives.artifacts.removeAll { it.archiveTask.is project.jar }
project.configurations.getByName('default').extendsFrom = []
project.artifacts {
archives bundle
'default' bundle
}
createIntegTestTask(project)
createBundleTask(project)
project.tasks.create('run', RunTask) // allow running ES with this plugin in the foreground of a build
}
static void configureDependencies(Project project) {
private static void configureDependencies(Project project) {
project.dependencies {
provided "org.elasticsearch:elasticsearch:${project.versions.elasticsearch}"
testCompile "org.elasticsearch:test-framework:${project.versions.elasticsearch}"
@ -79,21 +73,36 @@ class PluginBuildPlugin extends BuildPlugin {
}
}
static Task configureBundleTask(Project project) {
PluginPropertiesTask buildProperties = project.tasks.create(name: 'pluginProperties', type: PluginPropertiesTask)
File pluginMetadata = project.file("src/main/plugin-metadata")
project.sourceSets.test {
output.dir(buildProperties.generatedResourcesDir, builtBy: 'pluginProperties')
resources {
srcDir pluginMetadata
}
}
Task bundle = project.tasks.create(name: 'bundlePlugin', type: Zip, dependsOn: [project.jar, buildProperties])
bundle.configure {
from buildProperties
from pluginMetadata
from project.jar
from bundle.project.configurations.runtime - bundle.project.configurations.provided
/** Adds an integTest task which runs rest tests */
private static void createIntegTestTask(Project project) {
RestIntegTestTask integTest = project.tasks.create('integTest', RestIntegTestTask.class)
integTest.mustRunAfter(project.precommit, project.test)
project.check.dependsOn(integTest)
}
/**
* Adds a bundlePlugin task which builds the zip containing the plugin jars,
* metadata, properties, and packaging files
*/
private static void createBundleTask(Project project) {
File pluginMetadata = project.file('src/main/plugin-metadata')
// create a task to build the properties file for this plugin
PluginPropertiesTask buildProperties = project.tasks.create('pluginProperties', PluginPropertiesTask.class)
// add the plugin properties and metadata to test resources, so unit tests can
// know about the plugin (used by test security code to statically initialize the plugin in unit tests)
SourceSet testSourceSet = project.sourceSets.test
testSourceSet.output.dir(buildProperties.generatedResourcesDir, builtBy: 'pluginProperties')
testSourceSet.resources.srcDir(pluginMetadata)
// create the actual bundle task, which zips up all the files for the plugin
Zip bundle = project.tasks.create(name: 'bundlePlugin', type: Zip, dependsOn: [project.jar, buildProperties]) {
from buildProperties // plugin properties file
from pluginMetadata // metadata (eg custom security policy)
from project.jar // this plugin's jar
from project.configurations.runtime - project.configurations.provided // the dep jars
// extra files for the plugin to go into the zip
from('src/main/packaging') // TODO: move all config/bin/_size/etc into packaging
from('src/main') {
include 'config/**'
@ -104,6 +113,13 @@ class PluginBuildPlugin extends BuildPlugin {
}
}
project.assemble.dependsOn(bundle)
return bundle
// remove jar from the archives (things that will be published), and set it to the zip
project.configurations.archives.artifacts.removeAll { it.archiveTask.is project.jar }
project.artifacts.add('archives', bundle)
// also make the zip the default artifact (used when depending on this project)
project.configurations.getByName('default').extendsFrom = []
project.artifacts.add('default', bundle)
}
}

@ -69,7 +69,7 @@ import java.util.regex.Pattern
* </pre>
*/
public class DependencyLicensesTask extends DefaultTask {
private static final String SHA_EXTENSION = '.sha1'
static final String SHA_EXTENSION = '.sha1'
// TODO: we should be able to default this to eg compile deps, but we need to move the licenses
// check from distribution to core (ie this should only be run on java projects)

@ -35,6 +35,7 @@ public class UpdateShasTask extends DefaultTask {
public UpdateShasTask() {
description = 'Updates the sha files for the dependencyLicenses check'
onlyIf { parentTask.licensesDir.exists() }
}
@TaskAction
@ -42,13 +43,13 @@ public class UpdateShasTask extends DefaultTask {
Set<File> shaFiles = new HashSet<File>()
parentTask.licensesDir.eachFile {
String name = it.getName()
if (name.endsWith(SHA_EXTENSION)) {
if (name.endsWith(DependencyLicensesTask.SHA_EXTENSION)) {
shaFiles.add(it)
}
}
for (File dependency : parentTask.dependencies) {
String jarName = dependency.getName()
File shaFile = new File(parentTask.licensesDir, jarName + SHA_EXTENSION)
File shaFile = new File(parentTask.licensesDir, jarName + DependencyLicensesTask.SHA_EXTENSION)
if (shaFile.exists() == false) {
logger.lifecycle("Adding sha for ${jarName}")
String sha = MessageDigest.getInstance("SHA-1").digest(dependency.getBytes()).encodeHex().toString()

@ -31,55 +31,38 @@ import org.gradle.util.ConfigureUtil
* Runs integration tests, but first starts an ES cluster,
* and passes the ES cluster info as parameters to the tests.
*/
class RestIntegTestTask extends RandomizedTestingTask {
public class RestIntegTestTask extends RandomizedTestingTask {
ClusterConfiguration clusterConfig = new ClusterConfiguration()
/** Flag indicating whether the rest tests in the rest spec should be run. */
@Input
boolean includePackaged = false
static RestIntegTestTask configure(Project project) {
Map integTestOptions = [
name: 'integTest',
type: RestIntegTestTask,
dependsOn: 'testClasses',
group: JavaBasePlugin.VERIFICATION_GROUP,
description: 'Runs rest tests against an elasticsearch cluster.'
]
RestIntegTestTask integTest = project.tasks.create(integTestOptions)
integTest.configure(BuildPlugin.commonTestConfig(project))
integTest.configure {
include '**/*IT.class'
systemProperty 'tests.rest.load_packaged', 'false'
}
RandomizedTestingTask test = project.tasks.findByName('test')
if (test != null) {
integTest.classpath = test.classpath
integTest.testClassesDir = test.testClassesDir
integTest.mustRunAfter(test)
}
integTest.mustRunAfter(project.precommit)
project.check.dependsOn(integTest)
public RestIntegTestTask() {
description = 'Runs rest tests against an elasticsearch cluster.'
group = JavaBasePlugin.VERIFICATION_GROUP
dependsOn(project.testClasses)
classpath = project.sourceSets.test.runtimeClasspath
testClassesDir = project.sourceSets.test.output.classesDir
// start with the common test configuration
configure(BuildPlugin.commonTestConfig(project))
// override/add more for rest tests
parallelism = '1'
include('**/*IT.class')
systemProperty('tests.rest.load_packaged', 'false')
// copy the rest spec/tests into the test resources
RestSpecHack.configureDependencies(project)
project.afterEvaluate {
integTest.dependsOn(RestSpecHack.configureTask(project, integTest.includePackaged))
dependsOn(RestSpecHack.configureTask(project, includePackaged))
systemProperty('tests.cluster', "localhost:${clusterConfig.baseTransportPort}")
}
return integTest
}
RestIntegTestTask() {
// this must run after all projects have been configured, so we know any project
// references can be accessed as a fully configured
project.gradle.projectsEvaluated {
Task test = project.tasks.findByName('test')
if (test != null) {
mustRunAfter(test)
}
ClusterFormationTasks.setup(project, this, clusterConfig)
configure {
parallelism '1'
systemProperty 'tests.cluster', "localhost:${clusterConfig.baseTransportPort}"
}
}
}
@ -92,11 +75,11 @@ class RestIntegTestTask extends RandomizedTestingTask {
}
@Input
void cluster(Closure closure) {
public void cluster(Closure closure) {
ConfigureUtil.configure(closure, clusterConfig)
}
ClusterConfiguration getCluster() {
public ClusterConfiguration getCluster() {
return clusterConfig
}
}

@ -28,12 +28,12 @@ import org.gradle.api.tasks.Copy
* currently must be available on the local filesystem. This class encapsulates
* setting up tasks to copy the rest spec api to test resources.
*/
class RestSpecHack {
public class RestSpecHack {
/**
* Sets dependencies needed to copy the rest spec.
* @param project The project to add rest spec dependency to
*/
static void configureDependencies(Project project) {
public static void configureDependencies(Project project) {
project.configurations {
restSpec
}
@ -48,7 +48,7 @@ class RestSpecHack {
* @param project The project to add the copy task to
* @param includePackagedTests true if the packaged tests should be copied, false otherwise
*/
static Task configureTask(Project project, boolean includePackagedTests) {
public static Task configureTask(Project project, boolean includePackagedTests) {
Map copyRestSpecProps = [
name : 'copyRestSpec',
type : Copy,
@ -65,7 +65,6 @@ class RestSpecHack {
project.idea {
module {
if (scopes.TEST != null) {
// TODO: need to add the TEST scope somehow for rest test plugin...
scopes.TEST.plus.add(project.configurations.restSpec)
}
}

@ -18,23 +18,19 @@
*/
package org.elasticsearch.gradle.test
import com.carrotsearch.gradle.junit4.RandomizedTestingTask
import org.gradle.api.Plugin
import org.gradle.api.Project
/** Configures the build to have a rest integration test. */
class RestTestPlugin implements Plugin<Project> {
/** A plugin to add rest integration tests. Used for qa projects. */
public class RestTestPlugin implements Plugin<Project> {
@Override
void apply(Project project) {
public void apply(Project project) {
project.pluginManager.apply(StandaloneTestBasePlugin)
RandomizedTestingTask integTest = RestIntegTestTask.configure(project)
RestSpecHack.configureDependencies(project)
integTest.configure {
classpath = project.sourceSets.test.runtimeClasspath
testClassesDir project.sourceSets.test.output.classesDir
cluster.distribution = 'zip' // rest tests should run with the real zip
}
RestIntegTestTask integTest = project.tasks.create('integTest', RestIntegTestTask.class)
integTest.cluster.distribution = 'zip' // rest tests should run with the real zip
integTest.mustRunAfter(project.precommit)
project.check.dependsOn(integTest)
}
}

@ -2,13 +2,17 @@ package org.elasticsearch.gradle.test
import org.gradle.api.DefaultTask
import org.gradle.api.Project
import org.gradle.api.Task
import org.gradle.api.internal.tasks.options.Option
import org.gradle.util.ConfigureUtil
class RunTask extends DefaultTask {
public class RunTask extends DefaultTask {
ClusterConfiguration clusterConfig = new ClusterConfiguration(baseHttpPort: 9200, baseTransportPort: 9300, daemonize: false)
RunTask() {
public RunTask() {
description = "Runs elasticsearch with '${project.path}'"
group = 'Verification'
project.afterEvaluate {
ClusterFormationTasks.setup(project, this, clusterConfig)
}
@ -22,11 +26,10 @@ class RunTask extends DefaultTask {
clusterConfig.debug = enabled;
}
static void configure(Project project) {
RunTask task = project.tasks.create(
name: 'run',
type: RunTask,
description: "Runs elasticsearch with '${project.path}'",
group: 'Verification')
/** Configure the cluster that will be run. */
@Override
public Task configure(Closure closure) {
ConfigureUtil.configure(closure, clusterConfig)
return this
}
}

@ -27,35 +27,26 @@ import org.elasticsearch.gradle.precommit.PrecommitTasks
import org.gradle.api.Plugin
import org.gradle.api.Project
import org.gradle.api.plugins.JavaBasePlugin
import org.gradle.plugins.ide.eclipse.model.EclipseClasspath
/** Configures the build to have a rest integration test. */
class StandaloneTestBasePlugin implements Plugin<Project> {
public class StandaloneTestBasePlugin implements Plugin<Project> {
@Override
void apply(Project project) {
public void apply(Project project) {
project.pluginManager.apply(JavaBasePlugin)
project.pluginManager.apply(RandomizedTestingPlugin)
BuildPlugin.globalBuildInfo(project)
BuildPlugin.configureRepositories(project)
// remove some unnecessary tasks for a qa test
project.tasks.removeAll { it.name in ['assemble', 'buildDependents'] }
// only setup tests to build
project.sourceSets {
test
}
project.dependencies {
testCompile "org.elasticsearch:test-framework:${VersionProperties.elasticsearch}"
}
project.sourceSets.create('test')
project.dependencies.add('testCompile', "org.elasticsearch:test-framework:${VersionProperties.elasticsearch}")
project.eclipse.classpath.sourceSets = [project.sourceSets.test]
project.eclipse.classpath.plusConfigurations = [project.configurations.testRuntime]
project.eclipse {
classpath {
sourceSets = [project.sourceSets.test]
plusConfigurations = [project.configurations.testRuntime]
}
}
PrecommitTasks.create(project, false)
project.check.dependsOn(project.precommit)
}

@ -25,11 +25,11 @@ import org.gradle.api.Plugin
import org.gradle.api.Project
import org.gradle.api.plugins.JavaBasePlugin
/** Configures the build to have only unit tests. */
class StandaloneTestPlugin implements Plugin<Project> {
/** A plugin to add tests only. Used for QA tests that run arbitrary unit tests. */
public class StandaloneTestPlugin implements Plugin<Project> {
@Override
void apply(Project project) {
public void apply(Project project) {
project.pluginManager.apply(StandaloneTestBasePlugin)
Map testOptions = [
@ -41,10 +41,8 @@ class StandaloneTestPlugin implements Plugin<Project> {
]
RandomizedTestingTask test = project.tasks.create(testOptions)
test.configure(BuildPlugin.commonTestConfig(project))
test.configure {
classpath = project.sourceSets.test.runtimeClasspath
testClassesDir project.sourceSets.test.output.classesDir
}
test.classpath = project.sourceSets.test.runtimeClasspath
test.testClassesDir project.sourceSets.test.output.classesDir
test.mustRunAfter(project.precommit)
project.check.dependsOn(test)
}

@ -554,7 +554,7 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
NODE_DISCONNECTED_EXCEPTION(org.elasticsearch.transport.NodeDisconnectedException.class, org.elasticsearch.transport.NodeDisconnectedException::new, 84),
ALREADY_EXPIRED_EXCEPTION(org.elasticsearch.index.AlreadyExpiredException.class, org.elasticsearch.index.AlreadyExpiredException::new, 85),
AGGREGATION_EXECUTION_EXCEPTION(org.elasticsearch.search.aggregations.AggregationExecutionException.class, org.elasticsearch.search.aggregations.AggregationExecutionException::new, 86),
MERGE_MAPPING_EXCEPTION(org.elasticsearch.index.mapper.MergeMappingException.class, org.elasticsearch.index.mapper.MergeMappingException::new, 87),
// 87 used to be for MergeMappingException
INVALID_INDEX_TEMPLATE_EXCEPTION(org.elasticsearch.indices.InvalidIndexTemplateException.class, org.elasticsearch.indices.InvalidIndexTemplateException::new, 88),
PERCOLATE_EXCEPTION(org.elasticsearch.percolator.PercolateException.class, org.elasticsearch.percolator.PercolateException::new, 89),
REFRESH_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.RefreshFailedEngineException.class, org.elasticsearch.index.engine.RefreshFailedEngineException::new, 90),

@ -300,7 +300,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
String parent = null;
String[] fields = defaultFields;
String timestamp = null;
Long ttl = null;
TimeValue ttl = null;
String opType = null;
long version = Versions.MATCH_ANY;
VersionType versionType = VersionType.INTERNAL;
@ -333,9 +333,9 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
timestamp = parser.text();
} else if ("_ttl".equals(currentFieldName) || "ttl".equals(currentFieldName)) {
if (parser.currentToken() == XContentParser.Token.VALUE_STRING) {
ttl = TimeValue.parseTimeValue(parser.text(), null, currentFieldName).millis();
ttl = TimeValue.parseTimeValue(parser.text(), null, currentFieldName);
} else {
ttl = parser.longValue();
ttl = new TimeValue(parser.longValue());
}
} else if ("op_type".equals(currentFieldName) || "opType".equals(currentFieldName)) {
opType = parser.text();

@ -335,7 +335,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
indexRequest.process(clusterState.metaData(), mappingMd, allowIdGeneration, request.index());
}
return executeIndexRequestOnPrimary(request, indexRequest, indexShard);
return executeIndexRequestOnPrimary(indexRequest, indexShard);
}
private WriteResult<DeleteResponse> shardDeleteOperation(BulkShardRequest request, DeleteRequest deleteRequest, IndexShard indexShard) {

@ -35,6 +35,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.*;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.VersionType;
@ -136,7 +137,8 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
private String parent;
@Nullable
private String timestamp;
private long ttl = -1;
@Nullable
private TimeValue ttl;
private BytesReference source;
@ -229,6 +231,12 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
if (!versionType.validateVersionForWrites(version)) {
validationException = addValidationError("illegal version value [" + version + "] for version type [" + versionType.name() + "]", validationException);
}
if (ttl != null) {
if (ttl.millis() < 0) {
validationException = addValidationError("ttl must not be negative", validationException);
}
}
return validationException;
}
@ -324,22 +332,33 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
}
/**
* Sets the relative ttl value. It musts be &gt; 0 as it makes little sense otherwise. Setting it
* to <tt>null</tt> will reset to have no ttl.
* Sets the ttl value as a time value expression.
*/
public IndexRequest ttl(Long ttl) throws ElasticsearchGenerationException {
if (ttl == null) {
this.ttl = -1;
return this;
}
if (ttl <= 0) {
throw new IllegalArgumentException("TTL value must be > 0. Illegal value provided [" + ttl + "]");
}
public IndexRequest ttl(String ttl) {
this.ttl = TimeValue.parseTimeValue(ttl, null, "ttl");
return this;
}
/**
* Sets the ttl as a {@link TimeValue} instance.
*/
public IndexRequest ttl(TimeValue ttl) {
this.ttl = ttl;
return this;
}
public long ttl() {
/**
* Sets the relative ttl value in milliseconds. It musts be greater than 0 as it makes little sense otherwise.
*/
public IndexRequest ttl(long ttl) {
this.ttl = new TimeValue(ttl);
return this;
}
/**
* Returns the ttl as a {@link TimeValue}
*/
public TimeValue ttl() {
return this.ttl;
}
@ -665,7 +684,7 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
routing = in.readOptionalString();
parent = in.readOptionalString();
timestamp = in.readOptionalString();
ttl = in.readLong();
ttl = in.readBoolean() ? TimeValue.readTimeValue(in) : null;
source = in.readBytesReference();
opType = OpType.fromId(in.readByte());
@ -682,7 +701,12 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
out.writeOptionalString(routing);
out.writeOptionalString(parent);
out.writeOptionalString(timestamp);
out.writeLong(ttl);
if (ttl == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
ttl.writeTo(out);
}
out.writeBytesReference(source);
out.writeByte(opType.id());
out.writeBoolean(refresh);

@ -23,6 +23,7 @@ import org.elasticsearch.action.support.replication.ReplicationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
@ -254,9 +255,27 @@ public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest,
return this;
}
// Sets the relative ttl value. It musts be > 0 as it makes little sense otherwise.
/**
* Sets the ttl value as a time value expression.
*/
public IndexRequestBuilder setTTL(String ttl) {
request.ttl(ttl);
return this;
}
/**
* Sets the relative ttl value in milliseconds. It musts be greater than 0 as it makes little sense otherwise.
*/
public IndexRequestBuilder setTTL(long ttl) {
request.ttl(ttl);
return this;
}
/**
* Sets the ttl as a {@link TimeValue} instance.
*/
public IndexRequestBuilder setTTL(TimeValue ttl) {
request.ttl(ttl);
return this;
}
}

@ -166,7 +166,7 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardRequest.shardId.id());
final WriteResult<IndexResponse> result = executeIndexRequestOnPrimary(null, request, indexShard);
final WriteResult<IndexResponse> result = executeIndexRequestOnPrimary(request, indexShard);
final IndexResponse response = result.response;
final Translog.Location location = result.location;

@ -25,7 +25,6 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionWriteResponse;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexRequest.OpType;
import org.elasticsearch.action.index.IndexResponse;
@ -1074,23 +1073,22 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
/** Utility method to create either an index or a create operation depending
* on the {@link OpType} of the request. */
private final Engine.Index prepareIndexOperationOnPrimary(BulkShardRequest shardRequest, IndexRequest request, IndexShard indexShard) {
private Engine.Index prepareIndexOperationOnPrimary(IndexRequest request, IndexShard indexShard) {
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.source()).index(request.index()).type(request.type()).id(request.id())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());
return indexShard.prepareIndex(sourceToParse, request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY);
}
/** Execute the given {@link IndexRequest} on a primary shard, throwing a
* {@link RetryOnPrimaryException} if the operation needs to be re-tried. */
protected final WriteResult<IndexResponse> executeIndexRequestOnPrimary(BulkShardRequest shardRequest, IndexRequest request, IndexShard indexShard) throws Throwable {
Engine.Index operation = prepareIndexOperationOnPrimary(shardRequest, request, indexShard);
protected final WriteResult<IndexResponse> executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard) throws Throwable {
Engine.Index operation = prepareIndexOperationOnPrimary(request, indexShard);
Mapping update = operation.parsedDoc().dynamicMappingsUpdate();
final ShardId shardId = indexShard.shardId();
if (update != null) {
final String indexName = shardId.getIndex();
mappingUpdatedAction.updateMappingOnMasterSynchronously(indexName, request.type(), update);
operation = prepareIndexOperationOnPrimary(shardRequest, request, indexShard);
operation = prepareIndexOperationOnPrimary(request, indexShard);
update = operation.parsedDoc().dynamicMappingsUpdate();
if (update != null) {
throw new RetryOnPrimaryException(shardId,

@ -88,7 +88,7 @@ public class UpdateHelper extends AbstractComponent {
throw new DocumentMissingException(new ShardId(request.index(), request.shardId()), request.type(), request.id());
}
IndexRequest indexRequest = request.docAsUpsert() ? request.doc() : request.upsertRequest();
Long ttl = indexRequest.ttl();
TimeValue ttl = indexRequest.ttl();
if (request.scriptedUpsert() && request.script() != null) {
// Run the script to perform the create logic
IndexRequest upsert = request.upsertRequest();
@ -99,7 +99,7 @@ public class UpdateHelper extends AbstractComponent {
ctx.put("_source", upsertDoc);
ctx = executeScript(request, ctx);
//Allow the script to set TTL using ctx._ttl
if (ttl < 0) {
if (ttl == null) {
ttl = getTTLFromScriptContext(ctx);
}
@ -124,7 +124,7 @@ public class UpdateHelper extends AbstractComponent {
indexRequest.index(request.index()).type(request.type()).id(request.id())
// it has to be a "create!"
.create(true)
.ttl(ttl == null || ttl < 0 ? null : ttl)
.ttl(ttl)
.refresh(request.refresh())
.routing(request.routing())
.parent(request.parent())
@ -151,7 +151,7 @@ public class UpdateHelper extends AbstractComponent {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true);
String operation = null;
String timestamp = null;
Long ttl = null;
TimeValue ttl = null;
final Map<String, Object> updatedSourceAsMap;
final XContentType updateSourceContentType = sourceAndContent.v1();
String routing = getResult.getFields().containsKey(RoutingFieldMapper.NAME) ? getResult.field(RoutingFieldMapper.NAME).getValue().toString() : null;
@ -160,7 +160,7 @@ public class UpdateHelper extends AbstractComponent {
if (request.script() == null && request.doc() != null) {
IndexRequest indexRequest = request.doc();
updatedSourceAsMap = sourceAndContent.v2();
if (indexRequest.ttl() > 0) {
if (indexRequest.ttl() != null) {
ttl = indexRequest.ttl();
}
timestamp = indexRequest.timestamp();
@ -211,9 +211,9 @@ public class UpdateHelper extends AbstractComponent {
// apply script to update the source
// No TTL has been given in the update script so we keep previous TTL value if there is one
if (ttl == null) {
ttl = getResult.getFields().containsKey(TTLFieldMapper.NAME) ? (Long) getResult.field(TTLFieldMapper.NAME).getValue() : null;
if (ttl != null) {
ttl = ttl - TimeValue.nsecToMSec(System.nanoTime() - getDateNS); // It is an approximation of exact TTL value, could be improved
Long ttlAsLong = getResult.getFields().containsKey(TTLFieldMapper.NAME) ? (Long) getResult.field(TTLFieldMapper.NAME).getValue() : null;
if (ttlAsLong != null) {
ttl = new TimeValue(ttlAsLong - TimeValue.nsecToMSec(System.nanoTime() - getDateNS));// It is an approximation of exact TTL value, could be improved
}
}
@ -256,17 +256,15 @@ public class UpdateHelper extends AbstractComponent {
return ctx;
}
private Long getTTLFromScriptContext(Map<String, Object> ctx) {
Long ttl = null;
private TimeValue getTTLFromScriptContext(Map<String, Object> ctx) {
Object fetchedTTL = ctx.get("_ttl");
if (fetchedTTL != null) {
if (fetchedTTL instanceof Number) {
ttl = ((Number) fetchedTTL).longValue();
} else {
ttl = TimeValue.parseTimeValue((String) fetchedTTL, null, "_ttl").millis();
return new TimeValue(((Number) fetchedTTL).longValue());
}
return TimeValue.parseTimeValue((String) fetchedTTL, null, "_ttl");
}
return ttl;
return null;
}
/**
@ -337,13 +335,10 @@ public class UpdateHelper extends AbstractComponent {
}
}
public static enum Operation {
public enum Operation {
UPSERT,
INDEX,
DELETE,
NONE
}
}

@ -24,6 +24,7 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.single.instance.InstanceShardOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
@ -325,7 +326,7 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
}
/**
* Set the new ttl of the document. Note that if detectNoop is true (the default)
* Set the new ttl of the document as a long. Note that if detectNoop is true (the default)
* and the source of the document isn't changed then the ttl update won't take
* effect.
*/
@ -333,4 +334,24 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
request.doc().ttl(ttl);
return this;
}
/**
* Set the new ttl of the document as a time value expression. Note that if detectNoop is true (the default)
* and the source of the document isn't changed then the ttl update won't take
* effect.
*/
public UpdateRequestBuilder setTtl(String ttl) {
request.doc().ttl(ttl);
return this;
}
/**
* Set the new ttl of the document as a {@link TimeValue} instance. Note that if detectNoop is true (the default)
* and the source of the document isn't changed then the ttl update won't take
* effect.
*/
public UpdateRequestBuilder setTtl(TimeValue ttl) {
request.doc().ttl(ttl);
return this;
}
}

@ -19,9 +19,9 @@
package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.cluster.DiffableUtils.KeyedReader;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -469,6 +469,16 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
}
builder.endArray();
builder.startObject(IndexMetaData.KEY_ACTIVE_ALLOCATIONS);
for (IntObjectCursor<Set<String>> cursor : indexMetaData.getActiveAllocationIds()) {
builder.startArray(String.valueOf(cursor.key));
for (String allocationId : cursor.value) {
builder.value(allocationId);
}
builder.endArray();
}
builder.endObject();
builder.endObject();
}
builder.endObject();
@ -584,6 +594,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
public Builder routingResult(RoutingAllocation.Result routingResult) {
this.routingTable = routingResult.routingTable();
this.metaData = routingResult.metaData();
return this;
}
@ -759,7 +770,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
nodes = after.nodes.diff(before.nodes);
metaData = after.metaData.diff(before.metaData);
blocks = after.blocks.diff(before.blocks);
customs = DiffableUtils.diff(before.customs, after.customs);
customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer());
}
public ClusterStateDiff(StreamInput in, ClusterState proto) throws IOException {
@ -771,14 +782,15 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
nodes = proto.nodes.readDiffFrom(in);
metaData = proto.metaData.readDiffFrom(in);
blocks = proto.blocks.readDiffFrom(in);
customs = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader<Custom>() {
customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
new DiffableUtils.DiffableValueSerializer<String, Custom>() {
@Override
public Custom readFrom(StreamInput in, String key) throws IOException {
public Custom read(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readFrom(in);
}
@Override
public Diff<Custom> readDiffFrom(StreamInput in, String key) throws IOException {
public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readDiffFrom(in);
}
});

@ -29,7 +29,7 @@ import java.io.IOException;
public interface Diff<T> {
/**
* Applies difference to the specified part and retunrs the resulted part
* Applies difference to the specified part and returns the resulted part
*/
T apply(T part);

@ -19,263 +19,630 @@
package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.IntCursor;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
public final class DiffableUtils {
private DiffableUtils() {
}
/**
* Returns a map key serializer for String keys
*/
public static KeySerializer<String> getStringKeySerializer() {
return StringKeySerializer.INSTANCE;
}
/**
* Returns a map key serializer for Integer keys. Encodes as Int.
*/
public static KeySerializer<Integer> getIntKeySerializer() {
return IntKeySerializer.INSTANCE;
}
/**
* Returns a map key serializer for Integer keys. Encodes as VInt.
*/
public static KeySerializer<Integer> getVIntKeySerializer() {
return VIntKeySerializer.INSTANCE;
}
/**
* Calculates diff between two ImmutableOpenMaps of Diffable objects
*/
public static <T extends Diffable<T>> Diff<ImmutableOpenMap<String, T>> diff(ImmutableOpenMap<String, T> before, ImmutableOpenMap<String, T> after) {
public static <K, T extends Diffable<T>> MapDiff<K, T, ImmutableOpenMap<K, T>> diff(ImmutableOpenMap<K, T> before, ImmutableOpenMap<K, T> after, KeySerializer<K> keySerializer) {
assert after != null && before != null;
return new ImmutableOpenMapDiff<>(before, after);
return new ImmutableOpenMapDiff<>(before, after, keySerializer, DiffableValueSerializer.getWriteOnlyInstance());
}
/**
* Calculates diff between two ImmutableOpenMaps of non-diffable objects
*/
public static <K, T> MapDiff<K, T, ImmutableOpenMap<K, T>> diff(ImmutableOpenMap<K, T> before, ImmutableOpenMap<K, T> after, KeySerializer<K> keySerializer, NonDiffableValueSerializer<K, T> valueSerializer) {
assert after != null && before != null;
return new ImmutableOpenMapDiff<>(before, after, keySerializer, valueSerializer);
}
/**
* Calculates diff between two ImmutableOpenIntMaps of Diffable objects
*/
public static <T extends Diffable<T>> MapDiff<Integer, T, ImmutableOpenIntMap<T>> diff(ImmutableOpenIntMap<T> before, ImmutableOpenIntMap<T> after, KeySerializer<Integer> keySerializer) {
assert after != null && before != null;
return new ImmutableOpenIntMapDiff<>(before, after, keySerializer, DiffableValueSerializer.getWriteOnlyInstance());
}
/**
* Calculates diff between two ImmutableOpenIntMaps of non-diffable objects
*/
public static <T> MapDiff<Integer, T, ImmutableOpenIntMap<T>> diff(ImmutableOpenIntMap<T> before, ImmutableOpenIntMap<T> after, KeySerializer<Integer> keySerializer, NonDiffableValueSerializer<Integer, T> valueSerializer) {
assert after != null && before != null;
return new ImmutableOpenIntMapDiff<>(before, after, keySerializer, valueSerializer);
}
/**
* Calculates diff between two Maps of Diffable objects.
*/
public static <T extends Diffable<T>> Diff<Map<String, T>> diff(Map<String, T> before, Map<String, T> after) {
public static <K, T extends Diffable<T>> MapDiff<K, T, Map<K, T>> diff(Map<K, T> before, Map<K, T> after, KeySerializer<K> keySerializer) {
assert after != null && before != null;
return new JdkMapDiff<>(before, after);
return new JdkMapDiff<>(before, after, keySerializer, DiffableValueSerializer.getWriteOnlyInstance());
}
/**
* Calculates diff between two Maps of non-diffable objects
*/
public static <K, T> MapDiff<K, T, Map<K, T>> diff(Map<K, T> before, Map<K, T> after, KeySerializer<K> keySerializer, NonDiffableValueSerializer<K, T> valueSerializer) {
assert after != null && before != null;
return new JdkMapDiff<>(before, after, keySerializer, valueSerializer);
}
/**
* Loads an object that represents difference between two ImmutableOpenMaps
*/
public static <T extends Diffable<T>> Diff<ImmutableOpenMap<String, T>> readImmutableOpenMapDiff(StreamInput in, KeyedReader<T> keyedReader) throws IOException {
return new ImmutableOpenMapDiff<>(in, keyedReader);
}
/**
* Loads an object that represents difference between two Maps.
*/
public static <T extends Diffable<T>> Diff<Map<String, T>> readJdkMapDiff(StreamInput in, KeyedReader<T> keyedReader) throws IOException {
return new JdkMapDiff<>(in, keyedReader);
public static <K, T> MapDiff<K, T, ImmutableOpenMap<K, T>> readImmutableOpenMapDiff(StreamInput in, KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) throws IOException {
return new ImmutableOpenMapDiff<>(in, keySerializer, valueSerializer);
}
/**
* Loads an object that represents difference between two ImmutableOpenMaps
*/
public static <T extends Diffable<T>> Diff<ImmutableOpenMap<String, T>> readImmutableOpenMapDiff(StreamInput in, T proto) throws IOException {
return new ImmutableOpenMapDiff<>(in, new PrototypeReader<>(proto));
public static <T> MapDiff<Integer, T, ImmutableOpenIntMap<T>> readImmutableOpenIntMapDiff(StreamInput in, KeySerializer<Integer> keySerializer, ValueSerializer<Integer, T> valueSerializer) throws IOException {
return new ImmutableOpenIntMapDiff<>(in, keySerializer, valueSerializer);
}
/**
* Loads an object that represents difference between two Maps.
* Loads an object that represents difference between two Maps of Diffable objects
*/
public static <T extends Diffable<T>> Diff<Map<String, T>> readJdkMapDiff(StreamInput in, T proto) throws IOException {
return new JdkMapDiff<>(in, new PrototypeReader<>(proto));
public static <K, T> MapDiff<K, T, Map<K, T>> readJdkMapDiff(StreamInput in, KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) throws IOException {
return new JdkMapDiff<>(in, keySerializer, valueSerializer);
}
/**
* A reader that can deserialize an object. The reader can select the deserialization type based on the key. It's
* used in custom metadata deserialization.
* Loads an object that represents difference between two ImmutableOpenMaps of Diffable objects using Diffable proto object
*/
public interface KeyedReader<T> {
/**
* reads an object of the type T from the stream input
*/
T readFrom(StreamInput in, String key) throws IOException;
/**
* reads an object that respresents differences between two objects with the type T from the stream input
*/
Diff<T> readDiffFrom(StreamInput in, String key) throws IOException;
public static <K, T extends Diffable<T>> MapDiff<K, T, ImmutableOpenMap<K, T>> readImmutableOpenMapDiff(StreamInput in, KeySerializer<K> keySerializer, T proto) throws IOException {
return new ImmutableOpenMapDiff<>(in, keySerializer, new DiffablePrototypeValueReader<>(proto));
}
/**
* Implementation of the KeyedReader that is using a prototype object for reading operations
*
* Note: this implementation is ignoring the key.
* Loads an object that represents difference between two ImmutableOpenIntMaps of Diffable objects using Diffable proto object
*/
public static class PrototypeReader<T extends Diffable<T>> implements KeyedReader<T> {
private T proto;
public PrototypeReader(T proto) {
this.proto = proto;
}
@Override
public T readFrom(StreamInput in, String key) throws IOException {
return proto.readFrom(in);
}
@Override
public Diff<T> readDiffFrom(StreamInput in, String key) throws IOException {
return proto.readDiffFrom(in);
}
public static <T extends Diffable<T>> MapDiff<Integer, T, ImmutableOpenIntMap<T>> readImmutableOpenIntMapDiff(StreamInput in, KeySerializer<Integer> keySerializer, T proto) throws IOException {
return new ImmutableOpenIntMapDiff<>(in, keySerializer, new DiffablePrototypeValueReader<>(proto));
}
/**
* Represents differences between two Maps of Diffable objects.
* Loads an object that represents difference between two Maps of Diffable objects using Diffable proto object
*/
public static <K, T extends Diffable<T>> MapDiff<K, T, Map<K, T>> readJdkMapDiff(StreamInput in, KeySerializer<K> keySerializer, T proto) throws IOException {
return new JdkMapDiff<>(in, keySerializer, new DiffablePrototypeValueReader<>(proto));
}
/**
* Represents differences between two Maps of (possibly diffable) objects.
*
* @param <T> the diffable object
*/
private static class JdkMapDiff<T extends Diffable<T>> extends MapDiff<T, Map<String, T>> {
private static class JdkMapDiff<K, T> extends MapDiff<K, T, Map<K, T>> {
protected JdkMapDiff(StreamInput in, KeyedReader<T> reader) throws IOException {
super(in, reader);
protected JdkMapDiff(StreamInput in, KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) throws IOException {
super(in, keySerializer, valueSerializer);
}
public JdkMapDiff(Map<String, T> before, Map<String, T> after) {
public JdkMapDiff(Map<K, T> before, Map<K, T> after,
KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) {
super(keySerializer, valueSerializer);
assert after != null && before != null;
for (String key : before.keySet()) {
for (K key : before.keySet()) {
if (!after.containsKey(key)) {
deletes.add(key);
}
}
for (Map.Entry<String, T> partIter : after.entrySet()) {
for (Map.Entry<K, T> partIter : after.entrySet()) {
T beforePart = before.get(partIter.getKey());
if (beforePart == null) {
adds.put(partIter.getKey(), partIter.getValue());
upserts.put(partIter.getKey(), partIter.getValue());
} else if (partIter.getValue().equals(beforePart) == false) {
diffs.put(partIter.getKey(), partIter.getValue().diff(beforePart));
if (valueSerializer.supportsDiffableValues()) {
diffs.put(partIter.getKey(), valueSerializer.diff(partIter.getValue(), beforePart));
} else {
upserts.put(partIter.getKey(), partIter.getValue());
}
}
}
}
@Override
public Map<String, T> apply(Map<String, T> map) {
Map<String, T> builder = new HashMap<>();
public Map<K, T> apply(Map<K, T> map) {
Map<K, T> builder = new HashMap<>();
builder.putAll(map);
for (String part : deletes) {
for (K part : deletes) {
builder.remove(part);
}
for (Map.Entry<String, Diff<T>> diff : diffs.entrySet()) {
for (Map.Entry<K, Diff<T>> diff : diffs.entrySet()) {
builder.put(diff.getKey(), diff.getValue().apply(builder.get(diff.getKey())));
}
for (Map.Entry<String, T> additon : adds.entrySet()) {
builder.put(additon.getKey(), additon.getValue());
for (Map.Entry<K, T> upsert : upserts.entrySet()) {
builder.put(upsert.getKey(), upsert.getValue());
}
return builder;
}
}
/**
* Represents differences between two ImmutableOpenMap of diffable objects
* Represents differences between two ImmutableOpenMap of (possibly diffable) objects
*
* @param <T> the diffable object
* @param <T> the object type
*/
private static class ImmutableOpenMapDiff<T extends Diffable<T>> extends MapDiff<T, ImmutableOpenMap<String, T>> {
private static class ImmutableOpenMapDiff<K, T> extends MapDiff<K, T, ImmutableOpenMap<K, T>> {
protected ImmutableOpenMapDiff(StreamInput in, KeyedReader<T> reader) throws IOException {
super(in, reader);
protected ImmutableOpenMapDiff(StreamInput in, KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) throws IOException {
super(in, keySerializer, valueSerializer);
}
public ImmutableOpenMapDiff(ImmutableOpenMap<String, T> before, ImmutableOpenMap<String, T> after) {
public ImmutableOpenMapDiff(ImmutableOpenMap<K, T> before, ImmutableOpenMap<K, T> after,
KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) {
super(keySerializer, valueSerializer);
assert after != null && before != null;
for (ObjectCursor<String> key : before.keys()) {
for (ObjectCursor<K> key : before.keys()) {
if (!after.containsKey(key.value)) {
deletes.add(key.value);
}
}
for (ObjectObjectCursor<String, T> partIter : after) {
for (ObjectObjectCursor<K, T> partIter : after) {
T beforePart = before.get(partIter.key);
if (beforePart == null) {
adds.put(partIter.key, partIter.value);
upserts.put(partIter.key, partIter.value);
} else if (partIter.value.equals(beforePart) == false) {
diffs.put(partIter.key, partIter.value.diff(beforePart));
if (valueSerializer.supportsDiffableValues()) {
diffs.put(partIter.key, valueSerializer.diff(partIter.value, beforePart));
} else {
upserts.put(partIter.key, partIter.value);
}
}
}
}
@Override
public ImmutableOpenMap<String, T> apply(ImmutableOpenMap<String, T> map) {
ImmutableOpenMap.Builder<String, T> builder = ImmutableOpenMap.builder();
public ImmutableOpenMap<K, T> apply(ImmutableOpenMap<K, T> map) {
ImmutableOpenMap.Builder<K, T> builder = ImmutableOpenMap.builder();
builder.putAll(map);
for (String part : deletes) {
for (K part : deletes) {
builder.remove(part);
}
for (Map.Entry<String, Diff<T>> diff : diffs.entrySet()) {
for (Map.Entry<K, Diff<T>> diff : diffs.entrySet()) {
builder.put(diff.getKey(), diff.getValue().apply(builder.get(diff.getKey())));
}
for (Map.Entry<String, T> additon : adds.entrySet()) {
builder.put(additon.getKey(), additon.getValue());
for (Map.Entry<K, T> upsert : upserts.entrySet()) {
builder.put(upsert.getKey(), upsert.getValue());
}
return builder.build();
}
}
/**
* Represents differences between two maps of diffable objects
* Represents differences between two ImmutableOpenIntMap of (possibly diffable) objects
*
* This class is used as base class for different map implementations
*
* @param <T> the diffable object
* @param <T> the object type
*/
private static abstract class MapDiff<T extends Diffable<T>, M> implements Diff<M> {
private static class ImmutableOpenIntMapDiff<T> extends MapDiff<Integer, T, ImmutableOpenIntMap<T>> {
protected final List<String> deletes;
protected final Map<String, Diff<T>> diffs;
protected final Map<String, T> adds;
protected MapDiff() {
deletes = new ArrayList<>();
diffs = new HashMap<>();
adds = new HashMap<>();
protected ImmutableOpenIntMapDiff(StreamInput in, KeySerializer<Integer> keySerializer, ValueSerializer<Integer, T> valueSerializer) throws IOException {
super(in, keySerializer, valueSerializer);
}
protected MapDiff(StreamInput in, KeyedReader<T> reader) throws IOException {
public ImmutableOpenIntMapDiff(ImmutableOpenIntMap<T> before, ImmutableOpenIntMap<T> after,
KeySerializer<Integer> keySerializer, ValueSerializer<Integer, T> valueSerializer) {
super(keySerializer, valueSerializer);
assert after != null && before != null;
for (IntCursor key : before.keys()) {
if (!after.containsKey(key.value)) {
deletes.add(key.value);
}
}
for (IntObjectCursor<T> partIter : after) {
T beforePart = before.get(partIter.key);
if (beforePart == null) {
upserts.put(partIter.key, partIter.value);
} else if (partIter.value.equals(beforePart) == false) {
if (valueSerializer.supportsDiffableValues()) {
diffs.put(partIter.key, valueSerializer.diff(partIter.value, beforePart));
} else {
upserts.put(partIter.key, partIter.value);
}
}
}
}
@Override
public ImmutableOpenIntMap<T> apply(ImmutableOpenIntMap<T> map) {
ImmutableOpenIntMap.Builder<T> builder = ImmutableOpenIntMap.builder();
builder.putAll(map);
for (Integer part : deletes) {
builder.remove(part);
}
for (Map.Entry<Integer, Diff<T>> diff : diffs.entrySet()) {
builder.put(diff.getKey(), diff.getValue().apply(builder.get(diff.getKey())));
}
for (Map.Entry<Integer, T> upsert : upserts.entrySet()) {
builder.put(upsert.getKey(), upsert.getValue());
}
return builder.build();
}
}
/**
* Represents differences between two maps of objects and is used as base class for different map implementations.
*
* Implements serialization. How differences are applied is left to subclasses.
*
* @param <K> the type of map keys
* @param <T> the type of map values
* @param <M> the map implementation type
*/
public static abstract class MapDiff<K, T, M> implements Diff<M> {
protected final List<K> deletes;
protected final Map<K, Diff<T>> diffs; // incremental updates
protected final Map<K, T> upserts; // additions or full updates
protected final KeySerializer<K> keySerializer;
protected final ValueSerializer<K, T> valueSerializer;
protected MapDiff(KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) {
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
deletes = new ArrayList<>();
diffs = new HashMap<>();
adds = new HashMap<>();
upserts = new HashMap<>();
}
protected MapDiff(StreamInput in, KeySerializer<K> keySerializer, ValueSerializer<K, T> valueSerializer) throws IOException {
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
deletes = new ArrayList<>();
diffs = new HashMap<>();
upserts = new HashMap<>();
int deletesCount = in.readVInt();
for (int i = 0; i < deletesCount; i++) {
deletes.add(in.readString());
deletes.add(keySerializer.readKey(in));
}
int diffsCount = in.readVInt();
for (int i = 0; i < diffsCount; i++) {
String key = in.readString();
Diff<T> diff = reader.readDiffFrom(in, key);
K key = keySerializer.readKey(in);
Diff<T> diff = valueSerializer.readDiff(in, key);
diffs.put(key, diff);
}
int addsCount = in.readVInt();
for (int i = 0; i < addsCount; i++) {
String key = in.readString();
T part = reader.readFrom(in, key);
adds.put(key, part);
int upsertsCount = in.readVInt();
for (int i = 0; i < upsertsCount; i++) {
K key = keySerializer.readKey(in);
T newValue = valueSerializer.read(in, key);
upserts.put(key, newValue);
}
}
/**
* The keys that, when this diff is applied to a map, should be removed from the map.
*
* @return the list of keys that are deleted
*/
public List<K> getDeletes() {
return deletes;
}
/**
* Map entries that, when this diff is applied to a map, should be
* incrementally updated. The incremental update is represented using
* the {@link Diff} interface.
*
* @return the map entries that are incrementally updated
*/
public Map<K, Diff<T>> getDiffs() {
return diffs;
}
/**
* Map entries that, when this diff is applied to a map, should be
* added to the map or fully replace the previous value.
*
* @return the map entries that are additions or full updates
*/
public Map<K, T> getUpserts() {
return upserts;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(deletes.size());
for (String delete : deletes) {
out.writeString(delete);
for (K delete : deletes) {
keySerializer.writeKey(delete, out);
}
out.writeVInt(diffs.size());
for (Map.Entry<String, Diff<T>> entry : diffs.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);
for (Map.Entry<K, Diff<T>> entry : diffs.entrySet()) {
keySerializer.writeKey(entry.getKey(), out);
valueSerializer.writeDiff(entry.getValue(), out);
}
out.writeVInt(adds.size());
for (Map.Entry<String, T> entry : adds.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);
out.writeVInt(upserts.size());
for (Map.Entry<K, T> entry : upserts.entrySet()) {
keySerializer.writeKey(entry.getKey(), out);
valueSerializer.write(entry.getValue(), out);
}
}
}
/**
* Provides read and write operations to serialize keys of map
* @param <K> type of key
*/
public interface KeySerializer<K> {
void writeKey(K key, StreamOutput out) throws IOException;
K readKey(StreamInput in) throws IOException;
}
/**
* Serializes String keys of a map
*/
private static final class StringKeySerializer implements KeySerializer<String> {
private static final StringKeySerializer INSTANCE = new StringKeySerializer();
@Override
public void writeKey(String key, StreamOutput out) throws IOException {
out.writeString(key);
}
@Override
public String readKey(StreamInput in) throws IOException {
return in.readString();
}
}
/**
* Serializes Integer keys of a map as an Int
*/
private static final class IntKeySerializer implements KeySerializer<Integer> {
public static final IntKeySerializer INSTANCE = new IntKeySerializer();
@Override
public void writeKey(Integer key, StreamOutput out) throws IOException {
out.writeInt(key);
}
@Override
public Integer readKey(StreamInput in) throws IOException {
return in.readInt();
}
}
/**
* Serializes Integer keys of a map as a VInt. Requires keys to be positive.
*/
private static final class VIntKeySerializer implements KeySerializer<Integer> {
public static final IntKeySerializer INSTANCE = new IntKeySerializer();
@Override
public void writeKey(Integer key, StreamOutput out) throws IOException {
if (key < 0) {
throw new IllegalArgumentException("Map key [" + key + "] must be positive");
}
out.writeVInt(key);
}
@Override
public Integer readKey(StreamInput in) throws IOException {
return in.readVInt();
}
}
/**
* Provides read and write operations to serialize map values.
* Reading of values can be made dependent on map key.
*
* Also provides operations to distinguish whether map values are diffable.
*
* Should not be directly implemented, instead implement either
* {@link DiffableValueSerializer} or {@link NonDiffableValueSerializer}.
*
* @param <K> key type of map
* @param <V> value type of map
*/
public interface ValueSerializer<K, V> {
/**
* Writes value to stream
*/
void write(V value, StreamOutput out) throws IOException;
/**
* Reads value from stream. Reading operation can be made dependent on map key.
*/
V read(StreamInput in, K key) throws IOException;
/**
* Whether this serializer supports diffable values
*/
boolean supportsDiffableValues();
/**
* Computes diff if this serializer supports diffable values
*/
Diff<V> diff(V value, V beforePart);
/**
* Writes value as diff to stream if this serializer supports diffable values
*/
void writeDiff(Diff<V> value, StreamOutput out) throws IOException;
/**
* Reads value as diff from stream if this serializer supports diffable values.
* Reading operation can be made dependent on map key.
*/
Diff<V> readDiff(StreamInput in, K key) throws IOException;
}
/**
* Serializer for Diffable map values. Needs to implement read and readDiff methods.
*
* @param <K> type of map keys
* @param <V> type of map values
*/
public static abstract class DiffableValueSerializer<K, V extends Diffable<V>> implements ValueSerializer<K, V> {
private static final DiffableValueSerializer WRITE_ONLY_INSTANCE = new DiffableValueSerializer() {
@Override
public Object read(StreamInput in, Object key) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public Diff<Object> readDiff(StreamInput in, Object key) throws IOException {
throw new UnsupportedOperationException();
}
};
private static <K, V extends Diffable<V>> DiffableValueSerializer<K, V> getWriteOnlyInstance() {
return WRITE_ONLY_INSTANCE;
}
@Override
public boolean supportsDiffableValues() {
return true;
}
@Override
public Diff<V> diff(V value, V beforePart) {
return value.diff(beforePart);
}
@Override
public void write(V value, StreamOutput out) throws IOException {
value.writeTo(out);
}
public void writeDiff(Diff<V> value, StreamOutput out) throws IOException {
value.writeTo(out);
}
}
/**
* Serializer for non-diffable map values
*
* @param <K> type of map keys
* @param <V> type of map values
*/
public static abstract class NonDiffableValueSerializer<K, V> implements ValueSerializer<K, V> {
@Override
public boolean supportsDiffableValues() {
return false;
}
@Override
public Diff<V> diff(V value, V beforePart) {
throw new UnsupportedOperationException();
}
@Override
public void writeDiff(Diff<V> value, StreamOutput out) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public Diff<V> readDiff(StreamInput in, K key) throws IOException {
throw new UnsupportedOperationException();
}
}
/**
* Implementation of the ValueSerializer that uses a prototype object for reading operations
*
* Note: this implementation is ignoring the key.
*/
public static class DiffablePrototypeValueReader<K, V extends Diffable<V>> extends DiffableValueSerializer<K, V> {
private final V proto;
public DiffablePrototypeValueReader(V proto) {
this.proto = proto;
}
@Override
public V read(StreamInput in, K key) throws IOException {
return proto.readFrom(in);
}
@Override
public Diff<V> readDiff(StreamInput in, K key) throws IOException {
return proto.readDiffFrom(in);
}
}
/**
* Implementation of ValueSerializer that serializes immutable sets
*
* @param <K> type of map key
*/
public static class StringSetValueSerializer<K> extends NonDiffableValueSerializer<K, Set<String>> {
private static final StringSetValueSerializer INSTANCE = new StringSetValueSerializer();
public static <K> StringSetValueSerializer<K> getInstance() {
return INSTANCE;
}
@Override
public void write(Set<String> value, StreamOutput out) throws IOException {
out.writeStringArray(value.toArray(new String[value.size()]));
}
@Override
public Set<String> read(StreamInput in, K key) throws IOException {
return Collections.unmodifiableSet(new HashSet<>(Arrays.asList(in.readStringArray())));
}
}
}

@ -19,6 +19,7 @@
package org.elasticsearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
@ -30,6 +31,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.node.DiscoveryNodeFilters;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.compress.CompressedXContent;
@ -46,10 +48,13 @@ import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.text.ParseException;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.cluster.node.DiscoveryNodeFilters.OpType.AND;
import static org.elasticsearch.cluster.node.DiscoveryNodeFilters.OpType.OR;
@ -168,6 +173,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
public static final String SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE = "index.shared_filesystem.recover_on_any_node";
public static final String INDEX_UUID_NA_VALUE = "_na_";
public static final String KEY_ACTIVE_ALLOCATIONS = "active_allocations";
private final int numberOfShards;
private final int numberOfReplicas;
@ -184,6 +191,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
private final ImmutableOpenMap<String, Custom> customs;
private final ImmutableOpenIntMap<Set<String>> activeAllocationIds;
private transient final int totalNumberOfShards;
private final DiscoveryNodeFilters requireFilters;
@ -194,65 +203,29 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
private final Version indexUpgradedVersion;
private final org.apache.lucene.util.Version minimumCompatibleLuceneVersion;
private IndexMetaData(String index, long version, State state, Settings settings, ImmutableOpenMap<String, MappingMetaData> mappings, ImmutableOpenMap<String, AliasMetaData> aliases, ImmutableOpenMap<String, Custom> customs) {
Integer maybeNumberOfShards = settings.getAsInt(SETTING_NUMBER_OF_SHARDS, null);
if (maybeNumberOfShards == null) {
throw new IllegalArgumentException("must specify numberOfShards for index [" + index + "]");
}
int numberOfShards = maybeNumberOfShards;
if (numberOfShards <= 0) {
throw new IllegalArgumentException("must specify positive number of shards for index [" + index + "]");
}
private IndexMetaData(String index, long version, State state, int numberOfShards, int numberOfReplicas, Settings settings,
ImmutableOpenMap<String, MappingMetaData> mappings, ImmutableOpenMap<String, AliasMetaData> aliases,
ImmutableOpenMap<String, Custom> customs, ImmutableOpenIntMap<Set<String>> activeAllocationIds,
DiscoveryNodeFilters requireFilters, DiscoveryNodeFilters includeFilters, DiscoveryNodeFilters excludeFilters,
Version indexCreatedVersion, Version indexUpgradedVersion, org.apache.lucene.util.Version minimumCompatibleLuceneVersion) {
Integer maybeNumberOfReplicas = settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, null);
if (maybeNumberOfReplicas == null) {
throw new IllegalArgumentException("must specify numberOfReplicas for index [" + index + "]");
}
int numberOfReplicas = maybeNumberOfReplicas;
if (numberOfReplicas < 0) {
throw new IllegalArgumentException("must specify non-negative number of shards for index [" + index + "]");
}
this.index = index;
this.version = version;
this.state = state;
this.settings = settings;
this.mappings = mappings;
this.customs = customs;
this.numberOfShards = numberOfShards;
this.numberOfReplicas = numberOfReplicas;
this.totalNumberOfShards = numberOfShards * (numberOfReplicas + 1);
this.settings = settings;
this.mappings = mappings;
this.customs = customs;
this.aliases = aliases;
Map<String, String> requireMap = settings.getByPrefix("index.routing.allocation.require.").getAsMap();
if (requireMap.isEmpty()) {
requireFilters = null;
} else {
requireFilters = DiscoveryNodeFilters.buildFromKeyValue(AND, requireMap);
}
Map<String, String> includeMap = settings.getByPrefix("index.routing.allocation.include.").getAsMap();
if (includeMap.isEmpty()) {
includeFilters = null;
} else {
includeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, includeMap);
}
Map<String, String> excludeMap = settings.getByPrefix("index.routing.allocation.exclude.").getAsMap();
if (excludeMap.isEmpty()) {
excludeFilters = null;
} else {
excludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, excludeMap);
}
indexCreatedVersion = Version.indexCreated(settings);
indexUpgradedVersion = settings.getAsVersion(IndexMetaData.SETTING_VERSION_UPGRADED, indexCreatedVersion);
String stringLuceneVersion = settings.get(SETTING_VERSION_MINIMUM_COMPATIBLE);
if (stringLuceneVersion != null) {
try {
this.minimumCompatibleLuceneVersion = org.apache.lucene.util.Version.parse(stringLuceneVersion);
} catch (ParseException ex) {
throw new IllegalStateException("Cannot parse lucene version [" + stringLuceneVersion + "] in the [" + SETTING_VERSION_MINIMUM_COMPATIBLE +"] setting", ex);
}
} else {
this.minimumCompatibleLuceneVersion = null;
}
this.activeAllocationIds = activeAllocationIds;
this.requireFilters = requireFilters;
this.includeFilters = includeFilters;
this.excludeFilters = excludeFilters;
this.indexCreatedVersion = indexCreatedVersion;
this.indexUpgradedVersion = indexUpgradedVersion;
this.minimumCompatibleLuceneVersion = minimumCompatibleLuceneVersion;
}
public String getIndex() {
@ -364,6 +337,15 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
return (T) customs.get(type);
}
public ImmutableOpenIntMap<Set<String>> getActiveAllocationIds() {
return activeAllocationIds;
}
public Set<String> activeAllocationIds(int shardId) {
assert shardId >= 0 && shardId < numberOfShards;
return activeAllocationIds.get(shardId);
}
@Nullable
public DiscoveryNodeFilters requireFilters() {
return requireFilters;
@ -408,6 +390,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
if (!customs.equals(that.customs)) {
return false;
}
if (!activeAllocationIds.equals(that.activeAllocationIds)) {
return false;
}
return true;
}
@ -418,6 +403,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
result = 31 * result + aliases.hashCode();
result = 31 * result + settings.hashCode();
result = 31 * result + mappings.hashCode();
result = 31 * result + activeAllocationIds.hashCode();
return result;
}
@ -450,16 +436,19 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
private final Settings settings;
private final Diff<ImmutableOpenMap<String, MappingMetaData>> mappings;
private final Diff<ImmutableOpenMap<String, AliasMetaData>> aliases;
private Diff<ImmutableOpenMap<String, Custom>> customs;
private final Diff<ImmutableOpenMap<String, Custom>> customs;
private final Diff<ImmutableOpenIntMap<Set<String>>> activeAllocationIds;
public IndexMetaDataDiff(IndexMetaData before, IndexMetaData after) {
index = after.index;
version = after.version;
state = after.state;
settings = after.settings;
mappings = DiffableUtils.diff(before.mappings, after.mappings);
aliases = DiffableUtils.diff(before.aliases, after.aliases);
customs = DiffableUtils.diff(before.customs, after.customs);
mappings = DiffableUtils.diff(before.mappings, after.mappings, DiffableUtils.getStringKeySerializer());
aliases = DiffableUtils.diff(before.aliases, after.aliases, DiffableUtils.getStringKeySerializer());
customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer());
activeAllocationIds = DiffableUtils.diff(before.activeAllocationIds, after.activeAllocationIds,
DiffableUtils.getVIntKeySerializer(), DiffableUtils.StringSetValueSerializer.getInstance());
}
public IndexMetaDataDiff(StreamInput in) throws IOException {
@ -467,19 +456,22 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
version = in.readLong();
state = State.fromId(in.readByte());
settings = Settings.readSettingsFromStream(in);
mappings = DiffableUtils.readImmutableOpenMapDiff(in, MappingMetaData.PROTO);
aliases = DiffableUtils.readImmutableOpenMapDiff(in, AliasMetaData.PROTO);
customs = DiffableUtils.readImmutableOpenMapDiff(in, new DiffableUtils.KeyedReader<Custom>() {
mappings = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), MappingMetaData.PROTO);
aliases = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), AliasMetaData.PROTO);
customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
new DiffableUtils.DiffableValueSerializer<String, Custom>() {
@Override
public Custom readFrom(StreamInput in, String key) throws IOException {
public Custom read(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readFrom(in);
}
@Override
public Diff<Custom> readDiffFrom(StreamInput in, String key) throws IOException {
public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readDiffFrom(in);
}
});
activeAllocationIds = DiffableUtils.readImmutableOpenIntMapDiff(in, DiffableUtils.getVIntKeySerializer(),
DiffableUtils.StringSetValueSerializer.getInstance());
}
@Override
@ -491,6 +483,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
mappings.writeTo(out);
aliases.writeTo(out);
customs.writeTo(out);
activeAllocationIds.writeTo(out);
}
@Override
@ -502,6 +495,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
builder.mappings.putAll(mappings.apply(part.mappings));
builder.aliases.putAll(aliases.apply(part.aliases));
builder.customs.putAll(customs.apply(part.customs));
builder.activeAllocationIds.putAll(activeAllocationIds.apply(part.activeAllocationIds));
return builder.build();
}
}
@ -528,6 +522,12 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
Custom customIndexMetaData = lookupPrototypeSafe(type).readFrom(in);
builder.putCustom(type, customIndexMetaData);
}
int activeAllocationIdsSize = in.readVInt();
for (int i = 0; i < activeAllocationIdsSize; i++) {
int key = in.readVInt();
Set<String> allocationIds = DiffableUtils.StringSetValueSerializer.getInstance().read(in, key);
builder.putActiveAllocationIds(key, allocationIds);
}
return builder.build();
}
@ -550,6 +550,11 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
out.writeString(cursor.key);
cursor.value.writeTo(out);
}
out.writeVInt(activeAllocationIds.size());
for (IntObjectCursor<Set<String>> cursor : activeAllocationIds) {
out.writeVInt(cursor.key);
DiffableUtils.StringSetValueSerializer.getInstance().write(cursor.value, out);
}
}
public static Builder builder(String index) {
@ -569,12 +574,14 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
private final ImmutableOpenMap.Builder<String, MappingMetaData> mappings;
private final ImmutableOpenMap.Builder<String, AliasMetaData> aliases;
private final ImmutableOpenMap.Builder<String, Custom> customs;
private final ImmutableOpenIntMap.Builder<Set<String>> activeAllocationIds;
public Builder(String index) {
this.index = index;
this.mappings = ImmutableOpenMap.builder();
this.aliases = ImmutableOpenMap.builder();
this.customs = ImmutableOpenMap.builder();
this.activeAllocationIds = ImmutableOpenIntMap.builder();
}
public Builder(IndexMetaData indexMetaData) {
@ -585,6 +592,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
this.mappings = ImmutableOpenMap.builder(indexMetaData.mappings);
this.aliases = ImmutableOpenMap.builder(indexMetaData.aliases);
this.customs = ImmutableOpenMap.builder(indexMetaData.customs);
this.activeAllocationIds = ImmutableOpenIntMap.builder(indexMetaData.activeAllocationIds);
}
public String index() {
@ -693,6 +701,15 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
return this.customs.get(type);
}
public Builder putActiveAllocationIds(int shardId, Set<String> allocationIds) {
activeAllocationIds.put(shardId, new HashSet(allocationIds));
return this;
}
public Set<String> getActiveAllocationIds(int shardId) {
return activeAllocationIds.get(shardId);
}
public long version() {
return this.version;
}
@ -714,7 +731,72 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
}
}
return new IndexMetaData(index, version, state, tmpSettings, mappings.build(), tmpAliases.build(), customs.build());
Integer maybeNumberOfShards = settings.getAsInt(SETTING_NUMBER_OF_SHARDS, null);
if (maybeNumberOfShards == null) {
throw new IllegalArgumentException("must specify numberOfShards for index [" + index + "]");
}
int numberOfShards = maybeNumberOfShards;
if (numberOfShards <= 0) {
throw new IllegalArgumentException("must specify positive number of shards for index [" + index + "]");
}
Integer maybeNumberOfReplicas = settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, null);
if (maybeNumberOfReplicas == null) {
throw new IllegalArgumentException("must specify numberOfReplicas for index [" + index + "]");
}
int numberOfReplicas = maybeNumberOfReplicas;
if (numberOfReplicas < 0) {
throw new IllegalArgumentException("must specify non-negative number of shards for index [" + index + "]");
}
// fill missing slots in activeAllocationIds with empty set if needed and make all entries immutable
ImmutableOpenIntMap.Builder<Set<String>> filledActiveAllocationIds = ImmutableOpenIntMap.builder();
for (int i = 0; i < numberOfShards; i++) {
if (activeAllocationIds.containsKey(i)) {
filledActiveAllocationIds.put(i, Collections.unmodifiableSet(new HashSet<>(activeAllocationIds.get(i))));
} else {
filledActiveAllocationIds.put(i, Collections.emptySet());
}
}
Map<String, String> requireMap = settings.getByPrefix("index.routing.allocation.require.").getAsMap();
final DiscoveryNodeFilters requireFilters;
if (requireMap.isEmpty()) {
requireFilters = null;
} else {
requireFilters = DiscoveryNodeFilters.buildFromKeyValue(AND, requireMap);
}
Map<String, String> includeMap = settings.getByPrefix("index.routing.allocation.include.").getAsMap();
final DiscoveryNodeFilters includeFilters;
if (includeMap.isEmpty()) {
includeFilters = null;
} else {
includeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, includeMap);
}
Map<String, String> excludeMap = settings.getByPrefix("index.routing.allocation.exclude.").getAsMap();
final DiscoveryNodeFilters excludeFilters;
if (excludeMap.isEmpty()) {
excludeFilters = null;
} else {
excludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, excludeMap);
}
Version indexCreatedVersion = Version.indexCreated(settings);
Version indexUpgradedVersion = settings.getAsVersion(IndexMetaData.SETTING_VERSION_UPGRADED, indexCreatedVersion);
String stringLuceneVersion = settings.get(SETTING_VERSION_MINIMUM_COMPATIBLE);
final org.apache.lucene.util.Version minimumCompatibleLuceneVersion;
if (stringLuceneVersion != null) {
try {
minimumCompatibleLuceneVersion = org.apache.lucene.util.Version.parse(stringLuceneVersion);
} catch (ParseException ex) {
throw new IllegalStateException("Cannot parse lucene version [" + stringLuceneVersion + "] in the [" + SETTING_VERSION_MINIMUM_COMPATIBLE +"] setting", ex);
}
} else {
minimumCompatibleLuceneVersion = null;
}
return new IndexMetaData(index, version, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(),
tmpAliases.build(), customs.build(), filledActiveAllocationIds.build(), requireFilters, includeFilters, excludeFilters,
indexCreatedVersion, indexUpgradedVersion, minimumCompatibleLuceneVersion);
}
public static void toXContent(IndexMetaData indexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException {
@ -757,6 +839,15 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
}
builder.endObject();
builder.startObject(KEY_ACTIVE_ALLOCATIONS);
for (IntObjectCursor<Set<String>> cursor : indexMetaData.activeAllocationIds) {
builder.startArray(String.valueOf(cursor.key));
for (String allocationId : cursor.value) {
builder.value(allocationId);
}
builder.endArray();
}
builder.endObject();
builder.endObject();
}
@ -792,6 +883,21 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
while (parser.nextToken() != XContentParser.Token.END_OBJECT) {
builder.putAlias(AliasMetaData.Builder.fromXContent(parser));
}
} else if (KEY_ACTIVE_ALLOCATIONS.equals(currentFieldName)) {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.START_ARRAY) {
String shardId = currentFieldName;
Set<String> allocationIds = new HashSet<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.VALUE_STRING) {
allocationIds.add(parser.text());
}
}
builder.putActiveAllocationIds(Integer.valueOf(shardId), allocationIds);
}
}
} else {
// check if its a custom index metadata
Custom proto = lookupPrototype(currentFieldName);

@ -27,7 +27,6 @@ import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.Diffable;
import org.elasticsearch.cluster.DiffableUtils;
import org.elasticsearch.cluster.DiffableUtils.KeyedReader;
import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
@ -55,7 +54,6 @@ import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.warmer.IndexWarmersMetaData;
@ -641,9 +639,9 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
version = after.version;
transientSettings = after.transientSettings;
persistentSettings = after.persistentSettings;
indices = DiffableUtils.diff(before.indices, after.indices);
templates = DiffableUtils.diff(before.templates, after.templates);
customs = DiffableUtils.diff(before.customs, after.customs);
indices = DiffableUtils.diff(before.indices, after.indices, DiffableUtils.getStringKeySerializer());
templates = DiffableUtils.diff(before.templates, after.templates, DiffableUtils.getStringKeySerializer());
customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer());
}
public MetaDataDiff(StreamInput in) throws IOException {
@ -651,16 +649,17 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
version = in.readLong();
transientSettings = Settings.readSettingsFromStream(in);
persistentSettings = Settings.readSettingsFromStream(in);
indices = DiffableUtils.readImmutableOpenMapDiff(in, IndexMetaData.PROTO);
templates = DiffableUtils.readImmutableOpenMapDiff(in, IndexTemplateMetaData.PROTO);
customs = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader<Custom>() {
indices = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexMetaData.PROTO);
templates = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexTemplateMetaData.PROTO);
customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
new DiffableUtils.DiffableValueSerializer<String, Custom>() {
@Override
public Custom readFrom(StreamInput in, String key) throws IOException {
public Custom read(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readFrom(in);
}
@Override
public Diff<Custom> readDiffFrom(StreamInput in, String key) throws IOException {
public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readDiffFrom(in);
}
});

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InvalidTypeNameException;
@ -255,7 +254,7 @@ public class MetaDataMappingService extends AbstractComponent {
MergeResult mergeResult = existingMapper.merge(newMapper.mapping(), true, request.updateAllTypes());
// if we have conflicts, throw an exception
if (mergeResult.hasConflicts()) {
throw new MergeMappingException(mergeResult.buildConflicts());
throw new IllegalArgumentException("Merge failed with failures {" + Arrays.toString(mergeResult.buildConflicts()) + "}");
}
} else {
// TODO: can we find a better place for this validation?

@ -314,12 +314,12 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
public RoutingTableDiff(RoutingTable before, RoutingTable after) {
version = after.version;
indicesRouting = DiffableUtils.diff(before.indicesRouting, after.indicesRouting);
indicesRouting = DiffableUtils.diff(before.indicesRouting, after.indicesRouting, DiffableUtils.getStringKeySerializer());
}
public RoutingTableDiff(StreamInput in) throws IOException {
version = in.readLong();
indicesRouting = DiffableUtils.readImmutableOpenMapDiff(in, IndexRoutingTable.PROTO);
indicesRouting = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexRoutingTable.PROTO);
}
@Override

@ -27,7 +27,14 @@ import org.elasticsearch.cluster.health.ClusterStateHealth;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.AllocationId;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
@ -39,6 +46,8 @@ import org.elasticsearch.common.settings.Settings;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
@ -79,24 +88,83 @@ public class AllocationService extends AbstractComponent {
StartedRerouteAllocation allocation = new StartedRerouteAllocation(allocationDeciders, routingNodes, clusterState.nodes(), startedShards, clusterInfoService.getClusterInfo());
boolean changed = applyStartedShards(routingNodes, startedShards);
if (!changed) {
return new RoutingAllocation.Result(false, clusterState.routingTable());
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
shardsAllocators.applyStartedShards(allocation);
if (withReroute) {
reroute(allocation);
}
RoutingTable routingTable = new RoutingTable.Builder().updateNodes(routingNodes).build().validateRaiseException(clusterState.metaData());
RoutingAllocation.Result result = new RoutingAllocation.Result(true, routingTable);
final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
String startedShardsAsString = firstListElementsToCommaDelimitedString(startedShards, s -> s.shardId().toString());
logClusterHealthStateChange(
new ClusterStateHealth(clusterState),
new ClusterStateHealth(clusterState.metaData(), routingTable),
new ClusterStateHealth(clusterState.metaData(), result.routingTable()),
"shards started [" + startedShardsAsString + "] ..."
);
return result;
}
protected RoutingAllocation.Result buildChangedResult(MetaData metaData, RoutingNodes routingNodes) {
return buildChangedResult(metaData, routingNodes, new RoutingExplanations());
}
protected RoutingAllocation.Result buildChangedResult(MetaData metaData, RoutingNodes routingNodes, RoutingExplanations explanations) {
final RoutingTable routingTable = new RoutingTable.Builder().updateNodes(routingNodes).build();
MetaData newMetaData = updateMetaDataWithRoutingTable(metaData,routingTable);
return new RoutingAllocation.Result(true, routingTable.validateRaiseException(newMetaData), newMetaData, explanations);
}
/**
* Updates the current {@link MetaData} based on the newly created {@link RoutingTable}.
*
* @param currentMetaData {@link MetaData} object from before the routing table was changed.
* @param newRoutingTable new {@link RoutingTable} created by the allocation change
* @return adapted {@link MetaData}, potentially the original one if no change was needed.
*/
static MetaData updateMetaDataWithRoutingTable(MetaData currentMetaData, RoutingTable newRoutingTable) {
// make sure index meta data and routing tables are in sync w.r.t active allocation ids
MetaData.Builder metaDataBuilder = null;
for (IndexRoutingTable indexRoutingTable : newRoutingTable) {
final IndexMetaData indexMetaData = currentMetaData.index(indexRoutingTable.getIndex());
if (indexMetaData == null) {
throw new IllegalStateException("no metadata found for index [" + indexRoutingTable.index() + "]");
}
IndexMetaData.Builder indexMetaDataBuilder = null;
for (IndexShardRoutingTable shardRoutings : indexRoutingTable) {
Set<String> activeAllocationIds = shardRoutings.activeShards().stream()
.map(ShardRouting::allocationId)
.filter(Objects::nonNull)
.map(AllocationId::getId)
.collect(Collectors.toSet());
// only update active allocation ids if there is an active shard
if (activeAllocationIds.isEmpty() == false) {
// get currently stored allocation ids
Set<String> storedAllocationIds = indexMetaData.activeAllocationIds(shardRoutings.shardId().id());
if (activeAllocationIds.equals(storedAllocationIds) == false) {
if (indexMetaDataBuilder == null) {
indexMetaDataBuilder = IndexMetaData.builder(indexMetaData);
}
indexMetaDataBuilder.putActiveAllocationIds(shardRoutings.shardId().id(), activeAllocationIds);
}
}
}
if (indexMetaDataBuilder != null) {
if (metaDataBuilder == null) {
metaDataBuilder = MetaData.builder(currentMetaData);
}
metaDataBuilder.put(indexMetaDataBuilder);
}
}
if (metaDataBuilder != null) {
return metaDataBuilder.build();
} else {
return currentMetaData;
}
}
public RoutingAllocation.Result applyFailedShard(ClusterState clusterState, ShardRouting failedShard) {
return applyFailedShards(clusterState, Collections.singletonList(new FailedRerouteAllocation.FailedShard(failedShard, null, null)));
}
@ -117,16 +185,15 @@ public class AllocationService extends AbstractComponent {
System.nanoTime(), System.currentTimeMillis()));
}
if (!changed) {
return new RoutingAllocation.Result(false, clusterState.routingTable());
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
shardsAllocators.applyFailedShards(allocation);
reroute(allocation);
RoutingTable routingTable = new RoutingTable.Builder().updateNodes(routingNodes).build().validateRaiseException(clusterState.metaData());
RoutingAllocation.Result result = new RoutingAllocation.Result(true, routingTable);
final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.shard.shardId().toString());
logClusterHealthStateChange(
new ClusterStateHealth(clusterState),
new ClusterStateHealth(clusterState.getMetaData(), routingTable),
new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()),
"shards failed [" + failedShardsAsString + "] ..."
);
return result;
@ -169,11 +236,10 @@ public class AllocationService extends AbstractComponent {
// the assumption is that commands will move / act on shards (or fail through exceptions)
// so, there will always be shard "movements", so no need to check on reroute
reroute(allocation);
RoutingTable routingTable = new RoutingTable.Builder().updateNodes(routingNodes).build().validateRaiseException(clusterState.metaData());
RoutingAllocation.Result result = new RoutingAllocation.Result(true, routingTable, explanations);
RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes, explanations);
logClusterHealthStateChange(
new ClusterStateHealth(clusterState),
new ClusterStateHealth(clusterState.getMetaData(), routingTable),
new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()),
"reroute commands"
);
return result;
@ -200,13 +266,12 @@ public class AllocationService extends AbstractComponent {
RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState.nodes(), clusterInfoService.getClusterInfo(), currentNanoTime());
allocation.debugDecision(debug);
if (!reroute(allocation)) {
return new RoutingAllocation.Result(false, clusterState.routingTable());
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
RoutingTable routingTable = new RoutingTable.Builder().updateNodes(routingNodes).build().validateRaiseException(clusterState.metaData());
RoutingAllocation.Result result = new RoutingAllocation.Result(true, routingTable);
RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
logClusterHealthStateChange(
new ClusterStateHealth(clusterState),
new ClusterStateHealth(clusterState.getMetaData(), routingTable),
new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()),
reason
);
return result;

@ -52,29 +52,33 @@ public class RoutingAllocation {
private final RoutingTable routingTable;
private final MetaData metaData;
private RoutingExplanations explanations = new RoutingExplanations();
/**
* Creates a new {@link RoutingAllocation.Result}
*
* @param changed a flag to determine whether the actual {@link RoutingTable} has been changed
* @param routingTable the {@link RoutingTable} this Result references
* @param metaData the {@link MetaData} this Result references
*/
public Result(boolean changed, RoutingTable routingTable) {
public Result(boolean changed, RoutingTable routingTable, MetaData metaData) {
this.changed = changed;
this.routingTable = routingTable;
this.metaData = metaData;
}
/**
* Creates a new {@link RoutingAllocation.Result}
*
* @param changed a flag to determine whether the actual {@link RoutingTable} has been changed
* @param routingTable the {@link RoutingTable} this Result references
* @param metaData the {@link MetaData} this Result references
* @param explanations Explanation for the reroute actions
*/
public Result(boolean changed, RoutingTable routingTable, RoutingExplanations explanations) {
public Result(boolean changed, RoutingTable routingTable, MetaData metaData, RoutingExplanations explanations) {
this.changed = changed;
this.routingTable = routingTable;
this.metaData = metaData;
this.explanations = explanations;
}
@ -85,6 +89,14 @@ public class RoutingAllocation {
return this.changed;
}
/**
* Get the {@link MetaData} referenced by this result
* @return referenced {@link MetaData}
*/
public MetaData metaData() {
return metaData;
}
/**
* Get the {@link RoutingTable} referenced by this result
* @return referenced {@link RoutingTable}

@ -34,8 +34,6 @@ public class LineStringBuilder extends PointCollection<LineStringBuilder> {
public static final GeoShapeType TYPE = GeoShapeType.LINESTRING;
protected boolean translated = false;
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();

@ -89,6 +89,4 @@ public class MultiPolygonBuilder extends ShapeBuilder {
return new XShapeCollection<>(shapes, SPATIAL_CONTEXT);
//note: ShapeCollection is probably faster than a Multi* geom.
}
}

@ -38,6 +38,7 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* The {@link PolygonBuilder} implements the groundwork to create polygons. This contains
@ -141,9 +142,10 @@ public class PolygonBuilder extends ShapeBuilder {
Edge[] edges = new Edge[numEdges];
Edge[] holeComponents = new Edge[holes.size()];
int offset = createEdges(0, orientation, shell, null, edges, 0);
final AtomicBoolean translated = new AtomicBoolean(false);
int offset = createEdges(0, orientation, shell, null, edges, 0, translated);
for (int i = 0; i < holes.size(); i++) {
int length = createEdges(i+1, orientation, shell, this.holes.get(i), edges, offset);
int length = createEdges(i+1, orientation, shell, this.holes.get(i), edges, offset, translated);
holeComponents[i] = edges[offset];
offset += length;
}
@ -508,14 +510,157 @@ public class PolygonBuilder extends ShapeBuilder {
}
private static int createEdges(int component, Orientation orientation, LineStringBuilder shell,
LineStringBuilder hole,
Edge[] edges, int offset) {
LineStringBuilder hole, Edge[] edges, int offset, final AtomicBoolean translated) {
// inner rings (holes) have an opposite direction than the outer rings
// XOR will invert the orientation for outer ring cases (Truth Table:, T/T = F, T/F = T, F/T = T, F/F = F)
boolean direction = (component == 0 ^ orientation == Orientation.RIGHT);
// set the points array accordingly (shell or hole)
Coordinate[] points = (hole != null) ? hole.coordinates(false) : shell.coordinates(false);
Edge.ring(component, direction, orientation == Orientation.LEFT, shell, points, 0, edges, offset, points.length-1);
ring(component, direction, orientation == Orientation.LEFT, shell, points, 0, edges, offset, points.length-1, translated);
return points.length-1;
}
/**
* Create a connected list of a list of coordinates
*
* @param points
* array of point
* @param offset
* index of the first point
* @param length
* number of points
* @return Array of edges
*/
private static Edge[] ring(int component, boolean direction, boolean handedness, LineStringBuilder shell,
Coordinate[] points, int offset, Edge[] edges, int toffset, int length, final AtomicBoolean translated) {
// calculate the direction of the points:
// find the point a the top of the set and check its
// neighbors orientation. So direction is equivalent
// to clockwise/counterclockwise
final int top = top(points, offset, length);
final int prev = (offset + ((top + length - 1) % length));
final int next = (offset + ((top + 1) % length));
boolean orientation = points[offset + prev].x > points[offset + next].x;
// OGC requires shell as ccw (Right-Handedness) and holes as cw (Left-Handedness)
// since GeoJSON doesn't specify (and doesn't need to) GEO core will assume OGC standards
// thus if orientation is computed as cw, the logic will translate points across dateline
// and convert to a right handed system
// compute the bounding box and calculate range
double[] range = range(points, offset, length);
final double rng = range[1] - range[0];
// translate the points if the following is true
// 1. shell orientation is cw and range is greater than a hemisphere (180 degrees) but not spanning 2 hemispheres
// (translation would result in a collapsed poly)
// 2. the shell of the candidate hole has been translated (to preserve the coordinate system)
boolean incorrectOrientation = component == 0 && handedness != orientation;
if ( (incorrectOrientation && (rng > DATELINE && rng != 2*DATELINE)) || (translated.get() && component != 0)) {
translate(points);
// flip the translation bit if the shell is being translated
if (component == 0) {
translated.set(true);
}
// correct the orientation post translation (ccw for shell, cw for holes)
if (component == 0 || (component != 0 && handedness == orientation)) {
orientation = !orientation;
}
}
return concat(component, direction ^ orientation, points, offset, edges, toffset, length);
}
private static final int top(Coordinate[] points, int offset, int length) {
int top = 0; // we start at 1 here since top points to 0
for (int i = 1; i < length; i++) {
if (points[offset + i].y < points[offset + top].y) {
top = i;
} else if (points[offset + i].y == points[offset + top].y) {
if (points[offset + i].x < points[offset + top].x) {
top = i;
}
}
}
return top;
}
private static final double[] range(Coordinate[] points, int offset, int length) {
double minX = points[0].x;
double maxX = points[0].x;
double minY = points[0].y;
double maxY = points[0].y;
// compute the bounding coordinates (@todo: cleanup brute force)
for (int i = 1; i < length; ++i) {
if (points[offset + i].x < minX) {
minX = points[offset + i].x;
}
if (points[offset + i].x > maxX) {
maxX = points[offset + i].x;
}
if (points[offset + i].y < minY) {
minY = points[offset + i].y;
}
if (points[offset + i].y > maxY) {
maxY = points[offset + i].y;
}
}
return new double[] {minX, maxX, minY, maxY};
}
/**
* Concatenate a set of points to a polygon
*
* @param component
* component id of the polygon
* @param direction
* direction of the ring
* @param points
* list of points to concatenate
* @param pointOffset
* index of the first point
* @param edges
* Array of edges to write the result to
* @param edgeOffset
* index of the first edge in the result
* @param length
* number of points to use
* @return the edges creates
*/
private static Edge[] concat(int component, boolean direction, Coordinate[] points, final int pointOffset, Edge[] edges, final int edgeOffset,
int length) {
assert edges.length >= length+edgeOffset;
assert points.length >= length+pointOffset;
edges[edgeOffset] = new Edge(points[pointOffset], null);
for (int i = 1; i < length; i++) {
if (direction) {
edges[edgeOffset + i] = new Edge(points[pointOffset + i], edges[edgeOffset + i - 1]);
edges[edgeOffset + i].component = component;
} else if(!edges[edgeOffset + i - 1].coordinate.equals(points[pointOffset + i])) {
edges[edgeOffset + i - 1].next = edges[edgeOffset + i] = new Edge(points[pointOffset + i], null);
edges[edgeOffset + i - 1].component = component;
} else {
throw new InvalidShapeException("Provided shape has duplicate consecutive coordinates at: " + points[pointOffset + i]);
}
}
if (direction) {
edges[edgeOffset].setNext(edges[edgeOffset + length - 1]);
edges[edgeOffset].component = component;
} else {
edges[edgeOffset + length - 1].setNext(edges[edgeOffset]);
edges[edgeOffset + length - 1].component = component;
}
return edges;
}
/**
* Transforms coordinates in the eastern hemisphere (-180:0) to a (180:360) range
*/
private static void translate(Coordinate[] points) {
for (Coordinate c : points) {
if (c.x < 0) {
c.x += 2*DATELINE;
}
}
}
}

@ -362,150 +362,6 @@ public abstract class ShapeBuilder extends ToXContentToBytes implements NamedWri
}
}
private static final int top(Coordinate[] points, int offset, int length) {
int top = 0; // we start at 1 here since top points to 0
for (int i = 1; i < length; i++) {
if (points[offset + i].y < points[offset + top].y) {
top = i;
} else if (points[offset + i].y == points[offset + top].y) {
if (points[offset + i].x < points[offset + top].x) {
top = i;
}
}
}
return top;
}
private static final double[] range(Coordinate[] points, int offset, int length) {
double minX = points[0].x;
double maxX = points[0].x;
double minY = points[0].y;
double maxY = points[0].y;
// compute the bounding coordinates (@todo: cleanup brute force)
for (int i = 1; i < length; ++i) {
if (points[offset + i].x < minX) {
minX = points[offset + i].x;
}
if (points[offset + i].x > maxX) {
maxX = points[offset + i].x;
}
if (points[offset + i].y < minY) {
minY = points[offset + i].y;
}
if (points[offset + i].y > maxY) {
maxY = points[offset + i].y;
}
}
return new double[] {minX, maxX, minY, maxY};
}
/**
* Concatenate a set of points to a polygon
*
* @param component
* component id of the polygon
* @param direction
* direction of the ring
* @param points
* list of points to concatenate
* @param pointOffset
* index of the first point
* @param edges
* Array of edges to write the result to
* @param edgeOffset
* index of the first edge in the result
* @param length
* number of points to use
* @return the edges creates
*/
private static Edge[] concat(int component, boolean direction, Coordinate[] points, final int pointOffset, Edge[] edges, final int edgeOffset,
int length) {
assert edges.length >= length+edgeOffset;
assert points.length >= length+pointOffset;
edges[edgeOffset] = new Edge(points[pointOffset], null);
for (int i = 1; i < length; i++) {
if (direction) {
edges[edgeOffset + i] = new Edge(points[pointOffset + i], edges[edgeOffset + i - 1]);
edges[edgeOffset + i].component = component;
} else if(!edges[edgeOffset + i - 1].coordinate.equals(points[pointOffset + i])) {
edges[edgeOffset + i - 1].next = edges[edgeOffset + i] = new Edge(points[pointOffset + i], null);
edges[edgeOffset + i - 1].component = component;
} else {
throw new InvalidShapeException("Provided shape has duplicate consecutive coordinates at: " + points[pointOffset + i]);
}
}
if (direction) {
edges[edgeOffset].setNext(edges[edgeOffset + length - 1]);
edges[edgeOffset].component = component;
} else {
edges[edgeOffset + length - 1].setNext(edges[edgeOffset]);
edges[edgeOffset + length - 1].component = component;
}
return edges;
}
/**
* Create a connected list of a list of coordinates
*
* @param points
* array of point
* @param offset
* index of the first point
* @param length
* number of points
* @return Array of edges
*/
protected static Edge[] ring(int component, boolean direction, boolean handedness, LineStringBuilder shell,
Coordinate[] points, int offset, Edge[] edges, int toffset, int length) {
// calculate the direction of the points:
// find the point a the top of the set and check its
// neighbors orientation. So direction is equivalent
// to clockwise/counterclockwise
final int top = top(points, offset, length);
final int prev = (offset + ((top + length - 1) % length));
final int next = (offset + ((top + 1) % length));
boolean orientation = points[offset + prev].x > points[offset + next].x;
// OGC requires shell as ccw (Right-Handedness) and holes as cw (Left-Handedness)
// since GeoJSON doesn't specify (and doesn't need to) GEO core will assume OGC standards
// thus if orientation is computed as cw, the logic will translate points across dateline
// and convert to a right handed system
// compute the bounding box and calculate range
double[] range = range(points, offset, length);
final double rng = range[1] - range[0];
// translate the points if the following is true
// 1. shell orientation is cw and range is greater than a hemisphere (180 degrees) but not spanning 2 hemispheres
// (translation would result in a collapsed poly)
// 2. the shell of the candidate hole has been translated (to preserve the coordinate system)
boolean incorrectOrientation = component == 0 && handedness != orientation;
if ( (incorrectOrientation && (rng > DATELINE && rng != 2*DATELINE)) || (shell.translated && component != 0)) {
translate(points);
// flip the translation bit if the shell is being translated
if (component == 0) {
shell.translated = true;
}
// correct the orientation post translation (ccw for shell, cw for holes)
if (component == 0 || (component != 0 && handedness == orientation)) {
orientation = !orientation;
}
}
return concat(component, direction ^ orientation, points, offset, edges, toffset, length);
}
/**
* Transforms coordinates in the eastern hemisphere (-180:0) to a (180:360) range
*/
protected static void translate(Coordinate[] points) {
for (Coordinate c : points) {
if (c.x < 0) {
c.x += 2*DATELINE;
}
}
}
/**
* Set the intersection of this line segment to the given position
*
@ -517,7 +373,7 @@ public abstract class ShapeBuilder extends ToXContentToBytes implements NamedWri
return intersect = position(coordinate, next.coordinate, position);
}
public static Coordinate position(Coordinate p1, Coordinate p2, double position) {
protected static Coordinate position(Coordinate p1, Coordinate p2, double position) {
if (position == 0) {
return p1;
} else if (position == 1) {
@ -542,7 +398,6 @@ public abstract class ShapeBuilder extends ToXContentToBytes implements NamedWri
public int compare(Edge o1, Edge o2) {
return Double.compare(o1.intersect.y, o2.intersect.y);
}
}
public static enum Orientation {

@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
@ -41,7 +42,7 @@ import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.new
public class NodesFaultDetection extends FaultDetection {
public static final String PING_ACTION_NAME = "internal:discovery/zen/fd/ping";
public abstract static class Listener {
public void onNodeFailure(DiscoveryNode node, String reason) {}
@ -145,14 +146,18 @@ public class NodesFaultDetection extends FaultDetection {
}
private void notifyNodeFailure(final DiscoveryNode node, final String reason) {
threadPool.generic().execute(new Runnable() {
@Override
public void run() {
for (Listener listener : listeners) {
listener.onNodeFailure(node, reason);
try {
threadPool.generic().execute(new Runnable() {
@Override
public void run() {
for (Listener listener : listeners) {
listener.onNodeFailure(node, reason);
}
}
}
});
});
} catch (EsRejectedExecutionException ex) {
logger.trace("[node ] [{}] ignoring node failure (reason [{}]). Local node is shutting down", ex, node, reason);
}
}
private void notifyPingReceived(final PingRequest pingRequest) {

@ -336,8 +336,6 @@ public class DocumentMapper implements ToXContent {
private void addMappers(Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers, boolean updateAllTypes) {
assert mappingLock.isWriteLockedByCurrentThread();
// first ensure we don't have any incompatible new fields
mapperService.checkNewMappersCompatibility(objectMappers, fieldMappers, updateAllTypes);
// update mappers for this document type
Map<String, ObjectMapper> builder = new HashMap<>(this.objectMappers);
@ -356,6 +354,7 @@ public class DocumentMapper implements ToXContent {
public MergeResult 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);
if (simulate == false) {

@ -307,7 +307,6 @@ public abstract class FieldMapper extends Mapper {
if (ref.get().equals(fieldType()) == false) {
throw new IllegalStateException("Cannot overwrite field type reference to unequal reference");
}
ref.incrementAssociatedMappers();
this.fieldTypeRef = ref;
}
@ -360,7 +359,7 @@ public abstract class FieldMapper extends Mapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
if (!this.getClass().equals(mergeWith.getClass())) {
String mergedType = mergeWith.getClass().getSimpleName();
if (mergeWith instanceof FieldMapper) {
@ -380,11 +379,6 @@ public abstract class FieldMapper extends Mapper {
return;
}
boolean strict = this.fieldTypeRef.getNumAssociatedMappers() > 1 && mergeResult.updateAllTypes() == false;
fieldType().checkCompatibility(fieldMergeWith.fieldType(), subConflicts, strict);
for (String conflict : subConflicts) {
mergeResult.addConflict(conflict);
}
multiFields.merge(mergeWith, mergeResult);
if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
@ -614,7 +608,7 @@ public abstract class FieldMapper extends Mapper {
}
// No need for locking, because locking is taken care of in ObjectMapper#merge and DocumentMapper#merge
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
FieldMapper mergeWithMultiField = (FieldMapper) mergeWith;
List<FieldMapper> newFieldMappers = null;

@ -24,6 +24,7 @@ import org.elasticsearch.common.regex.Regex;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
@ -38,18 +39,49 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
/** Full field name to field type */
private final CopyOnWriteHashMap<String, MappedFieldTypeReference> fullNameToFieldType;
/** Full field name to types containing a mapping for this full name. */
private final CopyOnWriteHashMap<String, Set<String>> fullNameToTypes;
/** Index field name to field type */
private final CopyOnWriteHashMap<String, MappedFieldTypeReference> indexNameToFieldType;
/** Index field name to types containing a mapping for this index name. */
private final CopyOnWriteHashMap<String, Set<String>> indexNameToTypes;
/** Create a new empty instance. */
public FieldTypeLookup() {
fullNameToFieldType = new CopyOnWriteHashMap<>();
fullNameToTypes = new CopyOnWriteHashMap<>();
indexNameToFieldType = new CopyOnWriteHashMap<>();
indexNameToTypes = new CopyOnWriteHashMap<>();
}
private FieldTypeLookup(CopyOnWriteHashMap<String, MappedFieldTypeReference> fullName, CopyOnWriteHashMap<String, MappedFieldTypeReference> indexName) {
fullNameToFieldType = fullName;
indexNameToFieldType = indexName;
private FieldTypeLookup(
CopyOnWriteHashMap<String, MappedFieldTypeReference> fullName,
CopyOnWriteHashMap<String, Set<String>> fullNameToTypes,
CopyOnWriteHashMap<String, MappedFieldTypeReference> indexName,
CopyOnWriteHashMap<String, Set<String>> indexNameToTypes) {
this.fullNameToFieldType = fullName;
this.fullNameToTypes = fullNameToTypes;
this.indexNameToFieldType = indexName;
this.indexNameToTypes = indexNameToTypes;
}
private static CopyOnWriteHashMap<String, Set<String>> addType(CopyOnWriteHashMap<String, Set<String>> map, String key, String type) {
Set<String> types = map.get(key);
if (types == null) {
return map.copyAndPut(key, Collections.singleton(type));
} else if (types.contains(type)) {
// noting to do
return map;
} else {
Set<String> newTypes = new HashSet<>(types.size() + 1);
newTypes.addAll(types);
newTypes.add(type);
assert newTypes.size() == types.size() + 1;
newTypes = Collections.unmodifiableSet(newTypes);
return map.copyAndPut(key, newTypes);
}
}
/**
@ -63,7 +95,9 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
throw new IllegalArgumentException("Default mappings should not be added to the lookup");
}
CopyOnWriteHashMap<String, MappedFieldTypeReference> fullName = this.fullNameToFieldType;
CopyOnWriteHashMap<String, Set<String>> fullNameToTypes = this.fullNameToTypes;
CopyOnWriteHashMap<String, MappedFieldTypeReference> indexName = this.indexNameToFieldType;
CopyOnWriteHashMap<String, Set<String>> indexNameToTypes = this.indexNameToTypes;
for (FieldMapper fieldMapper : newFieldMappers) {
MappedFieldType fieldType = fieldMapper.fieldType();
@ -91,8 +125,23 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
// this new field bridges between two existing field names (a full and index name), which we cannot support
throw new IllegalStateException("insane mappings found. field " + fieldType.names().fullName() + " maps across types to field " + fieldType.names().indexName());
}
fullNameToTypes = addType(fullNameToTypes, fieldType.names().fullName(), type);
indexNameToTypes = addType(indexNameToTypes, fieldType.names().indexName(), type);
}
return new FieldTypeLookup(fullName, fullNameToTypes, indexName, indexNameToTypes);
}
private static boolean beStrict(String type, Set<String> types, boolean updateAllTypes) {
assert types.size() >= 1;
if (updateAllTypes) {
return false;
} else if (types.size() == 1 && types.contains(type)) {
// we are implicitly updating all types
return false;
} else {
return true;
}
return new FieldTypeLookup(fullName, indexName);
}
/**
@ -100,14 +149,15 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
* If any are not compatible, an IllegalArgumentException is thrown.
* If updateAllTypes is true, only basic compatibility is checked.
*/
public void checkCompatibility(Collection<FieldMapper> newFieldMappers, boolean updateAllTypes) {
for (FieldMapper fieldMapper : newFieldMappers) {
public void checkCompatibility(String type, Collection<FieldMapper> fieldMappers, boolean updateAllTypes) {
for (FieldMapper fieldMapper : fieldMappers) {
MappedFieldTypeReference ref = fullNameToFieldType.get(fieldMapper.fieldType().names().fullName());
if (ref != null) {
List<String> conflicts = new ArrayList<>();
ref.get().checkTypeName(fieldMapper.fieldType(), conflicts);
if (conflicts.isEmpty()) { // only check compat if they are the same type
boolean strict = updateAllTypes == false;
final Set<String> types = fullNameToTypes.get(fieldMapper.fieldType().names().fullName());
boolean strict = beStrict(type, types, updateAllTypes);
ref.get().checkCompatibility(fieldMapper.fieldType(), conflicts, strict);
}
if (conflicts.isEmpty() == false) {
@ -121,7 +171,8 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
List<String> conflicts = new ArrayList<>();
indexNameRef.get().checkTypeName(fieldMapper.fieldType(), conflicts);
if (conflicts.isEmpty()) { // only check compat if they are the same type
boolean strict = updateAllTypes == false;
final Set<String> types = indexNameToTypes.get(fieldMapper.fieldType().names().indexName());
boolean strict = beStrict(type, types, updateAllTypes);
indexNameRef.get().checkCompatibility(fieldMapper.fieldType(), conflicts, strict);
}
if (conflicts.isEmpty() == false) {
@ -138,6 +189,15 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
return ref.get();
}
/** Get the set of types that have a mapping for the given field. */
public Set<String> getTypes(String field) {
Set<String> types = fullNameToTypes.get(field);
if (types == null) {
types = Collections.emptySet();
}
return types;
}
/** Returns the field type for the given index name */
public MappedFieldType getByIndexName(String field) {
MappedFieldTypeReference ref = indexNameToFieldType.get(field);
@ -145,6 +205,15 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
return ref.get();
}
/** Get the set of types that have a mapping for the given field. */
public Set<String> getTypesByIndexName(String field) {
Set<String> types = indexNameToTypes.get(field);
if (types == null) {
types = Collections.emptySet();
}
return types;
}
/**
* Returns a list of the index names of a simple match regex like pattern against full name and index name.
*/

@ -23,12 +23,10 @@ package org.elasticsearch.index.mapper;
*/
public class MappedFieldTypeReference {
private MappedFieldType fieldType; // the current field type this reference points to
private int numAssociatedMappers;
public MappedFieldTypeReference(MappedFieldType fieldType) {
fieldType.freeze(); // ensure frozen
this.fieldType = fieldType;
this.numAssociatedMappers = 1;
}
public MappedFieldType get() {
@ -40,11 +38,4 @@ public class MappedFieldTypeReference {
this.fieldType = fieldType;
}
public int getNumAssociatedMappers() {
return numAssociatedMappers;
}
public void incrementAssociatedMappers() {
++numAssociatedMappers;
}
}

@ -154,5 +154,5 @@ 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) throws MergeMappingException;
public abstract void merge(Mapper mergeWith, MergeResult mergeResult);
}

@ -33,6 +33,7 @@ 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;
import org.elasticsearch.common.regex.Regex;
@ -253,20 +254,17 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
// simulate first
MergeResult result = oldMapper.merge(mapper.mapping(), true, updateAllTypes);
if (result.hasConflicts()) {
throw new MergeMappingException(result.buildConflicts());
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
return oldMapper;
} else {
List<ObjectMapper> newObjectMappers = new ArrayList<>();
List<FieldMapper> newFieldMappers = new ArrayList<>();
for (MetadataFieldMapper metadataMapper : mapper.mapping().metadataMappers) {
newFieldMappers.add(metadataMapper);
}
MapperUtils.collect(mapper.mapping().root, newObjectMappers, newFieldMappers);
checkNewMappersCompatibility(newObjectMappers, newFieldMappers, updateAllTypes);
Tuple<Collection<ObjectMapper>, Collection<FieldMapper>> newMappers = checkMappersCompatibility(
mapper.type(), mapper.mapping(), updateAllTypes);
Collection<ObjectMapper> newObjectMappers = newMappers.v1();
Collection<FieldMapper> newFieldMappers = newMappers.v2();
addMappers(mapper.type(), newObjectMappers, newFieldMappers);
for (DocumentTypeListener typeListener : typeListeners) {
@ -302,9 +300,9 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
return true;
}
protected void checkNewMappersCompatibility(Collection<ObjectMapper> newObjectMappers, Collection<FieldMapper> newFieldMappers, boolean updateAllTypes) {
protected void checkMappersCompatibility(String type, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers, boolean updateAllTypes) {
assert mappingLock.isWriteLockedByCurrentThread();
for (ObjectMapper newObjectMapper : newObjectMappers) {
for (ObjectMapper newObjectMapper : objectMappers) {
ObjectMapper existingObjectMapper = fullPathObjectMappers.get(newObjectMapper.fullPath());
if (existingObjectMapper != null) {
MergeResult result = new MergeResult(true, updateAllTypes);
@ -315,7 +313,19 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
}
}
}
fieldTypes.checkCompatibility(newFieldMappers, updateAllTypes);
fieldTypes.checkCompatibility(type, fieldMappers, updateAllTypes);
}
protected Tuple<Collection<ObjectMapper>, Collection<FieldMapper>> checkMappersCompatibility(
String type, Mapping mapping, boolean updateAllTypes) {
List<ObjectMapper> objectMappers = new ArrayList<>();
List<FieldMapper> fieldMappers = new ArrayList<>();
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
fieldMappers.add(metadataMapper);
}
MapperUtils.collect(mapping.root, objectMappers, fieldMappers);
checkMappersCompatibility(type, objectMappers, fieldMappers, updateAllTypes);
return new Tuple<>(objectMappers, fieldMappers);
}
protected void addMappers(String type, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers) {

@ -1,62 +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.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
/**
*
*/
public final class MergeMappingException extends MapperException {
private final String[] failures;
public MergeMappingException(String[] failures) {
super("Merge failed with failures {" + Arrays.toString(failures) + "}");
Objects.requireNonNull(failures, "failures must be non-null");
this.failures = failures;
}
public MergeMappingException(StreamInput in) throws IOException {
super(in);
failures = in.readStringArray();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStringArray(failures);
}
public String[] failures() {
return failures;
}
@Override
public RestStatus status() {
return RestStatus.BAD_REQUEST;
}
}

@ -20,6 +20,7 @@
package org.elasticsearch.index.mapper;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser;
/**
@ -159,16 +160,22 @@ public class SourceToParse {
return this.ttl;
}
public SourceToParse ttl(TimeValue ttl) {
if (ttl == null) {
this.ttl = -1;
return this;
}
this.ttl = ttl.millis();
return this;
}
public SourceToParse ttl(long ttl) {
this.ttl = ttl;
return this;
}
public static enum Origin {
public enum Origin {
PRIMARY,
REPLICA
}
}

@ -605,7 +605,7 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
CompletionFieldMapper fieldMergeWith = (CompletionFieldMapper) mergeWith;
if (!mergeResult.simulate()) {

@ -135,6 +135,15 @@ public abstract class NumberFieldMapper extends FieldMapper implements AllFieldM
super(ref);
}
@Override
public void checkCompatibility(MappedFieldType other,
List<String> conflicts, boolean strict) {
super.checkCompatibility(other, conflicts, strict);
if (numericPrecisionStep() != other.numericPrecisionStep()) {
conflicts.add("mapper [" + names().fullName() + "] has different [precision_step] values");
}
}
public abstract NumberFieldType clone();
@Override
@ -245,17 +254,12 @@ public abstract class NumberFieldMapper extends FieldMapper implements AllFieldM
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;
}
NumberFieldMapper nfmMergeWith = (NumberFieldMapper) mergeWith;
if (this.fieldTypeRef.getNumAssociatedMappers() > 1 && mergeResult.updateAllTypes() == false) {
if (fieldType().numericPrecisionStep() != nfmMergeWith.fieldType().numericPrecisionStep()) {
mergeResult.addConflict("mapper [" + fieldType().names().fullName() + "] is used by multiple types. Set update_all_types to true to update precision_step across all types.");
}
}
if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) {
this.includeInAll = nfmMergeWith.includeInAll;

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
@ -360,7 +359,7 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.StringFieldMapper.ValueAndBoost;
@ -191,7 +190,7 @@ public class TokenCountFieldMapper extends IntegerFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;

@ -38,7 +38,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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.DoubleFieldMapper;
@ -389,7 +388,7 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;

@ -39,7 +39,6 @@ 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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.DoubleFieldMapper;
@ -298,7 +297,7 @@ public class GeoPointFieldMapperLegacy extends BaseGeoPointFieldMapper implement
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
@ -472,7 +471,7 @@ public class GeoShapeFieldMapper extends FieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
if (!this.getClass().equals(mergeWith.getClass())) {
return;

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -310,7 +309,7 @@ public class AllFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
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());
}

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -332,7 +331,7 @@ public class IdFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
// do nothing here, no merging, but also no exception
}
}

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -280,7 +279,7 @@ public class IndexFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
IndexFieldMapper indexFieldMapperMergeWith = (IndexFieldMapper) mergeWith;
if (!mergeResult.simulate()) {
if (indexFieldMapperMergeWith.enabledState != enabledState && !indexFieldMapperMergeWith.enabledState.unset()) {

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -372,7 +371,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
super.merge(mergeWith, mergeResult);
ParentFieldMapper fieldMergeWith = (ParentFieldMapper) mergeWith;
if (Objects.equals(parentType, fieldMergeWith.parentType) == false) {

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -250,7 +249,7 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
// do nothing here, no merging, but also no exception
}
}

@ -44,7 +44,6 @@ 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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -442,7 +441,7 @@ public class SourceFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
SourceFieldMapper sourceMergeWith = (SourceFieldMapper) mergeWith;
if (mergeResult.simulate()) {
if (this.enabled != sourceMergeWith.enabled) {

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -259,7 +258,7 @@ public class TTLFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
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) {

@ -33,7 +33,6 @@ 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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
@ -380,7 +379,7 @@ public class TimestampFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
TimestampFieldMapper timestampFieldMapperMergeWith = (TimestampFieldMapper) mergeWith;
super.merge(mergeWith, mergeResult);
if (!mergeResult.simulate()) {

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -226,7 +225,7 @@ public class TypeFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
// do nothing here, no merging, but also no exception
}
}

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -226,7 +225,7 @@ public class UidFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
// do nothing here, no merging, but also no exception
}
}

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -167,7 +166,7 @@ public class VersionFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
// nothing to do
}
}

@ -464,7 +464,7 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll,
}
@Override
public void merge(final Mapper mergeWith, final MergeResult mergeResult) throws MergeMappingException {
public void merge(final Mapper mergeWith, final MergeResult mergeResult) {
if (!(mergeWith instanceof ObjectMapper)) {
mergeResult.addConflict("Can't merge a non object mapping [" + mergeWith.name() + "] with an object mapping [" + name() + "]");
return;

@ -45,9 +45,6 @@ public class RestForceMergeAction extends BaseRestHandler {
super(settings, controller, client);
controller.registerHandler(POST, "/_forcemerge", this);
controller.registerHandler(POST, "/{index}/_forcemerge", this);
controller.registerHandler(GET, "/_forcemerge", this);
controller.registerHandler(GET, "/{index}/_forcemerge", this);
}
@Override

@ -74,7 +74,7 @@ public class RestIndexAction extends BaseRestHandler {
indexRequest.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
indexRequest.timestamp(request.param("timestamp"));
if (request.hasParam("ttl")) {
indexRequest.ttl(request.paramAsTime("ttl", null).millis());
indexRequest.ttl(request.param("ttl"));
}
indexRequest.source(request.content());
indexRequest.timeout(request.paramAsTime("timeout", IndexRequest.DEFAULT_TIMEOUT));

@ -105,7 +105,7 @@ public class RestUpdateAction extends BaseRestHandler {
upsertRequest.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
upsertRequest.timestamp(request.param("timestamp"));
if (request.hasParam("ttl")) {
upsertRequest.ttl(request.paramAsTime("ttl", null).millis());
upsertRequest.ttl(request.param("ttl"));
}
upsertRequest.version(RestActions.parseVersion(request));
upsertRequest.versionType(VersionType.fromString(request.param("version_type"), upsertRequest.versionType()));
@ -116,7 +116,7 @@ public class RestUpdateAction extends BaseRestHandler {
doc.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
doc.timestamp(request.param("timestamp"));
if (request.hasParam("ttl")) {
doc.ttl(request.paramAsTime("ttl", null).millis());
doc.ttl(request.param("ttl"));
}
doc.version(RestActions.parseVersion(request));
doc.versionType(VersionType.fromString(request.param("version_type"), doc.versionType()));

@ -51,7 +51,6 @@ import org.elasticsearch.index.AlreadyExpiredException;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.engine.IndexFailedEngineException;
import org.elasticsearch.index.engine.RecoveryEngineException;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShardState;
@ -275,11 +274,6 @@ public class ExceptionSerializationTests extends ESTestCase {
assertEquals(-3, alreadyExpiredException.now());
}
public void testMergeMappingException() throws IOException {
MergeMappingException ex = serialize(new MergeMappingException(new String[]{"one", "two"}));
assertArrayEquals(ex.failures(), new String[]{"one", "two"});
}
public void testActionNotFoundTransportException() throws IOException {
ActionNotFoundTransportException ex = serialize(new ActionNotFoundTransportException("AACCCTION"));
assertEquals("AACCCTION", ex.action());
@ -725,7 +719,6 @@ public class ExceptionSerializationTests extends ESTestCase {
ids.put(84, org.elasticsearch.transport.NodeDisconnectedException.class);
ids.put(85, org.elasticsearch.index.AlreadyExpiredException.class);
ids.put(86, org.elasticsearch.search.aggregations.AggregationExecutionException.class);
ids.put(87, org.elasticsearch.index.mapper.MergeMappingException.class);
ids.put(88, org.elasticsearch.indices.InvalidIndexTemplateException.class);
ids.put(89, org.elasticsearch.percolator.PercolateException.class);
ids.put(90, org.elasticsearch.index.engine.RefreshFailedEngineException.class);

@ -18,6 +18,8 @@
*/
package org.elasticsearch.action.index;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.test.ESTestCase;
@ -25,10 +27,7 @@ import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.*;
/**
*/
@ -67,4 +66,43 @@ public class IndexRequestTests extends ESTestCase {
request.version(randomIntBetween(0, Integer.MAX_VALUE));
assertThat(request.validate().validationErrors(), not(empty()));
}
public void testSetTTLAsTimeValue() {
IndexRequest indexRequest = new IndexRequest();
TimeValue ttl = TimeValue.parseTimeValue(randomTimeValue(), null, "ttl");
indexRequest.ttl(ttl);
assertThat(indexRequest.ttl(), equalTo(ttl));
}
public void testSetTTLAsString() {
IndexRequest indexRequest = new IndexRequest();
String ttlAsString = randomTimeValue();
TimeValue ttl = TimeValue.parseTimeValue(ttlAsString, null, "ttl");
indexRequest.ttl(ttlAsString);
assertThat(indexRequest.ttl(), equalTo(ttl));
}
public void testSetTTLAsLong() {
IndexRequest indexRequest = new IndexRequest();
String ttlAsString = randomTimeValue();
TimeValue ttl = TimeValue.parseTimeValue(ttlAsString, null, "ttl");
indexRequest.ttl(ttl.millis());
assertThat(indexRequest.ttl(), equalTo(ttl));
}
public void testValidateTTL() {
IndexRequest indexRequest = new IndexRequest("index", "type");
if (randomBoolean()) {
indexRequest.ttl(randomIntBetween(Integer.MIN_VALUE, -1));
} else {
if (randomBoolean()) {
indexRequest.ttl(new TimeValue(randomIntBetween(Integer.MIN_VALUE, -1)));
} else {
indexRequest.ttl(randomIntBetween(Integer.MIN_VALUE, -1) + "ms");
}
}
ActionRequestValidationException validate = indexRequest.validate();
assertThat(validate, notNullValue());
assertThat(validate.getMessage(), containsString("ttl must not be negative"));
}
}

@ -23,6 +23,7 @@ import org.elasticsearch.Version;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.index.get.GetResult;
@ -33,11 +34,7 @@ import org.elasticsearch.test.ESTestCase;
import java.util.Map;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.*;
public class UpdateRequestTests extends ESTestCase {
public void testUpdateRequest() throws Exception {
@ -127,7 +124,7 @@ public class UpdateRequestTests extends ESTestCase {
// Related to issue 3256
public void testUpdateRequestWithTTL() throws Exception {
long providedTTLValue = randomIntBetween(500, 1000);
TimeValue providedTTLValue = TimeValue.parseTimeValue(randomTimeValue(), null, "ttl");
Settings settings = settings(Version.CURRENT).build();
UpdateHelper updateHelper = new UpdateHelper(settings, null);

@ -0,0 +1,81 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.test.ESAllocationTestCase;
import java.util.Arrays;
import java.util.HashSet;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
import static org.hamcrest.Matchers.equalTo;
public class ActiveAllocationIdTests extends ESAllocationTestCase {
public void testActiveAllocationIdsUpdated() {
AllocationService allocation = createAllocationService();
logger.info("creating an index with 1 shard, 2 replicas");
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
// add index metadata where we have no routing nodes to check that allocation ids are not removed
.put(IndexMetaData.builder("test-old").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2)
.putActiveAllocationIds(0, new HashSet<>(Arrays.asList("x", "y"))))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
logger.info("adding three nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(
newNode("node1")).put(newNode("node2")).put(newNode("node3"))).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.metaData().index("test").activeAllocationIds(0).size(), equalTo(0));
assertThat(clusterState.metaData().index("test-old").activeAllocationIds(0), equalTo(new HashSet<>(Arrays.asList("x", "y"))));
logger.info("start primary shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingTable().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.metaData().index("test").activeAllocationIds(0).size(), equalTo(1));
assertThat(clusterState.getRoutingTable().shardsWithState(STARTED).get(0).allocationId().getId(),
equalTo(clusterState.metaData().index("test").activeAllocationIds(0).iterator().next()));
assertThat(clusterState.metaData().index("test-old").activeAllocationIds(0), equalTo(new HashSet<>(Arrays.asList("x", "y"))));
logger.info("start replica shards");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.metaData().index("test").activeAllocationIds(0).size(), equalTo(3));
logger.info("remove a node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.remove("node1"))
.build();
rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.metaData().index("test").activeAllocationIds(0).size(), equalTo(2));
logger.info("remove all remaining nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.remove("node2").remove("node3"))
.build();
rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// active allocation ids should not be updated
assertThat(clusterState.getRoutingTable().shardsWithState(UNASSIGNED).size(), equalTo(3));
assertThat(clusterState.metaData().index("test").activeAllocationIds(0).size(), equalTo(2));
}
}

@ -22,68 +22,383 @@ package org.elasticsearch.cluster.serialization;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.DiffableUtils;
import org.elasticsearch.cluster.DiffableUtils.KeyedReader;
import org.elasticsearch.cluster.DiffableUtils.MapDiff;
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.StreamableReader;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.CoreMatchers.nullValue;
public class DiffableTests extends ESTestCase {
public void testJdkMapDiff() throws IOException {
Map<String, TestDiffable> before = new HashMap<>();
before.put("foo", new TestDiffable("1"));
before.put("bar", new TestDiffable("2"));
before.put("baz", new TestDiffable("3"));
before = unmodifiableMap(before);
Map<String, TestDiffable> map = new HashMap<>();
map.putAll(before);
map.remove("bar");
map.put("baz", new TestDiffable("4"));
map.put("new", new TestDiffable("5"));
Map<String, TestDiffable> after = unmodifiableMap(new HashMap<>(map));
Diff diff = DiffableUtils.diff(before, after);
BytesStreamOutput out = new BytesStreamOutput();
diff.writeTo(out);
StreamInput in = StreamInput.wrap(out.bytes());
Map<String, TestDiffable> serialized = DiffableUtils.readJdkMapDiff(in, TestDiffable.PROTO).apply(before);
assertThat(serialized.size(), equalTo(3));
assertThat(serialized.get("foo").value(), equalTo("1"));
assertThat(serialized.get("baz").value(), equalTo("4"));
assertThat(serialized.get("new").value(), equalTo("5"));
public void testJKDMapDiff() throws IOException {
new JdkMapDriver<TestDiffable>() {
@Override
protected boolean diffableValues() {
return true;
}
@Override
protected TestDiffable createValue(Integer key, boolean before) {
return new TestDiffable(String.valueOf(before ? key : key + 1));
}
@Override
protected MapDiff diff(Map<Integer, TestDiffable> before, Map<Integer, TestDiffable> after) {
return DiffableUtils.diff(before, after, keySerializer);
}
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return useProtoForDiffableSerialization
? DiffableUtils.readJdkMapDiff(in, keySerializer, TestDiffable.PROTO)
: DiffableUtils.readJdkMapDiff(in, keySerializer, diffableValueSerializer());
}
}.execute();
new JdkMapDriver<String>() {
@Override
protected boolean diffableValues() {
return false;
}
@Override
protected String createValue(Integer key, boolean before) {
return String.valueOf(before ? key : key + 1);
}
@Override
protected MapDiff diff(Map<Integer, String> before, Map<Integer, String> after) {
return DiffableUtils.diff(before, after, keySerializer, nonDiffableValueSerializer());
}
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return DiffableUtils.readJdkMapDiff(in, keySerializer, nonDiffableValueSerializer());
}
}.execute();
}
public void testImmutableOpenMapDiff() throws IOException {
ImmutableOpenMap.Builder<String, TestDiffable> builder = ImmutableOpenMap.builder();
builder.put("foo", new TestDiffable("1"));
builder.put("bar", new TestDiffable("2"));
builder.put("baz", new TestDiffable("3"));
ImmutableOpenMap<String, TestDiffable> before = builder.build();
builder = ImmutableOpenMap.builder(before);
builder.remove("bar");
builder.put("baz", new TestDiffable("4"));
builder.put("new", new TestDiffable("5"));
ImmutableOpenMap<String, TestDiffable> after = builder.build();
Diff diff = DiffableUtils.diff(before, after);
BytesStreamOutput out = new BytesStreamOutput();
diff.writeTo(out);
StreamInput in = StreamInput.wrap(out.bytes());
ImmutableOpenMap<String, TestDiffable> serialized = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader<TestDiffable>() {
new ImmutableOpenMapDriver<TestDiffable>() {
@Override
public TestDiffable readFrom(StreamInput in, String key) throws IOException {
protected boolean diffableValues() {
return true;
}
@Override
protected TestDiffable createValue(Integer key, boolean before) {
return new TestDiffable(String.valueOf(before ? key : key + 1));
}
@Override
protected MapDiff diff(ImmutableOpenMap<Integer, TestDiffable> before, ImmutableOpenMap<Integer, TestDiffable> after) {
return DiffableUtils.diff(before, after, keySerializer);
}
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return useProtoForDiffableSerialization
? DiffableUtils.readImmutableOpenMapDiff(in, keySerializer, TestDiffable.PROTO)
: DiffableUtils.readImmutableOpenMapDiff(in, keySerializer, diffableValueSerializer());
}
}.execute();
new ImmutableOpenMapDriver<String>() {
@Override
protected boolean diffableValues() {
return false;
}
@Override
protected String createValue(Integer key, boolean before) {
return String.valueOf(before ? key : key + 1);
}
@Override
protected MapDiff diff(ImmutableOpenMap<Integer, String> before, ImmutableOpenMap<Integer, String> after) {
return DiffableUtils.diff(before, after, keySerializer, nonDiffableValueSerializer());
}
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return DiffableUtils.readImmutableOpenMapDiff(in, keySerializer, nonDiffableValueSerializer());
}
}.execute();
}
public void testImmutableOpenIntMapDiff() throws IOException {
new ImmutableOpenIntMapDriver<TestDiffable>() {
@Override
protected boolean diffableValues() {
return true;
}
@Override
protected TestDiffable createValue(Integer key, boolean before) {
return new TestDiffable(String.valueOf(before ? key : key + 1));
}
@Override
protected MapDiff diff(ImmutableOpenIntMap<TestDiffable> before, ImmutableOpenIntMap<TestDiffable> after) {
return DiffableUtils.diff(before, after, keySerializer);
}
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return useProtoForDiffableSerialization
? DiffableUtils.readImmutableOpenIntMapDiff(in, keySerializer, TestDiffable.PROTO)
: DiffableUtils.readImmutableOpenIntMapDiff(in, keySerializer, diffableValueSerializer());
}
}.execute();
new ImmutableOpenIntMapDriver<String>() {
@Override
protected boolean diffableValues() {
return false;
}
@Override
protected String createValue(Integer key, boolean before) {
return String.valueOf(before ? key : key + 1);
}
@Override
protected MapDiff diff(ImmutableOpenIntMap<String> before, ImmutableOpenIntMap<String> after) {
return DiffableUtils.diff(before, after, keySerializer, nonDiffableValueSerializer());
}
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return DiffableUtils.readImmutableOpenIntMapDiff(in, keySerializer, nonDiffableValueSerializer());
}
}.execute();
}
/**
* Class that abstracts over specific map implementation type and value kind (Diffable or not)
* @param <T> map type
* @param <V> value type
*/
public abstract class MapDriver<T, V> {
protected final Set<Integer> keys = randomPositiveIntSet();
protected final Set<Integer> keysToRemove = new HashSet<>(randomSubsetOf(randomInt(keys.size()), keys.toArray(new Integer[keys.size()])));
protected final Set<Integer> keysThatAreNotRemoved = Sets.difference(keys, keysToRemove);
protected final Set<Integer> keysToOverride = new HashSet<>(randomSubsetOf(randomInt(keysThatAreNotRemoved.size()),
keysThatAreNotRemoved.toArray(new Integer[keysThatAreNotRemoved.size()])));
protected final Set<Integer> keysToAdd = Sets.difference(randomPositiveIntSet(), keys); // make sure keysToAdd does not contain elements in keys
protected final Set<Integer> keysUnchanged = Sets.difference(keysThatAreNotRemoved, keysToOverride);
protected final DiffableUtils.KeySerializer<Integer> keySerializer = randomBoolean()
? DiffableUtils.getIntKeySerializer()
: DiffableUtils.getVIntKeySerializer();
protected final boolean useProtoForDiffableSerialization = randomBoolean();
private Set<Integer> randomPositiveIntSet() {
int maxSetSize = randomInt(6);
Set<Integer> result = new HashSet<>();
for (int i = 0; i < maxSetSize; i++) {
// due to duplicates, set size can be smaller than maxSetSize
result.add(randomIntBetween(0, 100));
}
return result;
}
/**
* whether we operate on {@link org.elasticsearch.cluster.Diffable} values
*/
protected abstract boolean diffableValues();
/**
* functions that determines value in "before" or "after" map based on key
*/
protected abstract V createValue(Integer key, boolean before);
/**
* creates map based on JDK-based map
*/
protected abstract T createMap(Map<Integer, V> values);
/**
* calculates diff between two maps
*/
protected abstract MapDiff<Integer, V, T> diff(T before, T after);
/**
* reads diff of maps from stream
*/
protected abstract MapDiff<Integer, V, T> readDiff(StreamInput in) throws IOException;
/**
* gets element at key "key" in map "map"
*/
protected abstract V get(T map, Integer key);
/**
* returns size of given map
*/
protected abstract int size(T map);
/**
* executes the actual test
*/
public void execute() throws IOException {
logger.debug("Keys in 'before' map: {}", keys);
logger.debug("Keys to remove: {}", keysToRemove);
logger.debug("Keys to override: {}", keysToOverride);
logger.debug("Keys to add: {}", keysToAdd);
logger.debug("--> creating 'before' map");
Map<Integer, V> before = new HashMap<>();
for (Integer key : keys) {
before.put(key, createValue(key, true));
}
T beforeMap = createMap(before);
logger.debug("--> creating 'after' map");
Map<Integer, V> after = new HashMap<>();
after.putAll(before);
for (Integer key : keysToRemove) {
after.remove(key);
}
for (Integer key : keysToOverride) {
after.put(key, createValue(key, false));
}
for (Integer key : keysToAdd) {
after.put(key, createValue(key, false));
}
T afterMap = createMap(unmodifiableMap(after));
MapDiff<Integer, V, T> diffMap = diff(beforeMap, afterMap);
// check properties of diffMap
assertThat(new HashSet(diffMap.getDeletes()), equalTo(keysToRemove));
if (diffableValues()) {
assertThat(diffMap.getDiffs().keySet(), equalTo(keysToOverride));
for (Integer key : keysToOverride) {
assertThat(diffMap.getDiffs().get(key).apply(get(beforeMap, key)), equalTo(get(afterMap, key)));
}
assertThat(diffMap.getUpserts().keySet(), equalTo(keysToAdd));
for (Integer key : keysToAdd) {
assertThat(diffMap.getUpserts().get(key), equalTo(get(afterMap, key)));
}
} else {
assertThat(diffMap.getDiffs(), equalTo(emptyMap()));
Set<Integer> keysToAddAndOverride = Sets.union(keysToAdd, keysToOverride);
assertThat(diffMap.getUpserts().keySet(), equalTo(keysToAddAndOverride));
for (Integer key : keysToAddAndOverride) {
assertThat(diffMap.getUpserts().get(key), equalTo(get(afterMap, key)));
}
}
if (randomBoolean()) {
logger.debug("--> serializing diff");
BytesStreamOutput out = new BytesStreamOutput();
diffMap.writeTo(out);
StreamInput in = StreamInput.wrap(out.bytes());
logger.debug("--> reading diff back");
diffMap = readDiff(in);
}
T appliedDiffMap = diffMap.apply(beforeMap);
// check properties of appliedDiffMap
assertThat(size(appliedDiffMap), equalTo(keys.size() - keysToRemove.size() + keysToAdd.size()));
for (Integer key : keysToRemove) {
assertThat(get(appliedDiffMap, key), nullValue());
}
for (Integer key : keysUnchanged) {
assertThat(get(appliedDiffMap, key), equalTo(get(beforeMap, key)));
}
for (Integer key : keysToOverride) {
assertThat(get(appliedDiffMap, key), not(equalTo(get(beforeMap, key))));
assertThat(get(appliedDiffMap, key), equalTo(get(afterMap, key)));
}
for (Integer key : keysToAdd) {
assertThat(get(appliedDiffMap, key), equalTo(get(afterMap, key)));
}
}
}
abstract class JdkMapDriver<V> extends MapDriver<Map<Integer, V>, V> {
@Override
protected Map<Integer, V> createMap(Map values) {
return values;
}
@Override
protected V get(Map<Integer, V> map, Integer key) {
return map.get(key);
}
@Override
protected int size(Map<Integer, V> map) {
return map.size();
}
}
abstract class ImmutableOpenMapDriver<V> extends MapDriver<ImmutableOpenMap<Integer, V>, V> {
@Override
protected ImmutableOpenMap<Integer, V> createMap(Map values) {
return ImmutableOpenMap.<Integer, V>builder().putAll(values).build();
}
@Override
protected V get(ImmutableOpenMap<Integer, V> map, Integer key) {
return map.get(key);
}
@Override
protected int size(ImmutableOpenMap<Integer, V> map) {
return map.size();
}
}
abstract class ImmutableOpenIntMapDriver<V> extends MapDriver<ImmutableOpenIntMap<V>, V> {
@Override
protected ImmutableOpenIntMap<V> createMap(Map values) {
return ImmutableOpenIntMap.<V>builder().putAll(values).build();
}
@Override
protected V get(ImmutableOpenIntMap<V> map, Integer key) {
return map.get(key);
}
@Override
protected int size(ImmutableOpenIntMap<V> map) {
return map.size();
}
}
private static <K> DiffableUtils.DiffableValueSerializer<K, TestDiffable> diffableValueSerializer() {
return new DiffableUtils.DiffableValueSerializer<K, TestDiffable>() {
@Override
public TestDiffable read(StreamInput in, K key) throws IOException {
return new TestDiffable(in.readString());
}
@Override
public Diff<TestDiffable> readDiffFrom(StreamInput in, String key) throws IOException {
public Diff<TestDiffable> readDiff(StreamInput in, K key) throws IOException {
return AbstractDiffable.readDiffFrom(new StreamableReader<TestDiffable>() {
@Override
public TestDiffable readFrom(StreamInput in) throws IOException {
@ -91,13 +406,23 @@ public class DiffableTests extends ESTestCase {
}
}, in);
}
}).apply(before);
assertThat(serialized.size(), equalTo(3));
assertThat(serialized.get("foo").value(), equalTo("1"));
assertThat(serialized.get("baz").value(), equalTo("4"));
assertThat(serialized.get("new").value(), equalTo("5"));
};
}
private static <K> DiffableUtils.NonDiffableValueSerializer<K, String> nonDiffableValueSerializer() {
return new DiffableUtils.NonDiffableValueSerializer<K, String>() {
@Override
public void write(String value, StreamOutput out) throws IOException {
out.writeString(value);
}
@Override
public String read(StreamInput in, K key) throws IOException {
return in.readString();
}
};
}
public static class TestDiffable extends AbstractDiffable<TestDiffable> {
public static final TestDiffable PROTO = new TestDiffable("");
@ -121,6 +446,22 @@ public class DiffableTests extends ESTestCase {
public void writeTo(StreamOutput out) throws IOException {
out.writeString(value);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TestDiffable that = (TestDiffable) o;
return !(value != null ? !value.equals(that.value) : that.value != null);
}
@Override
public int hashCode() {
return value != null ? value.hashCode() : 0;
}
}
}

@ -488,17 +488,17 @@ public class NodeJoinControllerTests extends ESTestCase {
@Override
public RoutingAllocation.Result applyStartedShards(ClusterState clusterState, List<? extends ShardRouting> startedShards, boolean withReroute) {
return new RoutingAllocation.Result(false, clusterState.routingTable());
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
@Override
public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, List<FailedRerouteAllocation.FailedShard> failedShards) {
return new RoutingAllocation.Result(false, clusterState.routingTable());
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
@Override
protected RoutingAllocation.Result reroute(ClusterState clusterState, String reason, boolean debug) {
return new RoutingAllocation.Result(false, clusterState.routingTable());
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
}
}

@ -37,6 +37,8 @@ public class FieldTypeLookupTests extends ESTestCase {
FieldTypeLookup lookup = new FieldTypeLookup();
assertNull(lookup.get("foo"));
assertNull(lookup.getByIndexName("foo"));
assertEquals(Collections.emptySet(), lookup.getTypes("foo"));
assertEquals(Collections.emptySet(), lookup.getTypesByIndexName("foo"));
Collection<String> names = lookup.simpleMatchToFullName("foo");
assertNotNull(names);
assertTrue(names.isEmpty());
@ -70,6 +72,14 @@ public class FieldTypeLookupTests extends ESTestCase {
assertNull(lookup.get("bar"));
assertEquals(f.fieldType(), lookup2.getByIndexName("bar"));
assertNull(lookup.getByIndexName("foo"));
assertEquals(Collections.emptySet(), lookup.getTypes("foo"));
assertEquals(Collections.emptySet(), lookup.getTypesByIndexName("foo"));
assertEquals(Collections.emptySet(), lookup.getTypes("bar"));
assertEquals(Collections.emptySet(), lookup.getTypesByIndexName("bar"));
assertEquals(Collections.singleton("type"), lookup2.getTypes("foo"));
assertEquals(Collections.emptySet(), lookup2.getTypesByIndexName("foo"));
assertEquals(Collections.emptySet(), lookup2.getTypes("bar"));
assertEquals(Collections.singleton("type"), lookup2.getTypesByIndexName("bar"));
assertEquals(1, size(lookup2.iterator()));
}
@ -144,7 +154,7 @@ public class FieldTypeLookupTests extends ESTestCase {
public void testCheckCompatibilityNewField() {
FakeFieldMapper f1 = new FakeFieldMapper("foo", "bar");
FieldTypeLookup lookup = new FieldTypeLookup();
lookup.checkCompatibility(newList(f1), false);
lookup.checkCompatibility("type", newList(f1), false);
}
public void testCheckCompatibilityMismatchedTypes() {
@ -155,14 +165,14 @@ public class FieldTypeLookupTests extends ESTestCase {
MappedFieldType ft2 = FakeFieldMapper.makeOtherFieldType("foo", "foo");
FieldMapper f2 = new FakeFieldMapper("foo", ft2);
try {
lookup.checkCompatibility(newList(f2), false);
lookup.checkCompatibility("type2", newList(f2), false);
fail("expected type mismatch");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("cannot be changed from type [faketype] to [otherfaketype]"));
}
// fails even if updateAllTypes == true
try {
lookup.checkCompatibility(newList(f2), true);
lookup.checkCompatibility("type2", newList(f2), true);
fail("expected type mismatch");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("cannot be changed from type [faketype] to [otherfaketype]"));
@ -178,25 +188,27 @@ public class FieldTypeLookupTests extends ESTestCase {
ft2.setBoost(2.0f);
FieldMapper f2 = new FakeFieldMapper("foo", ft2);
try {
lookup.checkCompatibility(newList(f2), false);
// different type
lookup.checkCompatibility("type2", newList(f2), false);
fail("expected conflict");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("to update [boost] across all types"));
}
lookup.checkCompatibility(newList(f2), true); // boost is updateable, so ok if forcing
lookup.checkCompatibility("type", newList(f2), false); // boost is updateable, so ok since we are implicitly updating all types
lookup.checkCompatibility("type2", newList(f2), true); // boost is updateable, so ok if forcing
// now with a non changeable setting
MappedFieldType ft3 = FakeFieldMapper.makeFieldType("foo", "bar");
ft3.setStored(true);
FieldMapper f3 = new FakeFieldMapper("foo", ft3);
try {
lookup.checkCompatibility(newList(f3), false);
lookup.checkCompatibility("type2", newList(f3), false);
fail("expected conflict");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("has different [store] values"));
}
// even with updateAllTypes == true, incompatible
try {
lookup.checkCompatibility(newList(f3), true);
lookup.checkCompatibility("type2", newList(f3), true);
fail("expected conflict");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("has different [store] values"));

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.BinaryFieldMapper;
@ -220,7 +219,7 @@ public class ExternalMapper extends FieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
// ignore this for now
}

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -67,7 +66,7 @@ public class ExternalMetadataMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
if (!(mergeWith instanceof ExternalMetadataMapper)) {
mergeResult.addConflict("Trying to merge " + mergeWith + " with " + this);
}

@ -25,12 +25,14 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
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;
@ -715,28 +717,25 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase {
String stage1Mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("point").field("type", "geo_point").field("lat_lon", true)
.field("geohash", true).endObject().endObject().endObject().endObject().string();
DocumentMapperParser parser = createIndex("test", settings).mapperService().documentMapperParser();
DocumentMapper stage1 = parser.parse(stage1Mapping);
MapperService mapperService = createIndex("test", settings).mapperService();
DocumentMapper stage1 = mapperService.merge("type", new CompressedXContent(stage1Mapping), true, false);
String stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("point").field("type", "geo_point").field("lat_lon", false)
.field("geohash", false).endObject().endObject().endObject().endObject().string();
DocumentMapper stage2 = parser.parse(stage2Mapping);
MergeResult mergeResult = stage1.merge(stage2.mapping(), false, false);
assertThat(mergeResult.hasConflicts(), equalTo(true));
assertThat(mergeResult.buildConflicts().length, equalTo(3));
// todo better way of checking conflict?
assertThat("mapper [point] has different [lat_lon]", isIn(new ArrayList<>(Arrays.asList(mergeResult.buildConflicts()))));
assertThat("mapper [point] has different [geohash]", isIn(new ArrayList<>(Arrays.asList(mergeResult.buildConflicts()))));
assertThat("mapper [point] has different [geohash_precision]", isIn(new ArrayList<>(Arrays.asList(mergeResult.buildConflicts()))));
try {
mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [point] has different [lat_lon]"));
assertThat(e.getMessage(), containsString("mapper [point] has different [geohash]"));
assertThat(e.getMessage(), containsString("mapper [point] has different [geohash_precision]"));
}
// correct mapping and ensure no failures
stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("point").field("type", "geo_point").field("lat_lon", true)
.field("geohash", true).endObject().endObject().endObject().endObject().string();
stage2 = parser.parse(stage2Mapping);
mergeResult = stage1.merge(stage2.mapping(), false, false);
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(false));
mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false);
}
public void testGeoHashSearch() throws Exception {

@ -22,12 +22,14 @@ import org.apache.lucene.spatial.prefix.PrefixTreeStrategy;
import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
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;
@ -35,6 +37,7 @@ 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;
@ -376,23 +379,21 @@ public class GeoShapeFieldMapperTests extends ESSingleNodeTestCase {
.startObject("shape").field("type", "geo_shape").field("tree", "geohash").field("strategy", "recursive")
.field("precision", "1m").field("tree_levels", 8).field("distance_error_pct", 0.01).field("orientation", "ccw")
.endObject().endObject().endObject().endObject().string();
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapper stage1 = parser.parse(stage1Mapping);
MapperService mapperService = createIndex("test").mapperService();
DocumentMapper stage1 = mapperService.merge("type", new CompressedXContent(stage1Mapping), true, false);
String stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("shape").field("type", "geo_shape").field("tree", "quadtree")
.field("strategy", "term").field("precision", "1km").field("tree_levels", 26).field("distance_error_pct", 26)
.field("orientation", "cw").endObject().endObject().endObject().endObject().string();
DocumentMapper stage2 = parser.parse(stage2Mapping);
MergeResult mergeResult = stage1.merge(stage2.mapping(), false, false);
// check correct conflicts
assertThat(mergeResult.hasConflicts(), equalTo(true));
assertThat(mergeResult.buildConflicts().length, equalTo(4));
ArrayList<String> conflicts = new ArrayList<>(Arrays.asList(mergeResult.buildConflicts()));
assertThat("mapper [shape] has different [strategy]", isIn(conflicts));
assertThat("mapper [shape] has different [tree]", isIn(conflicts));
assertThat("mapper [shape] has different [tree_levels]", isIn(conflicts));
assertThat("mapper [shape] has different [precision]", isIn(conflicts));
try {
mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [shape] has different [strategy]"));
assertThat(e.getMessage(), containsString("mapper [shape] has different [tree]"));
assertThat(e.getMessage(), containsString("mapper [shape] has different [tree_levels]"));
assertThat(e.getMessage(), containsString("mapper [shape] has different [precision]"));
}
// verify nothing changed
FieldMapper fieldMapper = stage1.mappers().getMapper("shape");
@ -411,11 +412,7 @@ public class GeoShapeFieldMapperTests extends ESSingleNodeTestCase {
stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("shape").field("type", "geo_shape").field("precision", "1m")
.field("tree_levels", 8).field("distance_error_pct", 0.001).field("orientation", "cw").endObject().endObject().endObject().endObject().string();
stage2 = parser.parse(stage2Mapping);
mergeResult = stage1.merge(stage2.mapping(), false, false);
// verify mapping changes, and ensure no failures
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(false));
mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false);
fieldMapper = stage1.mappers().getMapper("shape");
assertThat(fieldMapper, instanceOf(GeoShapeFieldMapper.class));

@ -22,9 +22,11 @@ package org.elasticsearch.index.mapper.multifield.merge;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.compress.CompressedXContent;
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;
@ -32,6 +34,7 @@ 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;
@ -113,9 +116,9 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase {
public void testUpgradeFromMultiFieldTypeToMultiFields() throws Exception {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping1.json");
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
MapperService mapperService = createIndex("test").mapperService();
DocumentMapper docMapper = parser.parse(mapping);
DocumentMapper docMapper = mapperService.merge("person", new CompressedXContent(mapping), true, false);
assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions());
assertThat(docMapper.mappers().getMapper("name.indexed"), nullValue());
@ -129,12 +132,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase {
mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/upgrade1.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(), false, false);
mapperService.merge("person", new CompressedXContent(mapping), false, false);
assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions());
@ -151,12 +149,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase {
assertThat(f, notNullValue());
mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/upgrade2.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(), false, false);
mapperService.merge("person", new CompressedXContent(mapping), false, false);
assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions());
@ -168,24 +161,19 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase {
mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/upgrade3.json");
DocumentMapper docMapper4 = parser.parse(mapping);
mergeResult = docMapper.merge(docMapper4.mapping(), true, false);
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(true));
assertThat(mergeResult.buildConflicts()[0], equalTo("mapper [name] has different [index] values"));
assertThat(mergeResult.buildConflicts()[1], equalTo("mapper [name] has different [store] values"));
try {
mapperService.merge("person", new CompressedXContent(mapping), false, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [name] has different [index] values"));
assertThat(e.getMessage(), containsString("mapper [name] has different [store] values"));
}
mergeResult = docMapper.merge(docMapper4.mapping(), false, false);
assertThat(Arrays.toString(mergeResult.buildConflicts()), mergeResult.hasConflicts(), equalTo(true));
assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions());
assertThat(mergeResult.buildConflicts()[0], equalTo("mapper [name] has different [index] values"));
assertThat(mergeResult.buildConflicts()[1], equalTo("mapper [name] has different [store] values"));
// There are conflicts, but the `name.not_indexed3` has been added, b/c that field has no conflicts
// There are conflicts, so the `name.not_indexed3` has not been added
assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions());
assertThat(docMapper.mappers().getMapper("name.indexed"), notNullValue());
assertThat(docMapper.mappers().getMapper("name.not_indexed"), notNullValue());
assertThat(docMapper.mappers().getMapper("name.not_indexed2"), notNullValue());
assertThat(docMapper.mappers().getMapper("name.not_indexed3"), notNullValue());
assertThat(docMapper.mappers().getMapper("name.not_indexed3"), nullValue());
}
}

@ -25,6 +25,7 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.IndexableFieldType;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -478,7 +479,7 @@ public class SimpleStringMappingTests extends ESSingleNodeTestCase {
.startObject("properties").startObject("field").field("type", "string").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = parser.parse(mapping);
DocumentMapper defaultMapper = indexService.mapperService().merge("type", new CompressedXContent(mapping), true, false);
ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
.startObject()
@ -507,10 +508,12 @@ public class SimpleStringMappingTests extends ESSingleNodeTestCase {
updatedMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "string").startObject("norms").field("enabled", true).endObject()
.endObject().endObject().endObject().endObject().string();
mergeResult = defaultMapper.merge(parser.parse(updatedMapping).mapping(), true, false);
assertTrue(mergeResult.hasConflicts());
assertEquals(1, mergeResult.buildConflicts().length);
assertTrue(mergeResult.buildConflicts()[0].contains("different [omit_norms]"));
try {
defaultMapper.merge(parser.parse(updatedMapping).mapping(), true, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("different [omit_norms]"));
}
}
/**

@ -41,6 +41,7 @@ import org.elasticsearch.index.mapper.DocumentMapper;
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;
@ -557,7 +558,6 @@ public class TimestampMappingTests extends ESSingleNodeTestCase {
public void testMergingConflicts() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_timestamp").field("enabled", true)
.startObject("fielddata").field("format", "doc_values").endObject()
.field("store", "yes")
.field("index", "analyzed")
.field("path", "foo")
@ -565,9 +565,9 @@ public class TimestampMappingTests extends ESSingleNodeTestCase {
.endObject()
.endObject().endObject().string();
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
DocumentMapperParser parser = createIndex("test", indexSettings).mapperService().documentMapperParser();
MapperService mapperService = createIndex("test", indexSettings).mapperService();
DocumentMapper docMapper = parser.parse(mapping);
DocumentMapper docMapper = mapperService.merge("type", new CompressedXContent(mapping), true, false);
assertThat(docMapper.timestampFieldMapper().fieldType().fieldDataType().getLoading(), equalTo(MappedFieldType.Loading.LAZY));
mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_timestamp").field("enabled", false)
@ -579,20 +579,32 @@ public class TimestampMappingTests extends ESSingleNodeTestCase {
.endObject()
.endObject().endObject().string();
MergeResult mergeResult = docMapper.merge(parser.parse(mapping).mapping(), true, false);
List<String> expectedConflicts = new ArrayList<>(Arrays.asList(
"mapper [_timestamp] has different [index] values",
"mapper [_timestamp] has different [store] values",
"Cannot update default in _timestamp value. Value is 1970-01-01 now encountering 1970-01-02",
"Cannot update path in _timestamp value. Value is foo path in merged mapping is bar"));
for (String conflict : mergeResult.buildConflicts()) {
assertTrue("found unexpected conflict [" + conflict + "]", expectedConflicts.remove(conflict));
try {
mapperService.merge("type", new CompressedXContent(mapping), false, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [_timestamp] has different [index] values"));
assertThat(e.getMessage(), containsString("mapper [_timestamp] has different [store] values"));
}
assertTrue("missing conflicts: " + Arrays.toString(expectedConflicts.toArray()), expectedConflicts.isEmpty());
assertThat(docMapper.timestampFieldMapper().fieldType().fieldDataType().getLoading(), equalTo(MappedFieldType.Loading.LAZY));
assertTrue(docMapper.timestampFieldMapper().enabled());
assertThat(docMapper.timestampFieldMapper().fieldType().fieldDataType().getFormat(indexSettings), equalTo("doc_values"));
mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_timestamp").field("enabled", true)
.field("store", "yes")
.field("index", "analyzed")
.field("path", "bar")
.field("default", "1970-01-02")
.endObject()
.endObject().endObject().string();
try {
mapperService.merge("type", new CompressedXContent(mapping), false, false);
fail();
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("Cannot update default in _timestamp value. Value is 1970-01-01 now encountering 1970-01-02"));
assertThat(e.getMessage(), containsString("Cannot update path in _timestamp value. Value is foo path in merged mapping is bar"));
}
}
public void testBackcompatMergingConflictsForIndexValues() throws Exception {

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
@ -169,8 +168,8 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
try {
client().admin().indices().preparePutMapping("testindex").setSource(mappingWithTtlDisabled).setType("type").get();
fail();
} catch (MergeMappingException mme) {
assertThat(mme.getDetailedMessage(), containsString("_ttl cannot be disabled once it was enabled."));
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("_ttl cannot be disabled once it was enabled."));
}
GetMappingsResponse mappingsAfterUpdateResponse = client().admin().indices().prepareGetMappings("testindex").addTypes("type").get();
assertThat(mappingsBeforeUpdateResponse.getMappings().get("testindex").get("type").source(), equalTo(mappingsAfterUpdateResponse.getMappings().get("testindex").get("type").source()));
@ -295,7 +294,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase {
request.process(MetaData.builder().build(), mappingMetaData, true, "test");
// _ttl in a document never worked, so backcompat is ignoring the field
assertEquals(-1, request.ttl());
assertNull(request.ttl());
assertNull(docMapper.parse("test", "type", "1", doc.bytes()).rootDoc().get("_ttl"));
}

@ -24,7 +24,6 @@ import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.HashMap;
@ -49,7 +48,7 @@ public class UpdateMappingOnClusterIT extends ESIntegTestCase {
public void testAllConflicts() throws Exception {
String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/update/all_mapping_create_index.json");
String mappingUpdate = copyToStringFromClasspath("/org/elasticsearch/index/mapper/update/all_mapping_update_with_conflicts.json");
String[] errorMessage = {"[_all] enabled is true now encountering false",
String[] errorMessage = {
"[_all] has different [omit_norms] values",
"[_all] has different [store] values",
"[_all] has different [store_term_vector] values",
@ -62,6 +61,13 @@ public class UpdateMappingOnClusterIT extends ESIntegTestCase {
testConflict(mapping, mappingUpdate, errorMessage);
}
public void testAllDisabled() throws Exception {
XContentBuilder mapping = jsonBuilder().startObject().startObject("mappings").startObject(TYPE).startObject("_all").field("enabled", true).endObject().endObject().endObject().endObject();
XContentBuilder mappingUpdate = jsonBuilder().startObject().startObject("_all").field("enabled", false).endObject().startObject("properties").startObject("text").field("type", "string").endObject().endObject().endObject();
String errorMessage = "[_all] enabled is true now encountering false";
testConflict(mapping.string(), mappingUpdate.string(), errorMessage);
}
public void testAllWithDefault() throws Exception {
String defaultMapping = jsonBuilder().startObject().startObject("_default_")
.startObject("_all")
@ -150,9 +156,9 @@ public class UpdateMappingOnClusterIT extends ESIntegTestCase {
try {
client().admin().indices().preparePutMapping(INDEX).setType(TYPE).setSource(mappingUpdate).get();
fail();
} catch (MergeMappingException e) {
} catch (IllegalArgumentException e) {
for (String errorMessage : errorMessages) {
assertThat(e.getDetailedMessage(), containsString(errorMessage));
assertThat(e.getMessage(), containsString(errorMessage));
}
}
compareMappingOnNodes(mappingsBeforeUpdateResponse);

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.core.LongFieldMapper;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -38,6 +37,7 @@ import java.io.IOException;
import java.util.LinkedHashMap;
import static org.elasticsearch.test.StreamsUtils.copyToStringFromClasspath;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.equalTo;
@ -122,15 +122,15 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
try {
mapperService.merge("type", new CompressedXContent(update.string()), false, false);
fail();
} catch (MergeMappingException e) {
// expected
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [foo] cannot be changed from type [long] to [double]"));
}
try {
mapperService.merge("type", new CompressedXContent(update.string()), false, false);
fail();
} catch (MergeMappingException e) {
// expected
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [foo] cannot be changed from type [long] to [double]"));
}
assertTrue(mapperService.documentMapper("type").mapping().root().getMapper("foo") instanceof LongFieldMapper);
@ -167,7 +167,6 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
}
// same as the testConflictNewType except that the mapping update is on an existing type
@AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/15049")
public void testConflictNewTypeUpdate() throws Exception {
XContentBuilder mapping1 = XContentFactory.jsonBuilder().startObject().startObject("type1")
.startObject("properties").startObject("foo").field("type", "long").endObject()

@ -31,7 +31,6 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.hamcrest.Matchers;
@ -140,8 +139,8 @@ public class UpdateMappingIntegrationIT extends ESIntegTestCase {
client().admin().indices().preparePutMapping("test").setType("type")
.setSource("{\"type\":{\"properties\":{\"body\":{\"type\":\"integer\"}}}}").execute().actionGet();
fail("Expected MergeMappingException");
} catch (MergeMappingException e) {
assertThat(e.getMessage(), containsString("mapper [body] of different type"));
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [body] cannot be changed from type [string] to [int]"));
}
}
@ -154,7 +153,7 @@ public class UpdateMappingIntegrationIT extends ESIntegTestCase {
.setSource("{\"type\":{\"properties\":{\"body\":{\"type\":\"string\", \"norms\": { \"enabled\": true }}}}}").execute()
.actionGet();
fail("Expected MergeMappingException");
} catch (MergeMappingException e) {
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("mapper [body] has different [omit_norms]"));
}
}

@ -31,7 +31,6 @@ import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.query.HasChildQueryBuilder;
import org.elasticsearch.index.query.IdsQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
@ -1176,7 +1175,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
.startObject("_parent").field("type", "parent").endObject()
.endObject().endObject()).get();
fail();
} catch (MergeMappingException e) {
} catch (IllegalArgumentException e) {
assertThat(e.toString(), containsString("Merge failed with failures {[The _parent field's type option can't be changed: [null]->[parent]"));
}
}

@ -78,7 +78,7 @@ subprojects {
* Rest test config *
*****************************************************************************/
apply plugin: 'elasticsearch.rest-test'
integTest {
project.integTest {
dependsOn(project.assemble)
includePackaged true
cluster {
@ -345,7 +345,7 @@ task updateShas(type: UpdateShasTask) {
parentTask = dependencyLicenses
}
RunTask.configure(project)
task run(type: RunTask) {}
/**
* Build some variables that are replaced in the packages. This includes both

@ -343,6 +343,14 @@ to your URL provider. Note that this setting will be used for all S3 repositorie
Different `endpoint`, `region` and `protocol` settings can be set on a per-repository basis
See <<repository-s3-repository>> for details.
[[repository-s3-aws-vpc]]
[float]
==== AWS VPC Bandwidth Settings
AWS instances resolve S3 endpoints to a public IP. If the elasticsearch instances reside in a private subnet in an AWS VPC then all traffic to S3 will go through that VPC's NAT instance. If your VPC's NAT instance is a smaller instance size (e.g. a t1.micro) or is handling a high volume of network traffic your bandwidth to S3 may be limited by that NAT instance's networking bandwidth limitations.
Instances residing in a public subnet in an AWS VPC will connect to S3 via the VPC's internet gateway and not be bandwidth limited by the VPC's NAT instance.
[[repository-s3-testing]]
==== Testing AWS
@ -387,4 +395,3 @@ To run test:
----
mvn -Dtests.aws=true -Dtests.config=/path/to/config/file/elasticsearch.yml clean test
----

@ -9,7 +9,7 @@ of documents that contain the term.
Sorting, aggregations, and access to field values in scripts requires a
different data access pattern. Instead of lookup up the term and finding
documents, we need to be able to look up the document and find the terms that
is has in a field.
it has in a field.
Doc values are the on-disk data structure, built at document index time, which
makes this data access pattern possible. They store the same values as the

@ -90,6 +90,9 @@ The search exists api has been removed in favour of using the search api with
The deprecated `/_optimize` endpoint has been removed. The `/_forcemerge`
endpoint should be used in lieu of optimize.
The `GET` HTTP verb for `/_forcemerge` is no longer supported, please use the
`POST` HTTP verb.
==== Deprecated queries removed
The following deprecated queries have been removed:

@ -425,8 +425,10 @@ curl -XPOST localhost:9200/_search -d '{
"functions": [
{
"script_score": {
"id": "my_script",
"lang" : "native"
"script": {
"id": "my_script",
"lang" : "native"
}
}
}
]

@ -23,18 +23,19 @@ esplugin {
}
versions << [
'azure': '0.7.0',
'azure': '0.9.0',
'jersey': '1.13'
]
dependencies {
compile "com.microsoft.azure:azure-management-compute:${versions.azure}"
compile "com.microsoft.azure:azure-management:${versions.azure}"
compile "com.microsoft.azure:azure-svc-mgmt-compute:${versions.azure}"
compile "com.microsoft.azure:azure-core:${versions.azure}"
compile "org.apache.httpcomponents:httpclient:${versions.httpclient}"
compile "org.apache.httpcomponents:httpcore:${versions.httpcore}"
compile "commons-logging:commons-logging:${versions.commonslogging}"
compile "commons-codec:commons-codec:${versions.commonscodec}"
compile "commons-lang:commons-lang:2.6"
compile "commons-io:commons-io:2.4"
compile 'javax.mail:mail:1.4.5'
compile 'javax.activation:activation:1.1'
compile 'javax.inject:javax.inject:1'

@ -1 +0,0 @@
feed802efe8a7a83d15962d11c6780c63997c528

@ -0,0 +1 @@
050719f91deceed1be1aaf87e85099a861295fa2

@ -1 +0,0 @@
0dfdd1c3a9bd783b087050e979f6ba34f06a68f3

@ -1 +0,0 @@
b945fc3968a4e5a64bbde419c14d92a4a53fa7a1

@ -0,0 +1 @@
887ca8ee5564e8ba2351e6b5db2a1293a8d04674

@ -0,0 +1 @@
b1b6ea3b7e4aa4f492509a4952029cd8e48019ad

@ -0,0 +1,202 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.

@ -0,0 +1,5 @@
Apache Commons IO
Copyright 2002-2014 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).

@ -0,0 +1 @@
0ce1edb914c94ebc388f086c6827e8bdeec71ac2

@ -0,0 +1,202 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.

@ -0,0 +1,8 @@
Apache Commons Lang
Copyright 2001-2015 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
This product includes software from the Spring Framework,
under the Apache License 2.0 (see: StringUtils.containsWhitespace())

@ -602,7 +602,7 @@ public class AttachmentMapper extends FieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
// ignore this for now
}

@ -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.MergeMappingException;
import org.elasticsearch.index.mapper.MergeResult;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
@ -178,7 +177,7 @@ public class SizeFieldMapper extends MetadataFieldMapper {
}
@Override
public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException {
public void merge(Mapper mergeWith, MergeResult mergeResult) {
SizeFieldMapper sizeFieldMapperMergeWith = (SizeFieldMapper) mergeWith;
if (!mergeResult.simulate()) {
if (sizeFieldMapperMergeWith.enabledState != enabledState && !sizeFieldMapperMergeWith.enabledState.unset()) {

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