diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskInfo.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskInfo.java index d71c576093e..c027bfa7ab4 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskInfo.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskInfo.java @@ -29,6 +29,7 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import java.io.IOException; +import java.util.concurrent.TimeUnit; /** * Information about a currently running task. @@ -50,17 +51,24 @@ public class TaskInfo implements Writeable, ToXContent { private final String description; + private final long startTime; + + private final long runningTimeNanos; + private final Task.Status status; private final TaskId parentTaskId; - public TaskInfo(DiscoveryNode node, long id, String type, String action, String description, Task.Status status, TaskId parentTaskId) { + public TaskInfo(DiscoveryNode node, long id, String type, String action, String description, Task.Status status, long startTime, + long runningTimeNanos, TaskId parentTaskId) { this.node = node; this.taskId = new TaskId(node.getId(), id); this.type = type; this.action = action; this.description = description; this.status = status; + this.startTime = startTime; + this.runningTimeNanos = runningTimeNanos; this.parentTaskId = parentTaskId; } @@ -75,6 +83,8 @@ public class TaskInfo implements Writeable, ToXContent { } else { status = null; } + startTime = in.readLong(); + runningTimeNanos = in.readLong(); parentTaskId = new TaskId(in); } @@ -110,6 +120,23 @@ public class TaskInfo implements Writeable, ToXContent { return status; } + /** + * Returns the task start time + */ + public long getStartTime() { + return startTime; + } + + /** + * Returns the task running time + */ + public long getRunningTimeNanos() { + return runningTimeNanos; + } + + /** + * Returns the parent task id + */ public TaskId getParentTaskId() { return parentTaskId; } @@ -132,6 +159,8 @@ public class TaskInfo implements Writeable, ToXContent { } else { out.writeBoolean(false); } + out.writeLong(startTime); + out.writeLong(runningTimeNanos); parentTaskId.writeTo(out); } @@ -147,6 +176,8 @@ public class TaskInfo implements Writeable, ToXContent { if (description != null) { builder.field("description", description); } + builder.dateValueField("start_time_in_millis", "start_time", startTime); + builder.timeValueField("running_time_in_nanos", "running_time", runningTimeNanos, TimeUnit.NANOSECONDS); if (parentTaskId.isSet() == false) { builder.field("parent_task_id", parentTaskId.toString()); } diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 52336ccae0b..b2d58203954 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -30,10 +30,12 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.delete.TransportDeleteAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.action.update.TransportUpdateAction; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; @@ -42,12 +44,9 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AtomicArray; -import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndexAlreadyExistsException; @@ -197,10 +196,10 @@ public class TransportBulkAction extends HandledTransportAction listener) { final long startTime = System.currentTimeMillis(); - executeBulk(bulkRequest, startTime, listener, new AtomicArray(bulkRequest.requests.size())); + executeBulk(bulkRequest, startTime, listener, new AtomicArray<>(bulkRequest.requests.size())); } - private final long buildTookInMillis(long startTime) { + private long buildTookInMillis(long startTime) { // protect ourselves against time going backwards return Math.max(1, System.currentTimeMillis() - startTime); } @@ -214,33 +213,53 @@ public class TransportBulkAction extends HandledTransportAction list = requestsByShard.get(shardIt.shardId()); - if (list == null) { - list = new ArrayList<>(); - requestsByShard.put(shardIt.shardId(), list); - } - list.add(new BulkItemRequest(i, deleteRequest)); - } - } else { - ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, deleteRequest.type(), deleteRequest.id(), deleteRequest.routing()).shardId(); - List list = requestsByShard.get(shardId); - if (list == null) { - list = new ArrayList<>(); - requestsByShard.put(shardId, list); - } - list.add(new BulkItemRequest(i, request)); + ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, deleteRequest.type(), deleteRequest.id(), deleteRequest.routing()).shardId(); + List list = requestsByShard.get(shardId); + if (list == null) { + list = new ArrayList<>(); + requestsByShard.put(shardId, list); } + list.add(new BulkItemRequest(i, request)); } else if (request instanceof UpdateRequest) { UpdateRequest updateRequest = (UpdateRequest) request; String concreteIndex = concreteIndices.getConcreteIndex(updateRequest.index()); - MappingMetaData mappingMd = clusterState.metaData().index(concreteIndex).mappingOrDefault(updateRequest.type()); - if (mappingMd != null && mappingMd.routing().required() && updateRequest.routing() == null) { - BulkItemResponse.Failure failure = new BulkItemResponse.Failure(updateRequest.index(), updateRequest.type(), - updateRequest.id(), new IllegalArgumentException("routing is required for this item")); - responses.set(i, new BulkItemResponse(i, updateRequest.type(), failure)); - continue; - } ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, updateRequest.type(), updateRequest.id(), updateRequest.routing()).shardId(); List list = requestsByShard.get(shardId); if (list == null) { diff --git a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index 3a0e7aeec21..3ded0ed8e83 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -96,23 +96,27 @@ public class TransportDeleteAction extends TransportReplicationAction listener) { diff --git a/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java b/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java index 53d797fbf8f..545016828d1 100644 --- a/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java @@ -35,10 +35,8 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BaseTransportResponseHandler; @@ -91,11 +89,11 @@ public abstract class TransportInstanceSingleOperationAction listener); + protected abstract void resolveRequest(ClusterState state, Request request); protected boolean retryOnFailure(Throwable e) { return false; @@ -141,11 +139,7 @@ public abstract class TransportInstanceSingleOperationAction listener) { - request.routing((state.metaData().resolveIndexRouting(request.parent(), request.routing(), request.index()))); + protected void resolveRequest(ClusterState state, UpdateRequest request) { + resolveAndValidateRouting(state.metaData(), request.concreteIndex(), request); + } + + public static void resolveAndValidateRouting(MetaData metaData, String concreteIndex, UpdateRequest request) { + request.routing((metaData.resolveIndexRouting(request.parent(), request.routing(), request.index()))); // Fail fast on the node that received the request, rather than failing when translating on the index or delete request. - if (request.routing() == null && state.getMetaData().routingRequired(request.concreteIndex(), request.type())) { - throw new RoutingMissingException(request.concreteIndex(), request.type(), request.id()); + if (request.routing() == null && metaData.routingRequired(concreteIndex, request.type())) { + throw new RoutingMissingException(concreteIndex, request.type(), request.id()); } - return true; } @Override diff --git a/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java b/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java index ec4e5ba2421..81a59df539f 100644 --- a/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java +++ b/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java @@ -22,6 +22,7 @@ package org.elasticsearch.bootstrap; import org.apache.lucene.util.Constants; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.StringHelper; +import org.elasticsearch.Build; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.common.PidFile; @@ -193,21 +194,6 @@ final class Bootstrap { node = new Node(nodeSettings); } - @SuppressForbidden(reason = "Exception#printStackTrace()") - private static void setupLogging(Settings settings) { - try { - Class.forName("org.apache.log4j.Logger"); - LogConfigurator.configure(settings, true); - } catch (ClassNotFoundException e) { - // no log4j - } catch (NoClassDefFoundError e) { - // no log4j - } catch (Exception e) { - sysError("Failed to configure logging...", false); - e.printStackTrace(); - } - } - private static Environment initialSettings(boolean foreground) { Terminal terminal = foreground ? Terminal.DEFAULT : null; return InternalSettingsPreparer.prepareEnvironment(EMPTY_SETTINGS, terminal); @@ -254,7 +240,7 @@ final class Bootstrap { Environment environment = initialSettings(foreground); Settings settings = environment.settings(); - setupLogging(settings); + LogConfigurator.configure(settings, true); checkForCustomConfFile(); if (environment.pidFile() != null) { @@ -373,6 +359,9 @@ final class Bootstrap { ))); private static boolean enforceLimits(Settings settings) { + if (Build.CURRENT.isSnapshot()) { + return false; + } for (Setting setting : ENFORCE_SETTINGS) { if (setting.exists(settings)) { return true; diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java b/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java index 184db017c10..2b67b9f1820 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.math.MathUtils; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardNotFoundException; @@ -67,10 +66,6 @@ public class OperationRouting extends AbstractComponent { return preferenceActiveShardIterator(indexShard, clusterState.nodes().localNodeId(), clusterState.nodes(), preference); } - public GroupShardsIterator broadcastDeleteShards(ClusterState clusterState, String index) { - return indexRoutingTable(clusterState, index).groupByShardsIt(); - } - public int searchShardsCount(ClusterState clusterState, String[] concreteIndices, @Nullable Map> routing) { final Set shards = computeTargetedShards(clusterState, concreteIndices, routing); return shards.size(); diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java index 7ac098f5a64..8ca53af186c 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java @@ -48,7 +48,7 @@ import java.util.Date; import java.util.HashMap; import java.util.Locale; import java.util.Map; -import java.util.function.BiConsumer; +import java.util.concurrent.TimeUnit; /** * @@ -961,6 +961,23 @@ public final class XContentBuilder implements BytesStream, Releasable { return this; } + public XContentBuilder timeValueField(String rawFieldName, String readableFieldName, long rawTime, TimeUnit timeUnit) throws + IOException { + if (humanReadable) { + field(readableFieldName, new TimeValue(rawTime, timeUnit).toString()); + } + field(rawFieldName, rawTime); + return this; + } + + public XContentBuilder dateValueField(String rawFieldName, String readableFieldName, long rawTimestamp) throws IOException { + if (humanReadable) { + field(readableFieldName, defaultDatePrinter.print(rawTimestamp)); + } + field(rawFieldName, rawTimestamp); + return this; + } + public XContentBuilder byteSizeField(XContentBuilderString rawFieldName, XContentBuilderString readableFieldName, ByteSizeValue byteSizeValue) throws IOException { if (humanReadable) { field(readableFieldName, byteSizeValue.toString()); diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java index f714c641ca8..c3c1cd3b734 100644 --- a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java +++ b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java @@ -313,7 +313,7 @@ public abstract class MetaDataStateFormat { } return state; } catch (Throwable e) { - exceptions.add(e); + exceptions.add(new IOException("failed to read " + pathAndStateId.toString(), e)); logger.debug("{}: failed to read [{}], ignoring...", e, pathAndStateId.file.toAbsolutePath(), prefix); } } diff --git a/core/src/main/java/org/elasticsearch/tasks/Task.java b/core/src/main/java/org/elasticsearch/tasks/Task.java index 5aa034b7997..2334d10e11b 100644 --- a/core/src/main/java/org/elasticsearch/tasks/Task.java +++ b/core/src/main/java/org/elasticsearch/tasks/Task.java @@ -40,16 +40,26 @@ public class Task { private final TaskId parentTask; + private final long startTime; + + private final long startTimeNanos; + public Task(long id, String type, String action, String description) { this(id, type, action, description, TaskId.EMPTY_TASK_ID); } public Task(long id, String type, String action, String description, TaskId parentTask) { + this(id, type, action, description, parentTask, System.currentTimeMillis(), System.nanoTime()); + } + + public Task(long id, String type, String action, String description, TaskId parentTask, long startTime, long startTimeNanos) { this.id = id; this.type = type; this.action = action; this.description = description; this.parentTask = parentTask; + this.startTime = startTime; + this.startTimeNanos = startTimeNanos; } /** @@ -69,7 +79,8 @@ public class Task { description = getDescription(); status = getStatus(); } - return new TaskInfo(node, getId(), getType(), getAction(), description, status, parentTask); + return new TaskInfo(node, getId(), getType(), getAction(), description, status, startTime, System.nanoTime() - startTimeNanos, + parentTask); } /** @@ -100,6 +111,13 @@ public class Task { return description; } + /** + * Returns the task start time + */ + public long getStartTime() { + return startTime; + } + /** * Returns id of the parent task or NO_PARENT_ID if the task doesn't have any parent tasks */ diff --git a/core/src/main/resources/org/elasticsearch/bootstrap/untrusted.policy b/core/src/main/resources/org/elasticsearch/bootstrap/untrusted.policy index 8078516c7d5..4317cf9123f 100644 --- a/core/src/main/resources/org/elasticsearch/bootstrap/untrusted.policy +++ b/core/src/main/resources/org/elasticsearch/bootstrap/untrusted.policy @@ -25,6 +25,9 @@ grant { // groovy IndyInterface bootstrap requires this property for indy logging permission java.util.PropertyPermission "groovy.indy.logging", "read"; + // groovy requires this to enable workaround for certain JVMs (https://github.com/apache/groovy/pull/137) + permission java.util.PropertyPermission "java.vm.name", "read"; + permission java.util.PropertyPermission "groovy.use.classvalue", "read"; // needed by Rhino engine exception handling permission java.util.PropertyPermission "rhino.stack.style", "read"; diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java index b4464dc9f58..e1501f9b14c 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java @@ -63,6 +63,7 @@ import java.util.concurrent.atomic.AtomicReference; import static org.elasticsearch.action.support.PlainActionFuture.newFuture; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.endsWith; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.not; public class TransportTasksActionTests extends TaskManagerTestCase { @@ -461,10 +462,12 @@ public class TransportTasksActionTests extends TaskManagerTestCase { } public void testTasksDescriptions() throws Exception { + long minimalStartTime = System.currentTimeMillis(); setupTestNodes(Settings.EMPTY); connectNodes(testNodes); CountDownLatch checkLatch = new CountDownLatch(1); ActionFuture future = startBlockingTestNodesAction(checkLatch); + long maximumStartTimeNanos = System.nanoTime(); // Check task counts using transport with filtering TestNode testNode = testNodes[randomIntBetween(0, testNodes.length - 1)]; @@ -478,12 +481,15 @@ public class TransportTasksActionTests extends TaskManagerTestCase { } // Check task counts using transport with detailed description + long minimalDurationNanos = System.nanoTime() - maximumStartTimeNanos; listTasksRequest.detailed(true); // same request only with detailed description response = testNode.transportListTasksAction.execute(listTasksRequest).get(); assertEquals(testNodes.length, response.getPerNodeTasks().size()); for (Map.Entry> entry : response.getPerNodeTasks().entrySet()) { assertEquals(1, entry.getValue().size()); assertEquals("CancellableNodeRequest[Test Request, true]", entry.getValue().get(0).getDescription()); + assertThat(entry.getValue().get(0).getStartTime(), greaterThanOrEqualTo(minimalStartTime)); + assertThat(entry.getValue().get(0).getRunningTimeNanos(), greaterThanOrEqualTo(minimalDurationNanos)); } // Release all tasks and wait for response diff --git a/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java index 2bfc12c0883..cf7b6745c8e 100644 --- a/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java @@ -108,8 +108,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase { } @Override - protected boolean resolveRequest(ClusterState state, Request request, ActionListener listener) { - return true; + protected void resolveRequest(ClusterState state, Request request) { } @Override @@ -230,7 +229,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase { listener.get(); } - public void testSuccessAfterRetryWithExcpetionFromTransport() throws Exception { + public void testSuccessAfterRetryWithExceptionFromTransport() throws Exception { Request request = new Request().index("test"); request.shardId = 0; PlainActionFuture listener = new PlainActionFuture<>(); @@ -290,13 +289,13 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase { Settings.EMPTY, "indices:admin/test_unresolvable", transportService, - new ActionFilters(new HashSet()), + new ActionFilters(new HashSet<>()), new MyResolver(), Request::new ) { @Override - protected boolean resolveRequest(ClusterState state, Request request, ActionListener listener) { - return false; + protected void resolveRequest(ClusterState state, Request request) { + throw new IllegalStateException("request cannot be resolved"); } }; Request request = new Request().index("test"); diff --git a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapSettingsTests.java b/core/src/test/java/org/elasticsearch/bootstrap/BootstrapSettingsTests.java index 0ed6c8da6c1..83eecc63886 100644 --- a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/bootstrap/BootstrapSettingsTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.bootstrap; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.monitor.os.OsProbe; import org.elasticsearch.monitor.process.ProcessProbe; import org.elasticsearch.test.ESTestCase; @@ -34,6 +33,7 @@ public class BootstrapSettingsTests extends ESTestCase { assertTrue(BootstrapSettings.CTRLHANDLER_SETTING.get(Settings.EMPTY)); } + @AwaitsFix(bugUrl = "this feature is disabled for snapshot builds, for now - see #16835") public void testEnforceMaxFileDescriptorLimits() { // nothing should happen since we are in OOB mode Bootstrap.enforceOrLogLimits(Settings.EMPTY); diff --git a/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java b/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java index 36540355e4d..00c549ef2f1 100644 --- a/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java @@ -27,6 +27,7 @@ import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.store.SimpleFSDirectory; import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -59,7 +60,6 @@ import java.util.Set; import java.util.stream.StreamSupport; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -377,7 +377,7 @@ public class MetaDataStateFormatTests extends ESTestCase { format.loadLatestState(logger, dirList.toArray(new Path[0])); fail("latest version can not be read"); } catch (ElasticsearchException ex) { - assertThat(ex.getCause(), instanceOf(CorruptStateException.class)); + assertThat(ExceptionsHelper.unwrap(ex, CorruptStateException.class), notNullValue()); } } diff --git a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java index 88d4cbb2200..03e6cbf9ef1 100644 --- a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java @@ -20,24 +20,26 @@ package org.elasticsearch.routing; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.admin.indices.alias.Alias; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.explain.ExplainResponse; +import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsRequest; import org.elasticsearch.action.termvectors.TermVectorsResponse; +import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.client.Requests; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESIntegTestCase; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.nullValue; @@ -156,8 +158,7 @@ public class SimpleRoutingIT extends ESIntegTestCase { } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/16645") - public void testRequiredRoutingMapping() throws Exception { + public void testRequiredRoutingCrudApis() throws Exception { client().admin().indices().prepareCreate("test").addAlias(new Alias("alias")) .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject()) .execute().actionGet(); @@ -199,13 +200,31 @@ public class SimpleRoutingIT extends ESIntegTestCase { assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); } - logger.info("--> indexing with id [1], and routing [0]"); - client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); - logger.info("--> verifying get with no routing, should not find anything"); + try { + client().prepareUpdate(indexOrAlias(), "type1", "1").setDoc("field", "value2").execute().actionGet(); + fail("update with missing routing when routing is required should fail"); + } catch(ElasticsearchException e) { + assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class)); + } - logger.info("--> bulk deleting with no routing, should broadcast the delete since _routing is required"); - client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1")).execute().actionGet(); + client().prepareUpdate(indexOrAlias(), "type1", "1").setRouting("0").setDoc("field", "value2").execute().actionGet(); client().admin().indices().prepareRefresh().execute().actionGet(); + + for (int i = 0; i < 5; i++) { + try { + client().prepareGet(indexOrAlias(), "type1", "1").execute().actionGet().isExists(); + fail(); + } catch (RoutingMissingException e) { + assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST)); + assertThat(e.getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); + } + GetResponse getResponse = client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet(); + assertThat(getResponse.isExists(), equalTo(true)); + assertThat(getResponse.getSourceAsMap().get("field"), equalTo("value2")); + } + + client().prepareDelete(indexOrAlias(), "type1", "1").setRouting("0").setRefresh(true).execute().actionGet(); + for (int i = 0; i < 5; i++) { try { client().prepareGet(indexOrAlias(), "type1", "1").execute().actionGet().isExists(); @@ -227,28 +246,72 @@ public class SimpleRoutingIT extends ESIntegTestCase { .execute().actionGet(); ensureGreen(); - logger.info("--> indexing with id [1], and routing [0]"); - client().prepareBulk().add( - client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1")).execute().actionGet(); - client().admin().indices().prepareRefresh().execute().actionGet(); + { + BulkResponse bulkResponse = client().prepareBulk().add(Requests.indexRequest(indexOrAlias()).type("type1").id("1") + .source("field", "value")).execute().actionGet(); + assertThat(bulkResponse.getItems().length, equalTo(1)); + assertThat(bulkResponse.hasFailures(), equalTo(true)); - logger.info("--> verifying get with no routing, should fail"); - for (int i = 0; i < 5; i++) { - try { - client().prepareGet(indexOrAlias(), "type1", "1").execute().actionGet().isExists(); - fail(); - } catch (RoutingMissingException e) { - assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST)); - assertThat(e.getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); + for (BulkItemResponse bulkItemResponse : bulkResponse) { + assertThat(bulkItemResponse.isFailed(), equalTo(true)); + assertThat(bulkItemResponse.getOpType(), equalTo("index")); + assertThat(bulkItemResponse.getFailure().getStatus(), equalTo(RestStatus.BAD_REQUEST)); + assertThat(bulkItemResponse.getFailure().getCause(), instanceOf(RoutingMissingException.class)); + assertThat(bulkItemResponse.getFailureMessage(), containsString("routing is required for [test]/[type1]/[1]")); } } - logger.info("--> verifying get with routing, should find"); - for (int i = 0; i < 5; i++) { - assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); + + { + BulkResponse bulkResponse = client().prepareBulk().add(Requests.indexRequest(indexOrAlias()).type("type1").id("1").routing("0") + .source("field", "value")).execute().actionGet(); + assertThat(bulkResponse.hasFailures(), equalTo(false)); + } + + { + BulkResponse bulkResponse = client().prepareBulk().add(new UpdateRequest(indexOrAlias(), "type1", "1").doc("field", "value2")) + .execute().actionGet(); + assertThat(bulkResponse.getItems().length, equalTo(1)); + assertThat(bulkResponse.hasFailures(), equalTo(true)); + + for (BulkItemResponse bulkItemResponse : bulkResponse) { + assertThat(bulkItemResponse.isFailed(), equalTo(true)); + assertThat(bulkItemResponse.getOpType(), equalTo("update")); + assertThat(bulkItemResponse.getFailure().getStatus(), equalTo(RestStatus.BAD_REQUEST)); + assertThat(bulkItemResponse.getFailure().getCause(), instanceOf(RoutingMissingException.class)); + assertThat(bulkItemResponse.getFailureMessage(), containsString("routing is required for [test]/[type1]/[1]")); + } + } + + { + BulkResponse bulkResponse = client().prepareBulk().add(new UpdateRequest(indexOrAlias(), "type1", "1").doc("field", "value2") + .routing("0")).execute().actionGet(); + assertThat(bulkResponse.hasFailures(), equalTo(false)); + } + + { + BulkResponse bulkResponse = client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1")) + .execute().actionGet(); + assertThat(bulkResponse.getItems().length, equalTo(1)); + assertThat(bulkResponse.hasFailures(), equalTo(true)); + + for (BulkItemResponse bulkItemResponse : bulkResponse) { + assertThat(bulkItemResponse.isFailed(), equalTo(true)); + assertThat(bulkItemResponse.getOpType(), equalTo("delete")); + assertThat(bulkItemResponse.getFailure().getStatus(), equalTo(RestStatus.BAD_REQUEST)); + assertThat(bulkItemResponse.getFailure().getCause(), instanceOf(RoutingMissingException.class)); + assertThat(bulkItemResponse.getFailureMessage(), containsString("routing is required for [test]/[type1]/[1]")); + } + } + + { + BulkResponse bulkResponse = client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1") + .routing("0")).execute().actionGet(); + assertThat(bulkResponse.getItems().length, equalTo(1)); + assertThat(bulkResponse.hasFailures(), equalTo(false)); } } - public void testRequiredRoutingMapping_variousAPIs() throws Exception { + public void testRequiredRoutingMappingVariousAPIs() throws Exception { client().admin().indices().prepareCreate("test").addAlias(new Alias("alias")) .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject()) .execute().actionGet(); diff --git a/docs/java-api/client.asciidoc b/docs/java-api/client.asciidoc index 87aa7291b25..04b6cf49204 100644 --- a/docs/java-api/client.asciidoc +++ b/docs/java-api/client.asciidoc @@ -167,8 +167,6 @@ Client client = TransportClient.builder().settings(settings).build(); //Add transport addresses and do something with the client... -------------------------------------------------- -Or using `elasticsearch.yml` file as shown in <> - The client allows sniffing the rest of the cluster, which adds data nodes into its list of machines to use. In this case, note that the IP addresses used will be the ones that the other nodes were started with (the diff --git a/docs/plugins/discovery-ec2.asciidoc b/docs/plugins/discovery-ec2.asciidoc index 2ca0b8d9314..17659d496b8 100644 --- a/docs/plugins/discovery-ec2.asciidoc +++ b/docs/plugins/discovery-ec2.asciidoc @@ -205,9 +205,9 @@ as valid network host settings: |`_ec2:privateDns_` |The private host of the machine. |`_ec2:publicIpv4_` |The public IP address (ipv4) of the machine. |`_ec2:publicDns_` |The public host of the machine. -|`_ec2:privateIp_` |equivalent to _ec2:privateIpv4_. -|`_ec2:publicIp_` |equivalent to _ec2:publicIpv4_. -|`_ec2_` |equivalent to _ec2:privateIpv4_. +|`_ec2:privateIp_` |equivalent to `_ec2:privateIpv4_`. +|`_ec2:publicIp_` |equivalent to `_ec2:publicIpv4_`. +|`_ec2_` |equivalent to `_ec2:privateIpv4_`. |================================================================== [[discovery-ec2-permissions]] diff --git a/docs/plugins/lang-python.asciidoc b/docs/plugins/lang-python.asciidoc index 7c5595254ee..9146af03381 100644 --- a/docs/plugins/lang-python.asciidoc +++ b/docs/plugins/lang-python.asciidoc @@ -141,10 +141,10 @@ GET test/_search === File scripts You can save your scripts to a file in the `config/scripts/` directory on -every node. The `.python` file suffix identifies the script as containing +every node. The `.py` file suffix identifies the script as containing Python: -First, save this file as `config/scripts/my_script.python` on every node +First, save this file as `config/scripts/my_script.py` on every node in the cluster: [source,python] @@ -188,5 +188,5 @@ GET test/_search ---- // AUTOSENSE -<1> The function score query retrieves the script with filename `my_script.python`. +<1> The function score query retrieves the script with filename `my_script.py`. diff --git a/docs/plugins/transport.asciidoc b/docs/plugins/transport.asciidoc index f19f2614687..3524abd7c2b 100644 --- a/docs/plugins/transport.asciidoc +++ b/docs/plugins/transport.asciidoc @@ -8,7 +8,7 @@ Transport plugins offer alternatives to HTTP. The core transport plugins are: -https://github.com/elasticsearch/elasticsearch-transport-wares::[Servlet transport]:: +https://github.com/elastic/elasticsearch-transport-wares[Servlet transport]:: Use the REST interface over servlets. diff --git a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc index 17cfea93990..bbcc47d82eb 100644 --- a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc @@ -76,7 +76,7 @@ A `serial_diff` aggregation looks like this in isolation: }, "thirtieth_difference": { "serial_diff": { <3> - "buckets_path": "lemmings", + "buckets_path": "the_sum", "lag" : 30 } } diff --git a/docs/reference/mapping/params/fielddata.asciidoc b/docs/reference/mapping/params/fielddata.asciidoc index f8a840080f4..4d96fb61132 100644 --- a/docs/reference/mapping/params/fielddata.asciidoc +++ b/docs/reference/mapping/params/fielddata.asciidoc @@ -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. Most fields can use index-time, on-disk <> to support this type of data access pattern, but `analyzed` string fields do not support diff --git a/docs/reference/search/request/highlighting.asciidoc b/docs/reference/search/request/highlighting.asciidoc index 99742db77c1..58128dd36f8 100644 --- a/docs/reference/search/request/highlighting.asciidoc +++ b/docs/reference/search/request/highlighting.asciidoc @@ -421,6 +421,17 @@ be highlighted regardless of whether the query matched specifically on them. The default behaviour is `true`, meaning that only fields that hold a query match will be highlighted. +[source,js] +-------------------------------------------------- +{ + "query" : {...}, + "highlight" : { + "require_field_match": false + "fields" : {...} + } +} +-------------------------------------------------- + [[boundary-characters]] ==== Boundary Characters diff --git a/modules/lang-groovy/build.gradle b/modules/lang-groovy/build.gradle index 9216001e593..89444a4e926 100644 --- a/modules/lang-groovy/build.gradle +++ b/modules/lang-groovy/build.gradle @@ -23,7 +23,7 @@ esplugin { } dependencies { - compile 'org.codehaus.groovy:groovy:2.4.4:indy' + compile 'org.codehaus.groovy:groovy:2.4.6:indy' } integTest { diff --git a/modules/lang-groovy/licenses/groovy-2.4.4-indy.jar.sha1 b/modules/lang-groovy/licenses/groovy-2.4.4-indy.jar.sha1 deleted file mode 100644 index 30b996327b4..00000000000 --- a/modules/lang-groovy/licenses/groovy-2.4.4-indy.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -139af316ac35534120c53f05393ce46d60d6da48 \ No newline at end of file diff --git a/modules/lang-groovy/licenses/groovy-2.4.6-indy.jar.sha1 b/modules/lang-groovy/licenses/groovy-2.4.6-indy.jar.sha1 new file mode 100644 index 00000000000..0cd895b333c --- /dev/null +++ b/modules/lang-groovy/licenses/groovy-2.4.6-indy.jar.sha1 @@ -0,0 +1 @@ +af78e80fab591a6dcf2d6ccb8bf34d1e888291be \ No newline at end of file diff --git a/modules/lang-groovy/licenses/groovy-NOTICE.txt b/modules/lang-groovy/licenses/groovy-NOTICE.txt index 72eb32a9024..b863c49d312 100644 --- a/modules/lang-groovy/licenses/groovy-NOTICE.txt +++ b/modules/lang-groovy/licenses/groovy-NOTICE.txt @@ -1,5 +1,5 @@ -Apache Commons CLI -Copyright 2001-2009 The Apache Software Foundation +Apache Groovy +Copyright 2003-2016 The Apache Software Foundation -This product includes software developed by +This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/modules/lang-groovy/src/main/plugin-metadata/plugin-security.policy b/modules/lang-groovy/src/main/plugin-metadata/plugin-security.policy index ec749cad45b..a2318191540 100644 --- a/modules/lang-groovy/src/main/plugin-metadata/plugin-security.policy +++ b/modules/lang-groovy/src/main/plugin-metadata/plugin-security.policy @@ -25,7 +25,6 @@ grant { // needed by groovy engine permission java.lang.RuntimePermission "accessDeclaredMembers"; permission java.lang.RuntimePermission "accessClassInPackage.sun.reflect"; - permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; // needed by GroovyScriptEngineService to close its classloader (why?) permission java.lang.RuntimePermission "closeClassLoader"; // Allow executing groovy scripts with codesource of /untrusted diff --git a/plugins/ingest-geoip/build.gradle b/plugins/ingest-geoip/build.gradle index 7eee668793c..eee032d6d16 100644 --- a/plugins/ingest-geoip/build.gradle +++ b/plugins/ingest-geoip/build.gradle @@ -23,11 +23,11 @@ esplugin { } dependencies { - compile ('com.maxmind.geoip2:geoip2:2.4.0') + compile ('com.maxmind.geoip2:geoip2:2.6.0') // geoip2 dependencies: - compile('com.fasterxml.jackson.core:jackson-annotations:2.5.0') - compile('com.fasterxml.jackson.core:jackson-databind:2.5.3') - compile('com.maxmind.db:maxmind-db:1.0.1') + compile('com.fasterxml.jackson.core:jackson-annotations:2.7.1') + compile('com.fasterxml.jackson.core:jackson-databind:2.7.1') + compile('com.maxmind.db:maxmind-db:1.2.0') testCompile 'org.elasticsearch:geolite2-databases:20151029' } @@ -60,4 +60,4 @@ thirdPartyAudit.excludes = [ 'com.google.api.client.http.HttpResponseException', 'com.google.api.client.http.javanet.NetHttpTransport', 'com.google.api.client.http.javanet.NetHttpTransport', -] \ No newline at end of file +] diff --git a/plugins/ingest-geoip/licenses/geoip2-2.4.0.jar.sha1 b/plugins/ingest-geoip/licenses/geoip2-2.4.0.jar.sha1 deleted file mode 100644 index 485286f06a4..00000000000 --- a/plugins/ingest-geoip/licenses/geoip2-2.4.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ad40667ae87138e0aed075d2c15884497fa64acc \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/geoip2-2.6.0.jar.sha1 b/plugins/ingest-geoip/licenses/geoip2-2.6.0.jar.sha1 new file mode 100644 index 00000000000..ba5a0242ec8 --- /dev/null +++ b/plugins/ingest-geoip/licenses/geoip2-2.6.0.jar.sha1 @@ -0,0 +1 @@ +2574c8b878f1cd39709559f1b96f1b5f0cdd69d3 \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/jackson-annotations-2.5.0.jar.sha1 b/plugins/ingest-geoip/licenses/jackson-annotations-2.5.0.jar.sha1 deleted file mode 100644 index 862ac6f304f..00000000000 --- a/plugins/ingest-geoip/licenses/jackson-annotations-2.5.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a2a55a3375bc1cef830ca426d68d2ea22961190e diff --git a/plugins/ingest-geoip/licenses/jackson-annotations-2.7.1.jar.sha1 b/plugins/ingest-geoip/licenses/jackson-annotations-2.7.1.jar.sha1 new file mode 100644 index 00000000000..69b45742d84 --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-annotations-2.7.1.jar.sha1 @@ -0,0 +1 @@ +8b93f301823b79033fcbe873779b3d84f9730fc1 \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/jackson-databind-2.5.3.jar.sha1 b/plugins/ingest-geoip/licenses/jackson-databind-2.5.3.jar.sha1 deleted file mode 100644 index cdc66958059..00000000000 --- a/plugins/ingest-geoip/licenses/jackson-databind-2.5.3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c37875ff66127d93e5f672708cb2dcc14c8232ab diff --git a/plugins/ingest-geoip/licenses/jackson-databind-2.7.1.jar.sha1 b/plugins/ingest-geoip/licenses/jackson-databind-2.7.1.jar.sha1 new file mode 100644 index 00000000000..d9b4ca6a79b --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-databind-2.7.1.jar.sha1 @@ -0,0 +1 @@ +14d88822bca655de7aa6ed3e4c498d115505710a \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/maxmind-db-1.0.1.jar.sha1 b/plugins/ingest-geoip/licenses/maxmind-db-1.0.1.jar.sha1 deleted file mode 100644 index 6cb749e35ae..00000000000 --- a/plugins/ingest-geoip/licenses/maxmind-db-1.0.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -305429b84dbcd1cc3d393686f412cdcaec9cdbe6 \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/maxmind-db-1.2.0.jar.sha1 b/plugins/ingest-geoip/licenses/maxmind-db-1.2.0.jar.sha1 new file mode 100644 index 00000000000..ae48403db39 --- /dev/null +++ b/plugins/ingest-geoip/licenses/maxmind-db-1.2.0.jar.sha1 @@ -0,0 +1 @@ +b842823f24555f5d26608fef8122898365b3cd63 \ No newline at end of file