diff --git a/docs/reference/ilm/apis/get-lifecycle.asciidoc b/docs/reference/ilm/apis/get-lifecycle.asciidoc index 9bdf14d970caa..a36703967af67 100644 --- a/docs/reference/ilm/apis/get-lifecycle.asciidoc +++ b/docs/reference/ilm/apis/get-lifecycle.asciidoc @@ -69,7 +69,7 @@ PUT _ilm/policy/my_policy [source,js] -------------------------------------------------- -GET _ilm/policy +GET _ilm/policy/my_policy -------------------------------------------------- // CONSOLE // TEST[continued] diff --git a/docs/reference/ilm/update-lifecycle-policy.asciidoc b/docs/reference/ilm/update-lifecycle-policy.asciidoc index 3e6627fdd3a7e..bc41da6bdff63 100644 --- a/docs/reference/ilm/update-lifecycle-policy.asciidoc +++ b/docs/reference/ilm/update-lifecycle-policy.asciidoc @@ -87,7 +87,7 @@ PUT _ilm/policy/my_policy ////////// [source,js] -------------------------------------------------- -GET _ilm/policy +GET _ilm/policy/my_policy -------------------------------------------------- // CONSOLE // TEST[continued] diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java index f4fa0176cb9e5..9fc35dc7be38a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -155,6 +155,7 @@ private void buildResponse(final ClusterStateRequest request, if (request.metaData()) { if (request.indices().length > 0) { + mdBuilder.version(currentState.metaData().version()); String[] indices = indexNameExpressionResolver.concreteIndexNames(currentState, request); for (String filteredIndex : indices) { IndexMetaData indexMetaData = currentState.metaData().index(filteredIndex); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java index 926ae175d65ad..6868708145359 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java @@ -254,7 +254,7 @@ public static XContentBuilder buildFromSimplifiedDef(String type, Object... sour * The mapping source definition. */ public PutMappingRequest source(XContentBuilder mappingBuilder) { - return source(Strings.toString(mappingBuilder), mappingBuilder.contentType()); + return source(BytesReference.bytes(mappingBuilder), mappingBuilder.contentType()); } /** @@ -264,7 +264,7 @@ public PutMappingRequest source(Map mappingSource) { try { XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); builder.map(mappingSource); - return source(Strings.toString(builder), XContentType.JSON); + return source(BytesReference.bytes(builder), builder.contentType()); } catch (IOException e) { throw new ElasticsearchGenerationException("Failed to generate [" + mappingSource + "]", e); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java index ce82d277dbbd7..0ff5785fcd358 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java @@ -461,7 +461,7 @@ public void readFrom(StreamInput in) throws IOException { name = in.readString(); if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - indexPatterns = in.readList(StreamInput::readString); + indexPatterns = in.readStringList(); } else { indexPatterns = Collections.singletonList(in.readString()); } @@ -495,7 +495,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(cause); out.writeString(name); if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - out.writeStringList(indexPatterns); + out.writeStringCollection(indexPatterns); } else { out.writeString(indexPatterns.size() > 0 ? indexPatterns.get(0) : ""); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java index 82eb26d98b97e..b754e50a945c1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java @@ -188,8 +188,8 @@ protected ClusterState.Builder becomeMasterAndTrimConflictingNodes(ClusterState .blocks(currentState.blocks()) .removeGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID)).build(); logger.trace("becomeMasterAndTrimConflictingNodes: {}", tmpState.nodes()); - tmpState = PersistentTasksCustomMetaData.deassociateDeadNodes(tmpState); - return ClusterState.builder(allocationService.deassociateDeadNodes(tmpState, false, "removed dead nodes on election")); + tmpState = PersistentTasksCustomMetaData.disassociateDeadNodes(tmpState); + return ClusterState.builder(allocationService.disassociateDeadNodes(tmpState, false, "removed dead nodes on election")); } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutor.java b/server/src/main/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutor.java index 9d6051d0ccf03..bbfe64988c7cf 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutor.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutor.java @@ -92,9 +92,9 @@ public ClusterTasksResult execute(final ClusterState currentState, final L protected ClusterTasksResult getTaskClusterTasksResult(ClusterState currentState, List tasks, ClusterState remainingNodesClusterState) { - ClusterState ptasksDeassociatedState = PersistentTasksCustomMetaData.deassociateDeadNodes(remainingNodesClusterState); + ClusterState ptasksDisassociatedState = PersistentTasksCustomMetaData.disassociateDeadNodes(remainingNodesClusterState); final ClusterTasksResult.Builder resultBuilder = ClusterTasksResult.builder().successes(tasks); - return resultBuilder.build(allocationService.deassociateDeadNodes(ptasksDeassociatedState, true, describeTasks(tasks))); + return resultBuilder.build(allocationService.disassociateDeadNodes(ptasksDisassociatedState, true, describeTasks(tasks))); } // visible for testing diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index e21aa95865a6f..e66b55b1a7ed6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -189,7 +189,7 @@ public static IndexTemplateMetaData readFrom(StreamInput in) throws IOException Builder builder = new Builder(in.readString()); builder.order(in.readInt()); if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - builder.patterns(in.readList(StreamInput::readString)); + builder.patterns(in.readStringList()); } else { builder.patterns(Collections.singletonList(in.readString())); } @@ -224,7 +224,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(name); out.writeInt(order); if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - out.writeStringList(patterns); + out.writeStringCollection(patterns); } else { out.writeString(patterns.get(0)); } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java index 9349de94a66e2..c9cc5b628e292 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java @@ -45,7 +45,7 @@ * the delay marker). These are shards that have become unassigned due to a node leaving * and which were assigned the delay marker based on the index delay setting * {@link UnassignedInfo#INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING} - * (see {@link AllocationService#deassociateDeadNodes(RoutingAllocation)}). + * (see {@link AllocationService#disassociateDeadNodes(RoutingAllocation)}. * This class is responsible for choosing the next (closest) delay expiration of a * delayed shard to schedule a reroute to remove the delay marker. * The actual removal of the delay marker happens in diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 7e8fdb1edaca2..af617dccf6734 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -327,7 +327,7 @@ public ShardRouting activePrimary(ShardId shardId) { * */ public ShardRouting activeReplicaWithHighestVersion(ShardId shardId) { - // It's possible for replicaNodeVersion to be null, when deassociating dead nodes + // It's possible for replicaNodeVersion to be null, when disassociating dead nodes // that have been removed, the shards are failed, and part of the shard failing // calls this method with an out-of-date RoutingNodes, where the version might not // be accessible. Therefore, we need to protect against the version being null diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java index 57f7a313942bb..89e19e02b30ed 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java @@ -77,11 +77,6 @@ protected void doClose() { * Initiates a reroute. */ public final void reroute(String reason) { - performReroute(reason); - } - - // visible for testing - protected void performReroute(String reason) { try { if (lifecycle.stopped()) { return; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 7acf20185eedd..fd3fb8edd5e17 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -216,7 +216,7 @@ public ClusterState applyFailedShards(final ClusterState clusterState, final Lis * unassigned an shards that are associated with nodes that are no longer part of the cluster, potentially promoting replicas * if needed. */ - public ClusterState deassociateDeadNodes(ClusterState clusterState, boolean reroute, String reason) { + public ClusterState disassociateDeadNodes(ClusterState clusterState, boolean reroute, String reason) { RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); @@ -224,7 +224,7 @@ public ClusterState deassociateDeadNodes(ClusterState clusterState, boolean rero clusterInfoService.getClusterInfo(), currentNanoTime()); // first, clear from the shards any node id they used to belong to that is now dead - deassociateDeadNodes(allocation); + disassociateDeadNodes(allocation); if (allocation.routingNodesChanged()) { clusterState = buildResult(clusterState, allocation); @@ -400,7 +400,7 @@ private boolean hasDeadNodes(RoutingAllocation allocation) { } private void reroute(RoutingAllocation allocation) { - assert hasDeadNodes(allocation) == false : "dead nodes should be explicitly cleaned up. See deassociateDeadNodes"; + assert hasDeadNodes(allocation) == false : "dead nodes should be explicitly cleaned up. See disassociateDeadNodes"; assert AutoExpandReplicas.getAutoExpandReplicaChanges(allocation.metaData(), allocation.nodes()).isEmpty() : "auto-expand replicas out of sync with number of nodes in the cluster"; @@ -414,7 +414,7 @@ private void reroute(RoutingAllocation allocation) { assert RoutingNodes.assertShardStats(allocation.routingNodes()); } - private void deassociateDeadNodes(RoutingAllocation allocation) { + private void disassociateDeadNodes(RoutingAllocation allocation) { for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext(); ) { RoutingNode node = it.next(); if (allocation.nodes().getDataNodes().containsKey(node.nodeId())) { diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java index 7759e13e536b7..723de8fd5da31 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java @@ -946,12 +946,26 @@ public List readStreamableList(Supplier constructor } /** - * Reads a list of objects + * Reads a list of objects. The list is expected to have been written using {@link StreamOutput#writeList(List)} or + * {@link StreamOutput#writeStreamableList(List)}. + * + * @return the list of objects + * @throws IOException if an I/O exception occurs reading the list */ - public List readList(Writeable.Reader reader) throws IOException { + public List readList(final Writeable.Reader reader) throws IOException { return readCollection(reader, ArrayList::new); } + /** + * Reads a list of strings. The list is expected to have been written using {@link StreamOutput#writeStringCollection(Collection)}. + * + * @return the list of strings + * @throws IOException if an I/O exception occurs reading the list + */ + public List readStringList() throws IOException { + return readList(StreamInput::readString); + } + /** * Reads a set of objects */ diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java index 951bcf3c32c49..e9709de1a44a3 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java @@ -1048,27 +1048,27 @@ public void writeList(List list) throws IOException { } /** - * Writes a collection of generic objects via a {@link Writer} + * Writes a collection of objects via a {@link Writer}. + * + * @param collection the collection of objects + * @throws IOException if an I/O exception occurs writing the collection */ - public void writeCollection(Collection collection, Writer writer) throws IOException { + public void writeCollection(final Collection collection, final Writer writer) throws IOException { writeVInt(collection.size()); - for (T val: collection) { + for (final T val: collection) { writer.write(this, val); } } - public void writeStringCollection(final Collection list) throws IOException { - writeVInt(list.size()); - for (String string: list) { - this.writeString(string); - } - } - /** - * Writes a list of strings + * Writes a collection of a strings. The corresponding collection can be read from a stream input using + * {@link StreamInput#readList(Writeable.Reader)}. + * + * @param collection the collection of strings + * @throws IOException if an I/O exception occurs writing the collection */ - public void writeStringList(List list) throws IOException { - writeStringCollection(list); + public void writeStringCollection(final Collection collection) throws IOException { + writeCollection(collection, StreamOutput::writeString); } /** diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java index 605d202cbaf9b..acac7e732b38b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java @@ -60,9 +60,9 @@ public final class RecoveryResponse extends TransportResponse { RecoveryResponse(StreamInput in) throws IOException { super(in); - phase1FileNames = in.readList(StreamInput::readString); + phase1FileNames = in.readStringList(); phase1FileSizes = in.readList(StreamInput::readVLong); - phase1ExistingFileNames = in.readList(StreamInput::readString); + phase1ExistingFileNames = in.readStringList(); phase1ExistingFileSizes = in.readList(StreamInput::readVLong); phase1TotalSize = in.readVLong(); phase1ExistingTotalSize = in.readVLong(); @@ -76,9 +76,9 @@ public final class RecoveryResponse extends TransportResponse { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeStringList(phase1FileNames); + out.writeStringCollection(phase1FileNames); out.writeCollection(phase1FileSizes, StreamOutput::writeVLong); - out.writeStringList(phase1ExistingFileNames); + out.writeStringCollection(phase1ExistingFileNames); out.writeCollection(phase1ExistingFileSizes, StreamOutput::writeVLong); out.writeVLong(phase1TotalSize); out.writeVLong(phase1ExistingTotalSize); diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java index 71918cadb55d8..fb914a34bfbaa 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java @@ -247,6 +247,45 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS }); } + /** + * This unassigns a task from any node, i.e. it is assigned to a {@code null} node with the provided reason. + * + * Since the assignment executor node is null, the {@link PersistentTasksClusterService} will attempt to reassign it to a valid + * node quickly. + * + * @param taskId the id of a persistent task + * @param taskAllocationId the expected allocation id of the persistent task + * @param reason the reason for unassigning the task from any node + * @param listener the listener that will be called when task is unassigned + */ + public void unassignPersistentTask(final String taskId, + final long taskAllocationId, + final String reason, + final ActionListener> listener) { + clusterService.submitStateUpdateTask("unassign persistent task from any node", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + PersistentTasksCustomMetaData.Builder tasksInProgress = builder(currentState); + if (tasksInProgress.hasTask(taskId, taskAllocationId)) { + logger.trace("Unassigning task {} with allocation id {}", taskId, taskAllocationId); + return update(currentState, tasksInProgress.reassignTask(taskId, unassignedAssignment(reason))); + } else { + throw new ResourceNotFoundException("the task with id {} and allocation id {} doesn't exist", taskId, taskAllocationId); + } + } + + @Override + public void onFailure(String source, Exception e) { + listener.onFailure(e); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + listener.onResponse(PersistentTasksCustomMetaData.getTaskWithId(newState, taskId)); + } + }); + } + /** * Creates a new {@link Assignment} for the given persistent task. * @@ -263,7 +302,7 @@ private Assignment createAssignment(final AssignmentDecision decision = decider.canAssign(); if (decision.getType() == AssignmentDecision.Type.NO) { - return new Assignment(null, "persistent task [" + taskName + "] cannot be assigned [" + decision.getReason() + "]"); + return unassignedAssignment("persistent task [" + taskName + "] cannot be assigned [" + decision.getReason() + "]"); } return persistentTasksExecutor.getAssignment(taskParams, currentState); @@ -404,6 +443,10 @@ private static ClusterState update(ClusterState currentState, PersistentTasksCus } } + private static Assignment unassignedAssignment(String reason) { + return new Assignment(null, reason); + } + /** * Class to periodically try to reassign unassigned persistent tasks. */ diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java index 01e3d7450a60d..9f31c6348a0a1 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java @@ -225,7 +225,7 @@ public static PersistentTask getTa * @return If no changes the argument {@code clusterState} is returned else * a copy with the modified tasks */ - public static ClusterState deassociateDeadNodes(ClusterState clusterState) { + public static ClusterState disassociateDeadNodes(ClusterState clusterState) { PersistentTasksCustomMetaData tasks = getPersistentTasksCustomMetaData(clusterState); if (tasks == null) { return clusterState; diff --git a/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java b/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java index d211efef5173e..3efd149be19c6 100644 --- a/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java +++ b/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java @@ -103,7 +103,7 @@ public PluginInfo(final StreamInput in) throws IOException { } this.classname = in.readString(); if (in.getVersion().onOrAfter(Version.V_6_2_0)) { - extendedPlugins = in.readList(StreamInput::readString); + extendedPlugins = in.readStringList(); } else { extendedPlugins = Collections.emptyList(); } @@ -128,7 +128,7 @@ public void writeTo(final StreamOutput out) throws IOException { } out.writeString(classname); if (out.getVersion().onOrAfter(Version.V_6_2_0)) { - out.writeStringList(extendedPlugins); + out.writeStringCollection(extendedPlugins); } out.writeBoolean(hasNativeController); if (out.getVersion().onOrAfter(Version.V_6_0_0_beta2) && out.getVersion().before(Version.V_6_3_0)) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java index 822fdd7ba31a5..c5a1ab50b8230 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java @@ -69,7 +69,7 @@ public class InternalComposite public InternalComposite(StreamInput in) throws IOException { super(in); this.size = in.readVInt(); - this.sourceNames = in.readList(StreamInput::readString); + this.sourceNames = in.readStringList(); this.formats = new ArrayList<>(sourceNames.size()); for (int i = 0; i < sourceNames.size(); i++) { if (in.getVersion().onOrAfter(Version.V_6_3_0)) { @@ -90,7 +90,7 @@ public InternalComposite(StreamInput in) throws IOException { @Override protected void doWriteTo(StreamOutput out) throws IOException { out.writeVInt(size); - out.writeStringList(sourceNames); + out.writeStringCollection(sourceNames); if (out.getVersion().onOrAfter(Version.V_6_3_0)) { for (DocValueFormat format : formats) { out.writeNamedWriteable(format); diff --git a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index d7867af9ea139..8e072d36c2c33 100644 --- a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -243,7 +243,7 @@ public SearchSourceBuilder(StreamInput in) throws IOException { } } if (in.readBoolean()) { - stats = in.readList(StreamInput::readString); + stats = in.readStringList(); } suggestBuilder = in.readOptionalWriteable(SuggestBuilder::new); terminateAfter = in.readVInt(); @@ -311,7 +311,7 @@ public void writeTo(StreamOutput out) throws IOException { boolean hasStats = stats != null; out.writeBoolean(hasStats); if (hasStats) { - out.writeStringList(stats); + out.writeStringCollection(stats); } out.writeOptionalWriteable(suggestBuilder); out.writeVInt(terminateAfter); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java index 6fddf2dd5f85b..593cdc60e8a23 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; @@ -135,20 +134,8 @@ private static PutMappingRequest createTestItem() throws IOException { String type = randomAlphaOfLength(5); request.type(type); - request.source(randomMapping()); + request.source(RandomCreateIndexGenerator.randomMapping()); return request; } - - private static XContentBuilder randomMapping() throws IOException { - XContentBuilder builder = XContentFactory.jsonBuilder(); - builder.startObject(); - - if (randomBoolean()) { - RandomCreateIndexGenerator.randomMappingFields(builder, true); - } - - builder.endObject(); - return builder; - } } diff --git a/server/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java b/server/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java index 14919b7e9f0bc..8025d588820f9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java @@ -66,6 +66,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertIndexTemplateExists; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; @@ -118,6 +119,23 @@ public void testMetadata() throws Exception { assertThat(clusterStateResponse.getState().metaData().indices().size(), is(0)); } + public void testMetadataVersion() { + createIndex("index-1"); + createIndex("index-2"); + long metadataVersion = client().admin().cluster().prepareState().get().getState().metaData().version(); + assertThat(metadataVersion, greaterThan(0L)); + assertThat(client().admin().cluster().prepareState().setIndices("index-1").get().getState().metaData().version(), + equalTo(metadataVersion)); + assertThat(client().admin().cluster().prepareState().setIndices("index-2").get().getState().metaData().version(), + equalTo(metadataVersion)); + assertThat(client().admin().cluster().prepareState().setIndices("*").get().getState().metaData().version(), + equalTo(metadataVersion)); + assertThat(client().admin().cluster().prepareState().setIndices("not-found").get().getState().metaData().version(), + equalTo(metadataVersion)); + assertThat(client().admin().cluster().prepareState().clear().setMetaData(false).get().getState().metaData().version(), + equalTo(0L)); + } + public void testIndexTemplates() throws Exception { client().admin().indices().preparePutTemplate("foo_template") .setPatterns(Collections.singletonList("te*")) diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutorTests.java index 65df746789c89..e63e4e01196fb 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeRemovalClusterStateTaskExecutorTests.java @@ -70,7 +70,7 @@ public void testRemovingNonExistentNodes() throws Exception { public void testRerouteAfterRemovingNodes() throws Exception { final AllocationService allocationService = mock(AllocationService.class); - when(allocationService.deassociateDeadNodes(any(ClusterState.class), eq(true), any(String.class))) + when(allocationService.disassociateDeadNodes(any(ClusterState.class), eq(true), any(String.class))) .thenAnswer(im -> im.getArguments()[0]); final AtomicReference remainingNodesClusterState = new AtomicReference<>(); @@ -102,7 +102,7 @@ protected ClusterState remainingNodesClusterState(ClusterState currentState, final ClusterStateTaskExecutor.ClusterTasksResult result = executor.execute(clusterState, tasks); - verify(allocationService).deassociateDeadNodes(eq(remainingNodesClusterState.get()), eq(true), any(String.class)); + verify(allocationService).disassociateDeadNodes(eq(remainingNodesClusterState.get()), eq(true), any(String.class)); for (final NodeRemovalClusterStateTaskExecutor.Task task : tasks) { assertNull(result.resultingState.nodes().get(task.node().getId())); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java index b168bae87de41..6897380ee9f01 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java @@ -104,7 +104,7 @@ public void testNoDelayedUnassigned() throws Exception { nodes.add(newNode("node3")); } clusterState = ClusterState.builder(clusterState).nodes(nodes).build(); - clusterState = allocationService.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocationService.disassociateDeadNodes(clusterState, true, "reroute"); ClusterState newState = clusterState; List unassignedShards = newState.getRoutingTable().shardsWithState(ShardRoutingState.UNASSIGNED); if (nodeAvailableForAllocation) { @@ -153,7 +153,7 @@ public void testDelayedUnassignedScheduleReroute() throws Exception { // remove node that has replica and reroute clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove(nodeId)).build(); - clusterState = allocationService.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocationService.disassociateDeadNodes(clusterState, true, "reroute"); ClusterState stateWithDelayedShard = clusterState; // make sure the replica is marked as delayed (i.e. not reallocated) assertEquals(1, UnassignedInfo.getNumberOfDelayedUnassigned(stateWithDelayedShard)); @@ -264,7 +264,7 @@ public void testDelayedUnassignedScheduleRerouteAfterDelayedReroute() throws Exc .build(); // make sure both replicas are marked as delayed (i.e. not reallocated) allocationService.setNanoTimeOverride(baseTimestampNanos); - clusterState = allocationService.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocationService.disassociateDeadNodes(clusterState, true, "reroute"); final ClusterState stateWithDelayedShards = clusterState; assertEquals(2, UnassignedInfo.getNumberOfDelayedUnassigned(stateWithDelayedShards)); RoutingNodes.UnassignedShards.UnassignedIterator iter = stateWithDelayedShards.getRoutingNodes().unassigned().iterator(); @@ -401,7 +401,7 @@ public void testDelayedUnassignedScheduleRerouteRescheduledOnShorterDelay() thro // remove node that has replica and reroute clusterState = ClusterState.builder(clusterState).nodes( DiscoveryNodes.builder(clusterState.nodes()).remove(nodeIdOfFooReplica)).build(); - clusterState = allocationService.deassociateDeadNodes(clusterState, true, "fake node left"); + clusterState = allocationService.disassociateDeadNodes(clusterState, true, "fake node left"); ClusterState stateWithDelayedShard = clusterState; // make sure the replica is marked as delayed (i.e. not reallocated) assertEquals(1, UnassignedInfo.getNumberOfDelayedUnassigned(stateWithDelayedShard)); @@ -444,7 +444,7 @@ public void testDelayedUnassignedScheduleRerouteRescheduledOnShorterDelay() thro // remove node that has replica and reroute clusterState = ClusterState.builder(stateWithDelayedShard).nodes( DiscoveryNodes.builder(stateWithDelayedShard.nodes()).remove(nodeIdOfBarReplica)).build(); - ClusterState stateWithShorterDelay = allocationService.deassociateDeadNodes(clusterState, true, "fake node left"); + ClusterState stateWithShorterDelay = allocationService.disassociateDeadNodes(clusterState, true, "fake node left"); delayedAllocationService.setNanoTimeOverride(clusterChangeEventTimestampNanos); delayedAllocationService.clusterChanged( new ClusterChangedEvent("fake node left", stateWithShorterDelay, stateWithDelayedShard)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/RoutingServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/RoutingServiceTests.java deleted file mode 100644 index f9d3e0ab95c01..0000000000000 --- a/server/src/test/java/org/elasticsearch/cluster/routing/RoutingServiceTests.java +++ /dev/null @@ -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.cluster.routing; - -import org.elasticsearch.cluster.ESAllocationTestCase; -import org.junit.Before; - -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.hamcrest.Matchers.equalTo; - -public class RoutingServiceTests extends ESAllocationTestCase { - - private TestRoutingService routingService; - - @Before - public void createRoutingService() { - routingService = new TestRoutingService(); - } - - public void testReroute() { - assertThat(routingService.hasReroutedAndClear(), equalTo(false)); - routingService.reroute("test"); - assertThat(routingService.hasReroutedAndClear(), equalTo(true)); - } - - private class TestRoutingService extends RoutingService { - - private AtomicBoolean rerouted = new AtomicBoolean(); - - TestRoutingService() { - super(null, null); - } - - public boolean hasReroutedAndClear() { - return rerouted.getAndSet(false); - } - - @Override - protected void performReroute(String reason) { - logger.info("--> performing fake reroute [{}]", reason); - rerouted.set(true); - } - } -} diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 9cf4ef447093d..16cde1a990907 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -241,7 +241,7 @@ public void testNodeLeave() { assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(false)); // remove node2 and reroute clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); // verify that NODE_LEAVE is the reason for meta assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(true)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1)); @@ -332,7 +332,7 @@ public void testNumberOfDelayedUnassigned() throws Exception { // remove node2 and reroute clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build(); // make sure both replicas are marked as delayed (i.e. not reallocated) - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); assertThat(clusterState.toString(), UnassignedInfo.getNumberOfDelayedUnassigned(clusterState), equalTo(2)); } @@ -364,7 +364,7 @@ public void testFindNextDelayedAllocation() { final long baseTime = System.nanoTime(); allocation.setNanoTimeOverride(baseTime); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); final long delta = randomBoolean() ? 0 : randomInt((int) expectMinDelaySettingsNanos - 1); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java index b3e0146bd7f1d..d2ef2de5458cb 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java @@ -341,7 +341,7 @@ private ClusterState removeNodes(ClusterState clusterState, AllocationService se } clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build(); - clusterState = service.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = service.disassociateDeadNodes(clusterState, true, "reroute"); logger.info("start all the primary shards, replicas will start initializing"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java index 5c2bfbadd2165..10fa01c2a2b12 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java @@ -181,7 +181,7 @@ private ClusterState removeNodes(ClusterState clusterState, AllocationService st clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build(); if (removed) { - clusterState = strategy.deassociateDeadNodes(clusterState, randomBoolean(), "removed nodes"); + clusterState = strategy.disassociateDeadNodes(clusterState, randomBoolean(), "removed nodes"); } logger.info("start all the primary shards, replicas will start initializing"); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java index 1ee136b77b0b8..f9dee9807b400 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java @@ -84,7 +84,7 @@ public void testSimpleDeadNodeOnStartedPrimaryShard() { .add(newNode(nodeIdRemaining)) ).build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); assertThat(clusterState.getRoutingNodes().node(nodeIdRemaining).iterator().next().primary(), equalTo(true)); assertThat(clusterState.getRoutingNodes().node(nodeIdRemaining).iterator().next().state(), equalTo(STARTED)); @@ -154,7 +154,7 @@ public void testDeadNodeWhileRelocatingOnToNode() { .add(newNode(origPrimaryNodeId)) .add(newNode(origReplicaNodeId)) ).build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).iterator().next().state(), equalTo(STARTED)); assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).iterator().next().state(), equalTo(STARTED)); @@ -224,7 +224,7 @@ public void testDeadNodeWhileRelocatingOnFromNode() { .add(newNode("node3")) .add(newNode(origReplicaNodeId)) ).build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).iterator().next().state(), equalTo(STARTED)); assertThat(clusterState.getRoutingNodes().node("node3").iterator().next().state(), equalTo(INITIALIZING)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java index fa28b4b1e3482..74e7ac3273634 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java @@ -96,7 +96,7 @@ public void testElectReplicaAsPrimaryDuringRelocation() { indexShardRoutingTable.primaryShard().currentNodeId()); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(indexShardRoutingTable.primaryShard().currentNodeId())).build(); - clusterState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); logger.info("make sure all the primary shards are active"); assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().active(), equalTo(true)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java index 3e07d732db7b8..71a31b320591a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java @@ -111,7 +111,7 @@ public void testSimpleFailedNodeTest() { .remove(clusterState.routingTable().index("test2").shard(0).primaryShard().currentNodeId()) ) .build(); - clusterState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); routingNodes = clusterState.getRoutingNodes(); for (RoutingNode routingNode : routingNodes) { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java index 8630d2cc5b6d9..5e1ff70d9d68e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java @@ -101,7 +101,7 @@ public void testInSyncAllocationIdsUpdated() { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove("node1")) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); // in-sync allocation ids should not be updated assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(3)); @@ -110,7 +110,7 @@ public void testInSyncAllocationIdsUpdated() { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove("node2").remove("node3")) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); // in-sync allocation ids should not be updated assertThat(clusterState.getRoutingTable().shardsWithState(UNASSIGNED).size(), equalTo(3)); @@ -145,7 +145,7 @@ public void testInSyncAllocationIdsUpdated() { /** * Assume following scenario: indexing request is written to primary, but fails to be replicated to active replica. * The primary instructs master to fail replica before acknowledging write to client. In the meanwhile, the node of the replica was - * removed from the cluster (deassociateDeadNodes). This means that the ShardRouting of the replica was failed, but it's allocation + * removed from the cluster (disassociateDeadNodes). This means that the ShardRouting of the replica was failed, but it's allocation * id is still part of the in-sync set. We have to make sure that the failShard request from the primary removes the allocation id * from the in-sync set. */ @@ -158,7 +158,7 @@ public void testDeadNodesBeforeReplicaFailed() throws Exception { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(replicaShard.currentNodeId())) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(2)); logger.info("fail replica (for which there is no shard routing in the CS anymore)"); @@ -228,7 +228,7 @@ public void testInSyncIdsNotGrowingWithoutBounds() throws Exception { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(replicaShard.currentNodeId())) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); // in-sync allocation ids should not be updated assertEquals(inSyncSet, clusterState.metaData().index("test").inSyncAllocationIds(0)); @@ -248,7 +248,7 @@ public void testInSyncIdsNotGrowingWithoutBounds() throws Exception { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(replicaShard.currentNodeId())) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); } // in-sync allocation set is bounded @@ -276,7 +276,7 @@ public void testInSyncIdsNotTrimmedWhenNotGrowing() throws Exception { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(replicaShard.currentNodeId())) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); // in-sync allocation ids should not be updated assertEquals(inSyncSet, clusterState.metaData().index("test").inSyncAllocationIds(0)); @@ -285,7 +285,7 @@ public void testInSyncIdsNotTrimmedWhenNotGrowing() throws Exception { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(primaryShard.currentNodeId())) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); // in-sync allocation ids should not be updated assertEquals(inSyncSet, clusterState.metaData().index("test").inSyncAllocationIds(0)); @@ -327,7 +327,7 @@ public void testPrimaryAllocationIdNotRemovedFromInSyncSetWhenNoFailOver() throw clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(replicaShard.currentNodeId())) .build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); // in-sync allocation ids should not be updated assertEquals(inSyncSet, clusterState.metaData().index("test").inSyncAllocationIds(0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index e59b492606a06..f0d25175f6e43 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -388,7 +388,7 @@ public void testRestoreDoesNotAllocateSnapshotOnOlderNodes() { private ClusterState stabilize(ClusterState clusterState, AllocationService service) { logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes()); - clusterState = service.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = service.disassociateDeadNodes(clusterState, true, "reroute"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertRecoveryNodeVersions(routingNodes); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java index b47638884e883..d80feedabf0e6 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java @@ -94,7 +94,7 @@ public void testPreferLocalPrimaryAllocationOverFiltered() { .build(); clusterState = ClusterState.builder(clusterState).metaData(metaData) .nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build(); - clusterState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); logger.info("[{}] primaries should be still started but [{}] other primaries should be unassigned", numberOfShards, numberOfShards); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java index d6220ade9ef34..f06a38a2ba462 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java @@ -77,7 +77,7 @@ public void testBackupElectionToPrimaryWhenPrimaryCanBeAllocatedToAnotherNode() clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .add(newNode("node3")).remove("node1")).build(); RoutingTable prevRoutingTable = clusterState.routingTable(); - clusterState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); routingNodes = clusterState.getRoutingNodes(); routingTable = clusterState.routingTable(); @@ -129,7 +129,7 @@ public void testRemovingInitializingReplicasIfPrimariesFails() { String nodeIdToFail = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(); String nodeIdRemaining = nodeIdToFail.equals("node1") ? "node2" : "node1"; clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode(nodeIdRemaining))).build(); - clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = allocation.disassociateDeadNodes(clusterState, true, "reroute"); routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(1)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java index fe7c4a89c9fa2..0c8ebff594598 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java @@ -130,7 +130,7 @@ public void testRandomDecisions() { stateBuilder.nodes(newNodesBuilder.build()); clusterState = stateBuilder.build(); if (nodesRemoved) { - clusterState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); } else { clusterState = strategy.reroute(clusterState, "reroute"); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java index deed0b2b2fb72..72364e3fbc925 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java @@ -334,7 +334,7 @@ public void testBalanceAllNodesStartedAddIndex() { IndexShardRoutingTable indexShardRoutingTable = clusterState.routingTable().index("test").shard(0); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .remove(indexShardRoutingTable.primaryShard().currentNodeId())).build(); - clusterState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + clusterState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java index c674b8c3a292d..8aab17160fd65 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java @@ -117,7 +117,7 @@ public void testSingleIndexStartedShard() { logger.info("Killing node1 where the shard is, checking the shard is unassigned"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build(); - newState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + newState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java index 1bbbb2c8b91a3..a7980b4d0f9e5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java @@ -130,7 +130,7 @@ public void testSingleIndexFirstStartPrimaryThenBackups() { logger.info("Kill node1, backup shard should become primary"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build(); - newState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); + newState = strategy.disassociateDeadNodes(clusterState, true, "reroute"); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java index 70b728487ae91..2ca7ba7d1a306 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java @@ -111,7 +111,7 @@ public void testFilterInitialRecovery() { // now remove the node of the other copy and fail the current DiscoveryNode node1 = state.nodes().resolveNode("node1"); - state = service.deassociateDeadNodes( + state = service.disassociateDeadNodes( ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).remove("node1")).build(), true, "test"); state = service.applyFailedShard(state, routingTable.index("idx").shard(0).primaryShard(), randomBoolean()); diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java index 53a00111c4289..e2cdaf3c7d5b8 100644 --- a/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java +++ b/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java @@ -20,6 +20,8 @@ package org.elasticsearch.common.io.stream; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.CheckedBiConsumer; +import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; @@ -39,6 +41,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -293,15 +296,27 @@ public int hashCode() { } - final int length = randomIntBetween(0, 16); - final Collection fooBars = new ArrayList<>(length); + runWriteReadCollectionTest( + () -> new FooBar(randomInt(), randomInt()), StreamOutput::writeCollection, in -> in.readList(FooBar::new)); + } + + public void testStringCollection() throws IOException { + runWriteReadCollectionTest(() -> randomUnicodeOfLength(16), StreamOutput::writeStringCollection, StreamInput::readStringList); + } + + private void runWriteReadCollectionTest( + final Supplier supplier, + final CheckedBiConsumer, IOException> writer, + final CheckedFunction, IOException> reader) throws IOException { + final int length = randomIntBetween(0, 10); + final Collection collection = new ArrayList<>(length); for (int i = 0; i < length; i++) { - fooBars.add(new FooBar(randomInt(), randomInt())); + collection.add(supplier.get()); } try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeCollection(fooBars); + writer.accept(out, collection); try (StreamInput in = out.bytes().streamInput()) { - assertThat(fooBars, equalTo(in.readList(FooBar::new))); + assertThat(collection, equalTo(reader.apply(in))); } } } diff --git a/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java b/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java index 6a77da8738c4e..4302549e2f1fd 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java @@ -136,7 +136,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { builder.nodes(DiscoveryNodes.builder(currentState.nodes()).remove("_non_existent")); currentState = builder.build(); - return allocationService.deassociateDeadNodes(currentState, true, "reroute"); + return allocationService.disassociateDeadNodes(currentState, true, "reroute"); } diff --git a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java index ebf77d1e80360..c3863e8ec08e4 100644 --- a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java +++ b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java @@ -20,7 +20,9 @@ package org.elasticsearch.persistent; import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -63,10 +65,13 @@ import static org.elasticsearch.persistent.PersistentTasksClusterService.persistentTasksChanged; import static org.elasticsearch.persistent.PersistentTasksExecutor.NO_NODE_FOUND; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; +import static org.elasticsearch.test.ClusterServiceUtils.setState; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.core.Is.is; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; @@ -464,6 +469,56 @@ public void testPeriodicRecheck() throws Exception { }); } + public void testUnassignTask() { + ClusterState clusterState = initialState(); + ClusterState.Builder builder = ClusterState.builder(clusterState); + PersistentTasksCustomMetaData.Builder tasks = PersistentTasksCustomMetaData.builder( + clusterState.metaData().custom(PersistentTasksCustomMetaData.TYPE)); + DiscoveryNodes.Builder nodes = DiscoveryNodes.builder() + .add(new DiscoveryNode("_node_1", buildNewFakeTransportAddress(), Version.CURRENT)) + .localNodeId("_node_1") + .masterNodeId("_node_1") + .add(new DiscoveryNode("_node_2", buildNewFakeTransportAddress(), Version.CURRENT)); + + String unassignedId = addTask(tasks, "unassign", "_node_2"); + + MetaData.Builder metaData = MetaData.builder(clusterState.metaData()).putCustom(PersistentTasksCustomMetaData.TYPE, tasks.build()); + clusterState = builder.metaData(metaData).nodes(nodes).build(); + setState(clusterService, clusterState); + PersistentTasksClusterService service = createService((params, currentState) -> + new Assignment("_node_2", "test")); + service.unassignPersistentTask(unassignedId, tasks.getLastAllocationId(), "unassignment test", ActionListener.wrap( + task -> { + assertThat(task.getAssignment().getExecutorNode(), is(nullValue())); + assertThat(task.getId(), equalTo(unassignedId)); + assertThat(task.getAssignment().getExplanation(), equalTo("unassignment test")); + }, + e -> fail() + )); + } + + public void testUnassignNonExistentTask() { + ClusterState clusterState = initialState(); + ClusterState.Builder builder = ClusterState.builder(clusterState); + PersistentTasksCustomMetaData.Builder tasks = PersistentTasksCustomMetaData.builder( + clusterState.metaData().custom(PersistentTasksCustomMetaData.TYPE)); + DiscoveryNodes.Builder nodes = DiscoveryNodes.builder() + .add(new DiscoveryNode("_node_1", buildNewFakeTransportAddress(), Version.CURRENT)) + .localNodeId("_node_1") + .masterNodeId("_node_1") + .add(new DiscoveryNode("_node_2", buildNewFakeTransportAddress(), Version.CURRENT)); + + MetaData.Builder metaData = MetaData.builder(clusterState.metaData()).putCustom(PersistentTasksCustomMetaData.TYPE, tasks.build()); + clusterState = builder.metaData(metaData).nodes(nodes).build(); + setState(clusterService, clusterState); + PersistentTasksClusterService service = createService((params, currentState) -> + new Assignment("_node_2", "test")); + service.unassignPersistentTask("missing-task", tasks.getLastAllocationId(), "unassignment test", ActionListener.wrap( + task -> fail(), + e -> assertThat(e, instanceOf(ResourceNotFoundException.class)) + )); + } + private ClusterService createRecheckTestClusterService(ClusterState initialState, boolean shouldSimulateFailure) { AtomicBoolean testFailureNextTime = new AtomicBoolean(shouldSimulateFailure); AtomicReference state = new AtomicReference<>(initialState); @@ -728,9 +783,11 @@ private ClusterState.Builder addRandomTask(ClusterState.Builder clusterStateBuil tasks.addTask(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, new TestParams(param), assignment).build())); } - private void addTask(PersistentTasksCustomMetaData.Builder tasks, String param, String node) { - tasks.addTask(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, new TestParams(param), + private String addTask(PersistentTasksCustomMetaData.Builder tasks, String param, String node) { + String id = UUIDs.base64UUID(); + tasks.addTask(id, TestPersistentTasksExecutor.NAME, new TestParams(param), new Assignment(node, "explanation: " + param)); + return id; } private DiscoveryNode newNode(String nodeId) { diff --git a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksCustomMetaDataTests.java b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksCustomMetaDataTests.java index c25ca3cb7db63..9118fad66b0dd 100644 --- a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksCustomMetaDataTests.java +++ b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksCustomMetaDataTests.java @@ -317,7 +317,7 @@ public void testFeatureSerialization() throws IOException { public void testDisassociateDeadNodes_givenNoPersistentTasks() { ClusterState originalState = ClusterState.builder(new ClusterName("persistent-tasks-tests")).build(); - ClusterState returnedState = PersistentTasksCustomMetaData.deassociateDeadNodes(originalState); + ClusterState returnedState = PersistentTasksCustomMetaData.disassociateDeadNodes(originalState); assertThat(originalState, sameInstance(returnedState)); } @@ -337,7 +337,7 @@ public void testDisassociateDeadNodes_givenAssignedPersistentTask() { .nodes(nodes) .metaData(MetaData.builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())) .build(); - ClusterState returnedState = PersistentTasksCustomMetaData.deassociateDeadNodes(originalState); + ClusterState returnedState = PersistentTasksCustomMetaData.disassociateDeadNodes(originalState); assertThat(originalState, sameInstance(returnedState)); PersistentTasksCustomMetaData originalTasks = PersistentTasksCustomMetaData.getPersistentTasksCustomMetaData(originalState); @@ -363,7 +363,7 @@ public void testDisassociateDeadNodes() { .nodes(nodes) .metaData(MetaData.builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())) .build(); - ClusterState returnedState = PersistentTasksCustomMetaData.deassociateDeadNodes(originalState); + ClusterState returnedState = PersistentTasksCustomMetaData.disassociateDeadNodes(originalState); assertThat(originalState, not(sameInstance(returnedState))); PersistentTasksCustomMetaData originalTasks = PersistentTasksCustomMetaData.getPersistentTasksCustomMetaData(originalState); diff --git a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java index 8f6393986da9d..08c32665adc58 100644 --- a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java +++ b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java @@ -46,8 +46,10 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.core.Is.is; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, minNumDataNodes = 2) public class PersistentTasksExecutorIT extends ESIntegTestCase { @@ -155,11 +157,8 @@ public void testPersistentActionWithNoAvailableNode() throws Exception { Settings nodeSettings = Settings.builder().put(nodeSettings(0)).put("node.attr.test_attr", "test").build(); String newNode = internalCluster().startNode(nodeSettings); String newNodeId = internalCluster().clusterService(newNode).localNode().getId(); - assertBusy(() -> { - // Wait for the task to start - assertThat(client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]").get().getTasks() - .size(), equalTo(1)); - }); + waitForTaskToStart(); + TaskInfo taskInfo = client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]") .get().getTasks().get(0); @@ -199,11 +198,7 @@ public void testPersistentActionWithNonClusterStateCondition() throws Exception TestPersistentTasksExecutor.setNonClusterStateCondition(true); - assertBusy(() -> { - // Wait for the task to start - assertThat(client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]").get().getTasks() - .size(), equalTo(1)); - }); + waitForTaskToStart(); TaskInfo taskInfo = client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]") .get().getTasks().get(0); @@ -221,12 +216,7 @@ public void testPersistentActionStatusUpdate() throws Exception { PlainActionFuture> future = new PlainActionFuture<>(); persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, new TestParams("Blah"), future); String taskId = future.get().getId(); - - assertBusy(() -> { - // Wait for the task to start - assertThat(client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]").get().getTasks() - .size(), equalTo(1)); - }); + waitForTaskToStart(); TaskInfo firstRunningTask = client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]") .get().getTasks().get(0); @@ -307,6 +297,62 @@ public void testCreatePersistentTaskWithDuplicateId() throws Exception { }); } + public void testUnassignRunningPersistentTask() throws Exception { + PersistentTasksClusterService persistentTasksClusterService = + internalCluster().getInstance(PersistentTasksClusterService.class, internalCluster().getMasterName()); + // Speed up rechecks to a rate that is quicker than what settings would allow + persistentTasksClusterService.setRecheckInterval(TimeValue.timeValueMillis(1)); + PersistentTasksService persistentTasksService = internalCluster().getInstance(PersistentTasksService.class); + PlainActionFuture> future = new PlainActionFuture<>(); + TestParams testParams = new TestParams("Blah"); + testParams.setExecutorNodeAttr("test"); + persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, testParams, future); + PersistentTask task = future.get(); + String taskId = task.getId(); + + Settings nodeSettings = Settings.builder().put(nodeSettings(0)).put("node.attr.test_attr", "test").build(); + internalCluster().startNode(nodeSettings); + + waitForTaskToStart(); + + PlainActionFuture> unassignmentFuture = new PlainActionFuture<>(); + + // Disallow re-assignment after it is unallocated to verify master and node state + TestPersistentTasksExecutor.setNonClusterStateCondition(false); + + persistentTasksClusterService.unassignPersistentTask(taskId, + task.getAllocationId() + 1, + "unassignment test", + unassignmentFuture); + PersistentTask unassignedTask = unassignmentFuture.get(); + assertThat(unassignedTask.getId(), equalTo(taskId)); + assertThat(unassignedTask.getAssignment().getExplanation(), equalTo("unassignment test")); + assertThat(unassignedTask.getAssignment().getExecutorNode(), is(nullValue())); + + assertBusy(() -> { + // Verify that the task is NOT running on the node + List tasks = client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]").get() + .getTasks(); + assertThat(tasks.size(), equalTo(0)); + + // Verify that the task is STILL in internal cluster state + assertClusterStateHasTask(taskId); + }); + + // Allow it to be reassigned again to the same node + TestPersistentTasksExecutor.setNonClusterStateCondition(true); + + // Verify it starts again + waitForTaskToStart(); + + assertClusterStateHasTask(taskId); + + // Complete or cancel the running task + TaskInfo taskInfo = client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]") + .get().getTasks().get(0); + stopOrCancelTask(taskInfo.getTaskId()); + } + private void stopOrCancelTask(TaskId taskId) { if (randomBoolean()) { logger.info("Completing the running task"); @@ -322,6 +368,25 @@ private void stopOrCancelTask(TaskId taskId) { } } + private static void waitForTaskToStart() throws Exception { + assertBusy(() -> { + // Wait for the task to start + assertThat(client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]").get().getTasks() + .size(), equalTo(1)); + }); + } + + private static void assertClusterStateHasTask(String taskId) { + Collection> clusterTasks = ((PersistentTasksCustomMetaData) internalCluster() + .clusterService() + .state() + .getMetaData() + .custom(PersistentTasksCustomMetaData.TYPE)) + .tasks(); + assertThat(clusterTasks, hasSize(1)); + assertThat(clusterTasks.iterator().next().getId(), equalTo(taskId)); + } + private void assertNoRunningTasks() throws Exception { assertBusy(() -> { // Wait for the task to finish diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java b/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java index 35beb10934e3d..3cb71f6be235c 100644 --- a/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java @@ -395,6 +395,9 @@ public void testQueryRandomGeoCollection() throws Exception { geoShapeQueryBuilder.relation(ShapeRelation.INTERSECTS); SearchResponse result = client().prepareSearch("test").setQuery(geoShapeQueryBuilder).get(); assertSearchResponse(result); + assumeTrue("Skipping the check for the polygon with a degenerated dimension until " + +" https://issues.apache.org/jira/browse/LUCENE-8634 is fixed", + randomPoly.maxLat - randomPoly.minLat > 8.4e-8 && randomPoly.maxLon - randomPoly.minLon > 8.4e-8); assertHitCount(result, 1); } @@ -423,7 +426,8 @@ public void testRandomGeoCollectionQuery() throws Exception { } gcb.shape(new PolygonBuilder(cb)); - logger.info("Created Random GeometryCollection containing {} shapes", gcb.numShapes()); + logger.info("Created Random GeometryCollection containing {} shapes using {} tree", gcb.numShapes(), + usePrefixTrees ? "default" : "quadtree"); if (usePrefixTrees == false) { client().admin().indices().prepareCreate("test").addMapping("type", "location", "type=geo_shape") @@ -444,7 +448,11 @@ public void testRandomGeoCollectionQuery() throws Exception { geoShapeQueryBuilder.relation(ShapeRelation.INTERSECTS); SearchResponse result = client().prepareSearch("test").setQuery(geoShapeQueryBuilder).get(); assertSearchResponse(result); - assertTrue(result.getHits().getTotalHits().value > 0); + assumeTrue("Skipping the check for the polygon with a degenerated dimension until " + +" https://issues.apache.org/jira/browse/LUCENE-8634 is fixed", + randomPoly.maxLat - randomPoly.minLat > 8.4e-8 && randomPoly.maxLon - randomPoly.minLon > 8.4e-8); + assertTrue("query: " + geoShapeQueryBuilder.toString() + " doc: " + Strings.toString(docSource), + result.getHits().getTotalHits().value > 0); } /** tests querying a random geometry collection with a point */ diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java index 6b651444f2d7d..2b59ae21daf5a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java @@ -273,7 +273,7 @@ public AutoFollowPattern(String remoteCluster, public AutoFollowPattern(StreamInput in) throws IOException { remoteCluster = in.readString(); - leaderIndexPatterns = in.readList(StreamInput::readString); + leaderIndexPatterns = in.readStringList(); followIndexPattern = in.readOptionalString(); maxReadRequestOperationCount = in.readOptionalVInt(); maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); @@ -350,7 +350,7 @@ public TimeValue getPollTimeout() { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(remoteCluster); - out.writeStringList(leaderIndexPatterns); + out.writeStringCollection(leaderIndexPatterns); out.writeOptionalString(followIndexPattern); out.writeOptionalVInt(maxReadRequestOperationCount); out.writeOptionalWriteable(maxReadRequestSize); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java index d0969850705f7..12d30e4d9f9b1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java @@ -275,7 +275,7 @@ public Request(StreamInput in) throws IOException { super(in); name = in.readString(); remoteCluster = in.readString(); - leaderIndexPatterns = in.readList(StreamInput::readString); + leaderIndexPatterns = in.readStringList(); followIndexNamePattern = in.readOptionalString(); maxReadRequestOperationCount = in.readOptionalVInt(); maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); @@ -294,7 +294,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(name); out.writeString(remoteCluster); - out.writeStringList(leaderIndexPatterns); + out.writeStringCollection(leaderIndexPatterns); out.writeOptionalString(followIndexNamePattern); out.writeOptionalVInt(maxReadRequestOperationCount); out.writeOptionalWriteable(maxReadRequestSize); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AbstractUnfollowIndexStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AbstractUnfollowIndexStep.java index 8e0626425b490..570c4fdf26771 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AbstractUnfollowIndexStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AbstractUnfollowIndexStep.java @@ -7,6 +7,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import java.util.Map; @@ -20,7 +21,8 @@ abstract class AbstractUnfollowIndexStep extends AsyncActionStep { } @Override - public final void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, Listener listener) { + public final void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, + ClusterStateObserver observer, Listener listener) { String followerIndex = indexMetaData.getIndex().getName(); Map customIndexMetadata = indexMetaData.getCustomData(CCR_METADATA_KEY); if (customIndexMetadata == null) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AsyncActionStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AsyncActionStep.java index 700c817164503..3195d2d525750 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AsyncActionStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AsyncActionStep.java @@ -7,6 +7,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; /** @@ -29,7 +30,8 @@ public boolean indexSurvives() { return true; } - public abstract void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, Listener listener); + public abstract void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, + ClusterStateObserver observer, Listener listener); public interface Listener { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AsyncRetryDuringSnapshotActionStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AsyncRetryDuringSnapshotActionStep.java new file mode 100644 index 0000000000000..85c91a795f09d --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/AsyncRetryDuringSnapshotActionStep.java @@ -0,0 +1,165 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.indexlifecycle; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.Index; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.SnapshotInProgressException; + +import java.util.function.Consumer; + +/** + * This is an abstract AsyncActionStep that wraps the performed action listener, checking to see + * if the action fails due to a snapshot being in progress. If a snapshot is in progress, it + * registers an observer and waits to try again when a snapshot is no longer running. + */ +public abstract class AsyncRetryDuringSnapshotActionStep extends AsyncActionStep { + private final Logger logger = LogManager.getLogger(AsyncRetryDuringSnapshotActionStep.class); + + public AsyncRetryDuringSnapshotActionStep(StepKey key, StepKey nextStepKey, Client client) { + super(key, nextStepKey, client); + } + + @Override + public void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, + ClusterStateObserver observer, Listener listener) { + // Wrap the original listener to handle exceptions caused by ongoing snapshots + SnapshotExceptionListener snapshotExceptionListener = new SnapshotExceptionListener(indexMetaData.getIndex(), listener, observer); + performDuringNoSnapshot(indexMetaData, currentClusterState, snapshotExceptionListener); + } + + /** + * Method to be performed during which no snapshots for the index are already underway. + */ + abstract void performDuringNoSnapshot(IndexMetaData indexMetaData, ClusterState currentClusterState, Listener listener); + + /** + * SnapshotExceptionListener is an injected listener wrapper that checks to see if a particular + * action failed due to a {@code SnapshotInProgressException}. If it did, then it registers a + * ClusterStateObserver listener waiting for the next time the snapshot is not running, + * re-running the step's {@link #performAction(IndexMetaData, ClusterState, ClusterStateObserver, Listener)} + * method when the snapshot is no longer running. + */ + class SnapshotExceptionListener implements AsyncActionStep.Listener { + private final Index index; + private final Listener originalListener; + private final ClusterStateObserver observer; + + SnapshotExceptionListener(Index index, Listener originalListener, ClusterStateObserver observer) { + this.index = index; + this.originalListener = originalListener; + this.observer = observer; + } + + @Override + public void onResponse(boolean complete) { + originalListener.onResponse(complete); + } + + @Override + public void onFailure(Exception e) { + if (e instanceof SnapshotInProgressException) { + try { + logger.debug("[{}] attempted to run ILM step but a snapshot is in progress, step will retry at a later time", + index.getName()); + observer.waitForNextChange( + new NoSnapshotRunningListener(observer, index.getName(), state -> { + IndexMetaData idxMeta = state.metaData().index(index); + if (idxMeta == null) { + // The index has since been deleted, mission accomplished! + originalListener.onResponse(true); + } + // Re-invoke the performAction method with the new state + performAction(idxMeta, state, observer, originalListener); + }, originalListener::onFailure), + // TODO: what is a good timeout value for no new state received during this time? + TimeValue.timeValueHours(12)); + } catch (Exception secondError) { + // There was a second error trying to set up an observer, + // fail the original listener + secondError.addSuppressed(e); + originalListener.onFailure(secondError); + } + } else { + originalListener.onFailure(e); + } + } + } + + /** + * A {@link ClusterStateObserver.Listener} that invokes the given function with the new state, + * once no snapshots are running. If a snapshot is still running it registers a new listener + * and tries again. Passes any exceptions to the original exception listener if they occur. + */ + class NoSnapshotRunningListener implements ClusterStateObserver.Listener { + + private final Consumer reRun; + private final Consumer exceptionConsumer; + private final ClusterStateObserver observer; + private final String indexName; + + NoSnapshotRunningListener(ClusterStateObserver observer, String indexName, + Consumer reRun, + Consumer exceptionConsumer) { + this.observer = observer; + this.reRun = reRun; + this.exceptionConsumer = exceptionConsumer; + this.indexName = indexName; + } + + @Override + public void onNewClusterState(ClusterState state) { + try { + if (snapshotInProgress(state)) { + observer.waitForNextChange(this); + } else { + logger.debug("[{}] retrying ILM step after snapshot has completed", indexName); + reRun.accept(state); + } + } catch (Exception e) { + exceptionConsumer.accept(e); + } + } + + private boolean snapshotInProgress(ClusterState state) { + SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE); + if (snapshotsInProgress == null || snapshotsInProgress.entries().isEmpty()) { + // No snapshots are running, new state is acceptable to proceed + return false; + } + + for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) { + if (snapshot.indices().stream() + .map(IndexId::getName) + .anyMatch(name -> name.equals(indexName))) { + // There is a snapshot running with this index name + return true; + } + } + // There are snapshots, but none for this index, so it's okay to proceed with this state + return false; + } + + @Override + public void onClusterServiceClose() { + // This means the cluster is being shut down, so nothing to do here + } + + @Override + public void onTimeout(TimeValue timeout) { + exceptionConsumer.accept(new IllegalStateException("step timed out while waiting for snapshots to complete")); + } + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStep.java index 3fb6e145236bc..149bb6dfef324 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStep.java @@ -8,8 +8,14 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; -final class CloseFollowerIndexStep extends AbstractUnfollowIndexStep { +import java.util.Map; + +import static org.elasticsearch.xpack.core.indexlifecycle.UnfollowAction.CCR_METADATA_KEY; + +final class CloseFollowerIndexStep extends AsyncRetryDuringSnapshotActionStep { static final String NAME = "close-follower-index"; @@ -18,7 +24,14 @@ final class CloseFollowerIndexStep extends AbstractUnfollowIndexStep { } @Override - void innerPerformAction(String followerIndex, Listener listener) { + void performDuringNoSnapshot(IndexMetaData indexMetaData, ClusterState currentClusterState, Listener listener) { + String followerIndex = indexMetaData.getIndex().getName(); + Map customIndexMetadata = indexMetaData.getCustomData(CCR_METADATA_KEY); + if (customIndexMetadata == null) { + listener.onResponse(true); + return; + } + CloseIndexRequest closeIndexRequest = new CloseIndexRequest(followerIndex); getClient().admin().indices().close(closeIndexRequest, ActionListener.wrap( r -> { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStep.java index f34da641a0cb4..62a5b7bea4b5e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStep.java @@ -14,7 +14,7 @@ /** * Deletes a single index. */ -public class DeleteStep extends AsyncActionStep { +public class DeleteStep extends AsyncRetryDuringSnapshotActionStep { public static final String NAME = "delete"; public DeleteStep(StepKey key, StepKey nextStepKey, Client client) { @@ -22,10 +22,10 @@ public DeleteStep(StepKey key, StepKey nextStepKey, Client client) { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { + public void performDuringNoSnapshot(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { getClient().admin().indices() .delete(new DeleteIndexRequest(indexMetaData.getIndex().getName()), - ActionListener.wrap(response -> listener.onResponse(true) , listener::onFailure)); + ActionListener.wrap(response -> listener.onResponse(true), listener::onFailure)); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStep.java index 799959ce69f61..871a95419a410 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStep.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import java.util.Objects; @@ -30,7 +31,7 @@ public int getMaxNumSegments() { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { + public void performAction(IndexMetaData indexMetaData, ClusterState currentState, ClusterStateObserver observer, Listener listener) { ForceMergeRequest request = new ForceMergeRequest(indexMetaData.getIndex().getName()); request.maxNumSegments(maxNumSegments); getClient().admin().indices() diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStep.java index 523aad10a488a..ae7b0af6222dc 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStep.java @@ -14,7 +14,7 @@ /** * Freezes an index. */ -public class FreezeStep extends AsyncActionStep { +public class FreezeStep extends AsyncRetryDuringSnapshotActionStep { public static final String NAME = "freeze"; public FreezeStep(StepKey key, StepKey nextStepKey, Client client) { @@ -22,7 +22,7 @@ public FreezeStep(StepKey key, StepKey nextStepKey, Client client) { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { + public void performDuringNoSnapshot(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { getClient().admin().indices().execute(TransportFreezeIndexAction.FreezeIndexAction.INSTANCE, new TransportFreezeIndexAction.FreezeRequest(indexMetaData.getIndex().getName()), ActionListener.wrap(response -> listener.onResponse(true), listener::onFailure)); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/LifecyclePolicyUtils.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/LifecyclePolicyUtils.java new file mode 100644 index 0000000000000..c3752e3927b41 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/LifecyclePolicyUtils.java @@ -0,0 +1,76 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.indexlifecycle; + +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.compress.NotXContentException; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; + +/** + * A utility class used for loading index lifecycle policies from the resource classpath + */ +public class LifecyclePolicyUtils { + + private LifecyclePolicyUtils() {}; + + /** + * Loads a built-in index lifecycle policy and returns its source. + */ + public static LifecyclePolicy loadPolicy(String name, String resource, NamedXContentRegistry xContentRegistry) { + try { + BytesReference source = load(resource); + validate(source); + + try (XContentParser parser = XContentType.JSON.xContent() + .createParser(xContentRegistry, LoggingDeprecationHandler.THROW_UNSUPPORTED_OPERATION, source.utf8ToString())) { + return LifecyclePolicy.parse(parser, name); + } + } catch (Exception e) { + throw new IllegalArgumentException("unable to load policy [" + name + "] from [" + resource + "]", e); + } + } + + /** + * Loads a resource from the classpath and returns it as a {@link BytesReference} + */ + private static BytesReference load(String name) throws IOException { + try (InputStream is = LifecyclePolicyUtils.class.getResourceAsStream(name)) { + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + Streams.copy(is, out); + return new BytesArray(out.toByteArray()); + } + } + } + + /** + * Parses and validates that the source is not empty. + */ + private static void validate(BytesReference source) { + if (source == null) { + throw new ElasticsearchParseException("policy must not be null"); + } + + try { + XContentHelper.convertToMap(source, false, XContentType.JSON).v2(); + } catch (NotXContentException e) { + throw new ElasticsearchParseException("policy must not be empty"); + } catch (Exception e) { + throw new ElasticsearchParseException("invalid policy", e); + } + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStep.java index 7ba2c4633ab99..58ede7200cc4f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStep.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.admin.indices.open.OpenIndexRequest; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; final class OpenFollowerIndexStep extends AsyncActionStep { @@ -20,7 +21,8 @@ final class OpenFollowerIndexStep extends AsyncActionStep { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, Listener listener) { + public void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, + ClusterStateObserver observer, Listener listener) { OpenIndexRequest request = new OpenIndexRequest(indexMetaData.getIndex().getName()); getClient().admin().indices().open(request, ActionListener.wrap( r -> { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStep.java index f501c27d8c40e..91b2bbe82648e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStep.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; @@ -30,7 +31,8 @@ public RolloverStep(StepKey key, StepKey nextStepKey, Client client) { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, Listener listener) { + public void performAction(IndexMetaData indexMetaData, ClusterState currentClusterState, + ClusterStateObserver observer, Listener listener) { boolean indexingComplete = LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE_SETTING.get(indexMetaData.getSettings()); if (indexingComplete) { logger.trace(indexMetaData.getIndex() + " has lifecycle complete set, skipping " + RolloverStep.NAME); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStep.java index 973dd7f0ba08d..7973289b968d5 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStep.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingNode; @@ -42,7 +43,7 @@ public SetSingleNodeAllocateStep(StepKey key, StepKey nextStepKey, Client client } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState clusterState, Listener listener) { + public void performAction(IndexMetaData indexMetaData, ClusterState clusterState, ClusterStateObserver observer, Listener listener) { RoutingAllocation allocation = new RoutingAllocation(ALLOCATION_DECIDERS, clusterState.getRoutingNodes(), clusterState, null, System.nanoTime()); List validNodeIds = new ArrayList<>(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStep.java index da59b16ded99b..51497dccc2d4c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStep.java @@ -18,7 +18,7 @@ * Following shrinking an index and deleting the original index, this step creates an alias with the same name as the original index which * points to the new shrunken index to allow clients to continue to use the original index name without being aware that it has shrunk. */ -public class ShrinkSetAliasStep extends AsyncActionStep { +public class ShrinkSetAliasStep extends AsyncRetryDuringSnapshotActionStep { public static final String NAME = "aliases"; private String shrunkIndexPrefix; @@ -32,7 +32,7 @@ String getShrunkIndexPrefix() { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { + public void performDuringNoSnapshot(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { // get source index String index = indexMetaData.getIndex().getName(); // get target shrink index diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStep.java index 19ddbfe04cfa2..0e9f84bc3a843 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStep.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.admin.indices.shrink.ResizeRequest; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; @@ -38,7 +39,7 @@ String getShrunkIndexPrefix() { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { + public void performAction(IndexMetaData indexMetaData, ClusterState currentState, ClusterStateObserver observer, Listener listener) { LifecycleExecutionState lifecycleState = LifecycleExecutionState.fromIndexMetadata(indexMetaData); if (lifecycleState.getLifecycleDate() == null) { throw new IllegalStateException("source index [" + indexMetaData.getIndex().getName() + diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStep.java index c66c1427d07d4..9f6b3f7a84f2a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStep.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; @@ -28,7 +29,7 @@ public UpdateSettingsStep(StepKey key, StepKey nextStepKey, Client client, Setti } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { + public void performAction(IndexMetaData indexMetaData, ClusterState currentState, ClusterStateObserver observer, Listener listener) { UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(indexMetaData.getIndex().getName()).settings(settings); getClient().admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(response -> listener.onResponse(true), listener::onFailure)); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/action/RemoveIndexLifecyclePolicyAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/action/RemoveIndexLifecyclePolicyAction.java index 239e748e58d8a..0e530baa57f0c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/action/RemoveIndexLifecyclePolicyAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/action/RemoveIndexLifecyclePolicyAction.java @@ -82,13 +82,13 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - failedIndexes = in.readList(StreamInput::readString); + failedIndexes = in.readStringList(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeStringList(failedIndexes); + out.writeStringCollection(failedIndexes); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/FindFileStructureAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/FindFileStructureAction.java index 78fcc4939ca36..f4eb7fbc4e53b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/FindFileStructureAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/FindFileStructureAction.java @@ -327,7 +327,7 @@ public void readFrom(StreamInput in) throws IOException { timeout = in.readOptionalTimeValue(); charset = in.readOptionalString(); format = in.readBoolean() ? in.readEnum(FileStructure.Format.class) : null; - columnNames = in.readBoolean() ? in.readList(StreamInput::readString) : null; + columnNames = in.readBoolean() ? in.readStringList() : null; hasHeaderRow = in.readOptionalBoolean(); delimiter = in.readBoolean() ? (char) in.readVInt() : null; quote = in.readBoolean() ? (char) in.readVInt() : null; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/GetJobsStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/GetJobsStatsAction.java index c5dba63fcc6b3..dd3c4618f7025 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/GetJobsStatsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/GetJobsStatsAction.java @@ -84,7 +84,7 @@ public Request() {} public Request(StreamInput in) throws IOException { super(in); jobId = in.readString(); - expandedJobsIds = in.readList(StreamInput::readString); + expandedJobsIds = in.readStringList(); if (in.getVersion().onOrAfter(Version.V_6_1_0)) { allowNoJobs = in.readBoolean(); } @@ -94,7 +94,7 @@ public Request(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(jobId); - out.writeStringList(expandedJobsIds); + out.writeStringCollection(expandedJobsIds); if (out.getVersion().onOrAfter(Version.V_6_1_0)) { out.writeBoolean(allowNoJobs); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/StartDatafeedAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/StartDatafeedAction.java index 12faa157eeb9a..cdfdf06bf85ac 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/StartDatafeedAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/StartDatafeedAction.java @@ -197,7 +197,7 @@ public DatafeedParams(StreamInput in) throws IOException { timeout = TimeValue.timeValueMillis(in.readVLong()); if (in.getVersion().onOrAfter(Version.V_6_6_0)) { jobId = in.readOptionalString(); - datafeedIndices = in.readList(StreamInput::readString); + datafeedIndices = in.readStringList(); } } @@ -274,7 +274,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(timeout.millis()); if (out.getVersion().onOrAfter(Version.V_6_6_0)) { out.writeOptionalString(jobId); - out.writeStringList(datafeedIndices); + out.writeStringCollection(datafeedIndices); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java index 3002a43e9d8c2..938452d27cc58 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java @@ -246,14 +246,14 @@ public DatafeedConfig(StreamInput in) throws IOException { this.queryDelay = in.readOptionalTimeValue(); this.frequency = in.readOptionalTimeValue(); if (in.readBoolean()) { - this.indices = Collections.unmodifiableList(in.readList(StreamInput::readString)); + this.indices = Collections.unmodifiableList(in.readStringList()); } else { this.indices = null; } // This consumes the list of types if there was one. if (in.getVersion().before(Version.V_7_0_0)) { if (in.readBoolean()) { - in.readList(StreamInput::readString); + in.readStringList(); } } if (in.getVersion().before(Version.V_6_6_0)) { @@ -408,7 +408,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalTimeValue(frequency); if (indices != null) { out.writeBoolean(true); - out.writeStringList(indices); + out.writeStringCollection(indices); } else { out.writeBoolean(false); } @@ -416,7 +416,7 @@ public void writeTo(StreamOutput out) throws IOException { // An empty list is expected if (out.getVersion().before(Version.V_7_0_0)) { out.writeBoolean(true); - out.writeStringList(Collections.emptyList()); + out.writeStringCollection(Collections.emptyList()); } if (out.getVersion().before(Version.V_6_6_0)) { out.writeNamedWriteable(getParsedQuery()); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedUpdate.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedUpdate.java index 8bc49d4598a21..23c2eeccc6a59 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedUpdate.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedUpdate.java @@ -107,14 +107,14 @@ public DatafeedUpdate(StreamInput in) throws IOException { this.queryDelay = in.readOptionalTimeValue(); this.frequency = in.readOptionalTimeValue(); if (in.readBoolean()) { - this.indices = in.readList(StreamInput::readString); + this.indices = in.readStringList(); } else { this.indices = null; } // This consumes the list of types if there was one. if (in.getVersion().before(Version.V_7_0_0)) { if (in.readBoolean()) { - in.readList(StreamInput::readString); + in.readStringList(); } } this.query = in.readOptionalNamedWriteable(QueryBuilder.class); @@ -148,7 +148,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalTimeValue(frequency); if (indices != null) { out.writeBoolean(true); - out.writeStringList(indices); + out.writeStringCollection(indices); } else { out.writeBoolean(false); } @@ -156,7 +156,7 @@ public void writeTo(StreamOutput out) throws IOException { // An empty list is expected if (out.getVersion().before(Version.V_7_0_0)) { out.writeBoolean(true); - out.writeStringList(Collections.emptyList()); + out.writeStringCollection(Collections.emptyList()); } out.writeOptionalNamedWriteable(query); out.writeOptionalWriteable(aggregations); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java index 12c42f3df4d58..acaceace0472b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java @@ -206,20 +206,20 @@ public FileStructure(StreamInput in) throws IOException { format = in.readEnum(Format.class); multilineStartPattern = in.readOptionalString(); excludeLinesPattern = in.readOptionalString(); - columnNames = in.readBoolean() ? Collections.unmodifiableList(in.readList(StreamInput::readString)) : null; + columnNames = in.readBoolean() ? Collections.unmodifiableList(in.readStringList()) : null; hasHeaderRow = in.readOptionalBoolean(); delimiter = in.readBoolean() ? (char) in.readVInt() : null; quote = in.readBoolean() ? (char) in.readVInt() : null; shouldTrimFields = in.readOptionalBoolean(); grokPattern = in.readOptionalString(); - jodaTimestampFormats = in.readBoolean() ? Collections.unmodifiableList(in.readList(StreamInput::readString)) : null; - javaTimestampFormats = in.readBoolean() ? Collections.unmodifiableList(in.readList(StreamInput::readString)) : null; + jodaTimestampFormats = in.readBoolean() ? Collections.unmodifiableList(in.readStringList()) : null; + javaTimestampFormats = in.readBoolean() ? Collections.unmodifiableList(in.readStringList()) : null; timestampField = in.readOptionalString(); needClientTimezone = in.readBoolean(); mappings = Collections.unmodifiableSortedMap(new TreeMap<>(in.readMap())); ingestPipeline = in.readBoolean() ? Collections.unmodifiableMap(in.readMap()) : null; fieldStats = Collections.unmodifiableSortedMap(new TreeMap<>(in.readMap(StreamInput::readString, FieldStats::new))); - explanation = Collections.unmodifiableList(in.readList(StreamInput::readString)); + explanation = Collections.unmodifiableList(in.readStringList()); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisConfig.java index 7d462bd153371..933188c8221eb 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisConfig.java @@ -125,7 +125,7 @@ private AnalysisConfig(TimeValue bucketSpan, String categorizationFieldName, Lis public AnalysisConfig(StreamInput in) throws IOException { bucketSpan = in.readTimeValue(); categorizationFieldName = in.readOptionalString(); - categorizationFilters = in.readBoolean() ? Collections.unmodifiableList(in.readList(StreamInput::readString)) : null; + categorizationFilters = in.readBoolean() ? Collections.unmodifiableList(in.readStringList()) : null; if (in.getVersion().onOrAfter(Version.V_6_2_0)) { categorizationAnalyzerConfig = in.readOptionalWriteable(CategorizationAnalyzerConfig::new); } else { @@ -134,7 +134,7 @@ public AnalysisConfig(StreamInput in) throws IOException { latency = in.readOptionalTimeValue(); summaryCountFieldName = in.readOptionalString(); detectors = Collections.unmodifiableList(in.readList(Detector::new)); - influencers = Collections.unmodifiableList(in.readList(StreamInput::readString)); + influencers = Collections.unmodifiableList(in.readStringList()); multivariateByFields = in.readOptionalBoolean(); } @@ -145,7 +145,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(categorizationFieldName); if (categorizationFilters != null) { out.writeBoolean(true); - out.writeStringList(categorizationFilters); + out.writeStringCollection(categorizationFilters); } else { out.writeBoolean(false); } @@ -155,7 +155,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalTimeValue(latency); out.writeOptionalString(summaryCountFieldName); out.writeList(detectors); - out.writeStringList(influencers); + out.writeStringCollection(influencers); out.writeOptionalBoolean(multivariateByFields); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/Job.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/Job.java index 314a533914a6a..a912b5d65f208 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/Job.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/Job.java @@ -187,7 +187,7 @@ public Job(StreamInput in) throws IOException { jobType = in.readString(); jobVersion = in.readBoolean() ? Version.readVersion(in) : null; if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - groups = Collections.unmodifiableList(in.readList(StreamInput::readString)); + groups = Collections.unmodifiableList(in.readStringList()); } else { groups = Collections.emptyList(); } @@ -444,7 +444,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(false); } if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeStringList(groups); + out.writeStringCollection(groups); } out.writeOptionalString(description); out.writeVLong(createTime.getTime()); @@ -671,7 +671,7 @@ public Builder(StreamInput in) throws IOException { jobType = in.readString(); jobVersion = in.readBoolean() ? Version.readVersion(in) : null; if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - groups = in.readList(StreamInput::readString); + groups = in.readStringList(); } else { groups = Collections.emptyList(); } @@ -856,7 +856,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(false); } if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeStringList(groups); + out.writeStringCollection(groups); } out.writeOptionalString(description); if (createTime != null) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobUpdate.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobUpdate.java index a0519697e5909..36e1fc1096675 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobUpdate.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobUpdate.java @@ -125,7 +125,7 @@ public JobUpdate(StreamInput in) throws IOException { modelSnapshotRetentionDays = in.readOptionalLong(); resultsRetentionDays = in.readOptionalLong(); if (in.readBoolean()) { - categorizationFilters = in.readList(StreamInput::readString); + categorizationFilters = in.readStringList(); } else { categorizationFilters = null; } @@ -172,7 +172,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalLong(resultsRetentionDays); out.writeBoolean(categorizationFilters != null); if (categorizationFilters != null) { - out.writeStringList(categorizationFilters); + out.writeStringCollection(categorizationFilters); } out.writeMap(customSettings); out.writeOptionalString(modelSnapshotId); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/Bucket.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/Bucket.java index 793968802c23a..d335ba39e0026 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/Bucket.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/Bucket.java @@ -139,7 +139,7 @@ public Bucket(StreamInput in) throws IOException { in.readList(Bucket::readOldPerPartitionNormalization); } if (in.getVersion().onOrAfter(Version.V_6_2_0)) { - scheduledEvents = in.readList(StreamInput::readString); + scheduledEvents = in.readStringList(); if (scheduledEvents.isEmpty()) { scheduledEvents = Collections.emptyList(); } @@ -165,7 +165,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeList(Collections.emptyList()); } if (out.getVersion().onOrAfter(Version.V_6_2_0)) { - out.writeStringList(scheduledEvents); + out.writeStringCollection(scheduledEvents); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/CategoryDefinition.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/CategoryDefinition.java index 7d5fb0a1bae0c..576bed5dcea2f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/CategoryDefinition.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/CategoryDefinition.java @@ -77,7 +77,7 @@ public CategoryDefinition(StreamInput in) throws IOException { terms = in.readString(); regex = in.readString(); maxMatchingLength = in.readLong(); - examples = new TreeSet<>(in.readList(StreamInput::readString)); + examples = new TreeSet<>(in.readStringList()); if (in.getVersion().onOrAfter(Version.V_6_4_0)) { grokPattern = in.readOptionalString(); } @@ -90,7 +90,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(terms); out.writeString(regex); out.writeLong(maxMatchingLength); - out.writeStringList(new ArrayList<>(examples)); + out.writeStringCollection(examples); if (out.getVersion().onOrAfter(Version.V_6_4_0)) { out.writeOptionalString(grokPattern); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java index a9daa78a6362a..506fbff640f75 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java @@ -124,7 +124,7 @@ public ForecastRequestStats(StreamInput in) throws IOException { forecastId = in.readString(); recordCount = in.readLong(); if (in.readBoolean()) { - messages = in.readList(StreamInput::readString); + messages = in.readStringList(); } else { messages = null; } @@ -147,7 +147,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(recordCount); if (messages != null) { out.writeBoolean(true); - out.writeStringList(messages); + out.writeStringCollection(messages); } else { out.writeBoolean(false); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/MetricConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/MetricConfig.java index 46f0c7397c6e5..47b45dc69dd2d 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/MetricConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/MetricConfig.java @@ -86,7 +86,7 @@ public MetricConfig(final String field, final List metrics) { MetricConfig(final StreamInput in) throws IOException { field = in.readString(); - metrics = in.readList(StreamInput::readString); + metrics = in.readStringList(); } /** @@ -144,7 +144,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(field); - out.writeStringList(metrics); + out.writeStringCollection(metrics); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesResponse.java index 6d4a3f1ad44d0..0e1652af95b2e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesResponse.java @@ -49,12 +49,12 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(created, StreamOutput::writeString, StreamOutput::writeStringList); + out.writeMap(created, StreamOutput::writeString, StreamOutput::writeStringCollection); } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - this.created = Collections.unmodifiableMap(in.readMap(StreamInput::readString, si -> si.readList(StreamInput::readString))); + this.created = Collections.unmodifiableMap(in.readMap(StreamInput::readString, StreamInput::readStringList)); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/rolemapping/PutRoleMappingRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/rolemapping/PutRoleMappingRequest.java index 19a84525c3011..087e29ec8b56a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/rolemapping/PutRoleMappingRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/rolemapping/PutRoleMappingRequest.java @@ -125,7 +125,7 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); this.name = in.readString(); this.enabled = in.readBoolean(); - this.roles = in.readList(StreamInput::readString); + this.roles = in.readStringList(); this.rules = ExpressionParser.readExpression(in); this.metadata = in.readMap(); this.refreshPolicy = RefreshPolicy.readFrom(in); @@ -136,7 +136,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(name); out.writeBoolean(enabled); - out.writeStringList(roles); + out.writeStringCollection(roles); ExpressionParser.writeExpression(rules, out); out.writeMap(metadata); refreshPolicy.writeTo(out); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/TokensInvalidationResult.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/TokensInvalidationResult.java index 117ef3316e1b9..f6e7965d9630b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/TokensInvalidationResult.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/TokensInvalidationResult.java @@ -49,8 +49,8 @@ public TokensInvalidationResult(List invalidatedTokens, List pre } public TokensInvalidationResult(StreamInput in) throws IOException { - this.invalidatedTokens = in.readList(StreamInput::readString); - this.previouslyInvalidatedTokens = in.readList(StreamInput::readString); + this.invalidatedTokens = in.readStringList(); + this.previouslyInvalidatedTokens = in.readStringList(); this.errors = in.readList(StreamInput::readException); this.attemptCount = in.readVInt(); } @@ -97,8 +97,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public void writeTo(StreamOutput out) throws IOException { - out.writeStringList(invalidatedTokens); - out.writeStringList(previouslyInvalidatedTokens); + out.writeStringCollection(invalidatedTokens); + out.writeStringCollection(previouslyInvalidatedTokens); out.writeCollection(errors, StreamOutput::writeException); out.writeVInt(attemptCount); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/ExpressionRoleMapping.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/ExpressionRoleMapping.java index 8e3f8e5593df1..95d1e9fa77149 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/ExpressionRoleMapping.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/ExpressionRoleMapping.java @@ -78,7 +78,7 @@ public ExpressionRoleMapping(String name, RoleMapperExpression expr, List { - - @Override - protected CloseFollowerIndexStep newInstance(Step.StepKey key, Step.StepKey nextKey, Client client) { - return new CloseFollowerIndexStep(key, nextKey, client); - } +public class CloseFollowerIndexStepTests extends AbstractStepTestCase { public void testCloseFollowingIndex() { IndexMetaData indexMetadata = IndexMetaData.builder("follower-index") @@ -56,7 +51,7 @@ public void testCloseFollowingIndex() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; CloseFollowerIndexStep step = new CloseFollowerIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; @@ -98,7 +93,7 @@ public void testCloseFollowingIndexFailed() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; CloseFollowerIndexStep step = new CloseFollowerIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; @@ -114,4 +109,30 @@ public void onFailure(Exception e) { Mockito.verify(indicesClient).close(Mockito.any(), Mockito.any()); Mockito.verifyNoMoreInteractions(indicesClient); } + + @Override + protected CloseFollowerIndexStep createRandomInstance() { + Step.StepKey stepKey = randomStepKey(); + Step.StepKey nextStepKey = randomStepKey(); + return new CloseFollowerIndexStep(stepKey, nextStepKey, Mockito.mock(Client.class)); + } + + @Override + protected CloseFollowerIndexStep mutateInstance(CloseFollowerIndexStep instance) { + Step.StepKey key = instance.getKey(); + Step.StepKey nextKey = instance.getNextStepKey(); + + if (randomBoolean()) { + key = new Step.StepKey(key.getPhase(), key.getAction(), key.getName() + randomAlphaOfLength(5)); + } else { + nextKey = new Step.StepKey(key.getPhase(), key.getAction(), key.getName() + randomAlphaOfLength(5)); + } + + return new CloseFollowerIndexStep(key, nextKey, instance.getClient()); + } + + @Override + protected CloseFollowerIndexStep copyInstance(CloseFollowerIndexStep instance) { + return new CloseFollowerIndexStep(instance.getKey(), instance.getNextStepKey(), instance.getClient()); + } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStepTests.java index c85df6de659e8..405a39cd5906c 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/DeleteStepTests.java @@ -91,7 +91,7 @@ public void testDeleted() { SetOnce actionCompleted = new SetOnce<>(); DeleteStep step = createRandomInstance(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { actionCompleted.set(complete); @@ -138,7 +138,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { SetOnce exceptionThrown = new SetOnce<>(); DeleteStep step = createRandomInstance(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { throw new AssertionError("Unexpected method call"); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStepTests.java index 9a38ddf3a2677..4ce895ae349ea 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ForceMergeStepTests.java @@ -88,7 +88,7 @@ public void testPerformActionComplete() { ForceMergeStep step = new ForceMergeStep(stepKey, nextStepKey, client, maxNumSegments); SetOnce completed = new SetOnce<>(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed.set(complete); @@ -129,7 +129,7 @@ public void testPerformActionThrowsException() { ForceMergeStep step = new ForceMergeStep(stepKey, nextStepKey, client, maxNumSegments); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { throw new AssertionError("unexpected method call"); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStepTests.java index 94ca2c2635c63..0198ed7abee7c 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/FreezeStepTests.java @@ -92,7 +92,7 @@ public void testFreeze() { SetOnce actionCompleted = new SetOnce<>(); FreezeStep step = createRandomInstance(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { actionCompleted.set(complete); @@ -135,7 +135,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { SetOnce exceptionThrown = new SetOnce<>(); FreezeStep step = createRandomInstance(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { throw new AssertionError("Unexpected method call"); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStepTests.java index 2d5086ec88fac..027b5c811cead 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/OpenFollowerIndexStepTests.java @@ -77,7 +77,7 @@ public void testOpenFollowingIndex() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; OpenFollowerIndexStep step = new OpenFollowerIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; @@ -118,7 +118,7 @@ public void testOpenFollowingIndexFailed() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; OpenFollowerIndexStep step = new OpenFollowerIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/PauseFollowerIndexStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/PauseFollowerIndexStepTests.java index fa877ef080ff4..f1977bdbd8577 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/PauseFollowerIndexStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/PauseFollowerIndexStepTests.java @@ -56,7 +56,7 @@ public void testPauseFollowingIndex() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; PauseFollowerIndexStep step = new PauseFollowerIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; @@ -93,7 +93,7 @@ public void testPauseFollowingIndexFailed() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; PauseFollowerIndexStep step = new PauseFollowerIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStepTests.java index 5f65b581e00b3..007f0c2865db2 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/RolloverStepTests.java @@ -106,7 +106,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any()); SetOnce actionCompleted = new SetOnce<>(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { @@ -138,7 +138,7 @@ public void testPerformActionWithIndexingComplete() { RolloverStep step = createRandomInstance(); SetOnce actionCompleted = new SetOnce<>(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { @@ -183,7 +183,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any()); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { @@ -212,7 +212,7 @@ public void testPerformActionInvalidNullOrEmptyAlias() { RolloverStep step = createRandomInstance(); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { throw new AssertionError("Unexpected method call"); @@ -237,7 +237,7 @@ public void testPerformActionAliasDoesNotPointToIndex() { RolloverStep step = createRandomInstance(); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, null, new AsyncActionStep.Listener() { + step.performAction(indexMetaData, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { throw new AssertionError("Unexpected method call"); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStepTests.java index b42ada6956f87..525744d68af10 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetSingleNodeAllocateStepTests.java @@ -262,7 +262,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).updateSettings(Mockito.any(), Mockito.any()); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, clusterState, new Listener() { + step.performAction(indexMetaData, clusterState, null, new Listener() { @Override public void onResponse(boolean complete) { @@ -320,7 +320,7 @@ public void testPerformActionAttrsNoShard() { SetSingleNodeAllocateStep step = createRandomInstance(); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, clusterState, new Listener() { + step.performAction(indexMetaData, clusterState, null, new Listener() { @Override public void onResponse(boolean complete) { @@ -375,7 +375,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { SetOnce actionCompleted = new SetOnce<>(); - step.performAction(indexMetaData, clusterState, new Listener() { + step.performAction(indexMetaData, clusterState, null, new Listener() { @Override public void onResponse(boolean complete) { @@ -407,7 +407,7 @@ private void assertNoValidNode(IndexMetaData indexMetaData, Index index, Discove SetOnce actionCompleted = new SetOnce<>(); - step.performAction(indexMetaData, clusterState, new Listener() { + step.performAction(indexMetaData, clusterState, null, new Listener() { @Override public void onResponse(boolean complete) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStepTests.java index a5c0e4d7146bc..d1911000f6786 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkSetAliasStepTests.java @@ -119,7 +119,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).aliases(Mockito.any(), Mockito.any()); SetOnce actionCompleted = new SetOnce<>(); - step.performAction(indexMetaData, null, new Listener() { + step.performAction(indexMetaData, null, null, new Listener() { @Override public void onResponse(boolean complete) { @@ -163,7 +163,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).aliases(Mockito.any(), Mockito.any()); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, null, new Listener() { + step.performAction(indexMetaData, null, null, new Listener() { @Override public void onResponse(boolean complete) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStepTests.java index 0cd655cb9d60f..e9ceccc4e279c 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkStepTests.java @@ -127,7 +127,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).resizeIndex(Mockito.any(), Mockito.any()); SetOnce actionCompleted = new SetOnce<>(); - step.performAction(sourceIndexMetaData, null, new Listener() { + step.performAction(sourceIndexMetaData, null, null, new Listener() { @Override public void onResponse(boolean complete) { @@ -173,7 +173,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).resizeIndex(Mockito.any(), Mockito.any()); SetOnce actionCompleted = new SetOnce<>(); - step.performAction(indexMetaData, null, new Listener() { + step.performAction(indexMetaData, null, null, new Listener() { @Override public void onResponse(boolean complete) { @@ -220,7 +220,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).resizeIndex(Mockito.any(), Mockito.any()); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, null, new Listener() { + step.performAction(indexMetaData, null, null, new Listener() { @Override public void onResponse(boolean complete) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UnfollowFollowIndexStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UnfollowFollowIndexStepTests.java index 58558c92d2511..e92f1dce6a477 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UnfollowFollowIndexStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UnfollowFollowIndexStepTests.java @@ -56,7 +56,7 @@ public void testUnFollow() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; UnfollowFollowIndexStep step = new UnfollowFollowIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; @@ -98,7 +98,7 @@ public void testUnFollowUnfollowFailed() { Boolean[] completed = new Boolean[1]; Exception[] failure = new Exception[1]; UnfollowFollowIndexStep step = new UnfollowFollowIndexStep(randomStepKey(), randomStepKey(), client); - step.performAction(indexMetadata, null, new AsyncActionStep.Listener() { + step.performAction(indexMetadata, null, null, new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { completed[0] = complete; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStepTests.java index 22908146af21f..28683fa86291c 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/UpdateSettingsStepTests.java @@ -100,7 +100,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { SetOnce actionCompleted = new SetOnce<>(); - step.performAction(indexMetaData, null, new Listener() { + step.performAction(indexMetaData, null, null, new Listener() { @Override public void onResponse(boolean complete) { @@ -147,7 +147,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(indicesClient).updateSettings(Mockito.any(), Mockito.any()); SetOnce exceptionThrown = new SetOnce<>(); - step.performAction(indexMetaData, null, new Listener() { + step.performAction(indexMetaData, null, null, new Listener() { @Override public void onResponse(boolean complete) { diff --git a/x-pack/plugin/ilm/qa/multi-cluster/build.gradle b/x-pack/plugin/ilm/qa/multi-cluster/build.gradle index 59df733892944..34b7cf9e44c58 100644 --- a/x-pack/plugin/ilm/qa/multi-cluster/build.gradle +++ b/x-pack/plugin/ilm/qa/multi-cluster/build.gradle @@ -4,6 +4,8 @@ apply plugin: 'elasticsearch.standalone-test' dependencies { testCompile project(':x-pack:plugin:ccr:qa') + testCompile project(':x-pack:plugin:core') + testCompile project(':x-pack:plugin:ilm') } task leaderClusterTest(type: RestIntegTestTask) { @@ -25,6 +27,8 @@ leaderClusterTestCluster { leaderClusterTestRunner { systemProperty 'tests.target_cluster', 'leader' + /* To support taking index snapshots, we have to set path.repo setting */ + systemProperty 'tests.path.repo', new File(buildDir, "cluster/shared/repo") } task followClusterTest(type: RestIntegTestTask) {} @@ -47,6 +51,8 @@ followClusterTestCluster { followClusterTestRunner { systemProperty 'tests.target_cluster', 'follow' systemProperty 'tests.leader_host', "${-> leaderClusterTest.nodes.get(0).httpUri()}" + /* To support taking index snapshots, we have to set path.repo setting */ + systemProperty 'tests.path.repo', new File(buildDir, "cluster/shared/repo") finalizedBy 'leaderClusterTestCluster#stop' } diff --git a/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java b/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java index 797916c7c405f..9dbf32b376565 100644 --- a/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java +++ b/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java @@ -5,20 +5,34 @@ */ package org.elasticsearch.xpack.indexlifecycle; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.xpack.ccr.ESCCRRestTestCase; +import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction; +import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy; +import org.elasticsearch.xpack.core.indexlifecycle.Phase; +import org.elasticsearch.xpack.core.indexlifecycle.UnfollowAction; import java.io.IOException; +import java.io.InputStream; +import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; +import static java.util.Collections.singletonMap; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -89,6 +103,77 @@ public void testBasicCCRAndILMIntegration() throws Exception { } } + public void testCCRUnfollowDuringSnapshot() throws Exception { + String indexName = "unfollow-test-index"; + if ("leader".equals(targetCluster)) { + Settings indexSettings = Settings.builder() + .put("index.soft_deletes.enabled", true) + .put("index.number_of_shards", 2) + .put("index.number_of_replicas", 0) + .build(); + createIndex(indexName, indexSettings); + ensureGreen(indexName); + } else if ("follow".equals(targetCluster)) { + createNewSingletonPolicy("unfollow-only", "hot", new UnfollowAction(), TimeValue.ZERO); + followIndex(indexName, indexName); + + // Create the repository before taking the snapshot. + Request request = new Request("PUT", "/_snapshot/repo"); + request.setJsonEntity(Strings + .toString(JsonXContent.contentBuilder() + .startObject() + .field("type", "fs") + .startObject("settings") + .field("compress", randomBoolean()) + .field("location", System.getProperty("tests.path.repo")) + .field("max_snapshot_bytes_per_sec", "256b") + .endObject() + .endObject())); + assertOK(client().performRequest(request)); + + try (RestClient leaderClient = buildLeaderClient()) { + index(leaderClient, indexName, "1"); + assertDocumentExists(leaderClient, indexName, "1"); + + updateIndexSettings(leaderClient, indexName, Settings.builder() + .put("index.lifecycle.indexing_complete", true) + .build()); + + // start snapshot + request = new Request("PUT", "/_snapshot/repo/snapshot"); + request.addParameter("wait_for_completion", "false"); + request.setJsonEntity("{\"indices\": \"" + indexName + "\"}"); + assertOK(client().performRequest(request)); + + // add policy and expect it to trigger unfollow immediately (while snapshot in progress) + logger.info("--> starting unfollow"); + updatePolicy(indexName, "unfollow-only"); + + assertBusy(() -> { + // Ensure that 'index.lifecycle.indexing_complete' is replicated: + assertThat(getIndexSetting(leaderClient, indexName, "index.lifecycle.indexing_complete"), equalTo("true")); + assertThat(getIndexSetting(client(), indexName, "index.lifecycle.indexing_complete"), equalTo("true")); + // ILM should have unfollowed the follower index, so the following_index setting should have been removed: + // (this controls whether the follow engine is used) + assertThat(getIndexSetting(client(), indexName, "index.xpack.ccr.following_index"), nullValue()); + // Following index should have the document + assertDocumentExists(client(), indexName, "1"); + // ILM should have completed the unfollow + assertILMPolicy(client(), indexName, "unfollow-only", "completed"); + }, 2, TimeUnit.MINUTES); + + // assert that snapshot succeeded + assertThat(getSnapshotState("snapshot"), equalTo("SUCCESS")); + assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/snapshot"))); + ResponseException e = expectThrows(ResponseException.class, + () -> client().performRequest(new Request("GET", "/_snapshot/repo/snapshot"))); + assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + } + } else { + fail("unexpected target cluster [" + targetCluster + "]"); + } + } + public void testCcrAndIlmWithRollover() throws Exception { String alias = "metrics"; String indexName = "metrics-000001"; @@ -282,4 +367,35 @@ private static void assertDocumentExists(RestClient client, String index, String assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); } + private void createNewSingletonPolicy(String policyName, String phaseName, LifecycleAction action, TimeValue after) throws IOException { + Phase phase = new Phase(phaseName, after, singletonMap(action.getWriteableName(), action)); + LifecyclePolicy lifecyclePolicy = new LifecyclePolicy(policyName, singletonMap(phase.getName(), phase)); + XContentBuilder builder = jsonBuilder(); + lifecyclePolicy.toXContent(builder, null); + final StringEntity entity = new StringEntity( + "{ \"policy\":" + Strings.toString(builder) + "}", ContentType.APPLICATION_JSON); + Request request = new Request("PUT", "_ilm/policy/" + policyName); + request.setEntity(entity); + client().performRequest(request); + } + + public static void updatePolicy(String indexName, String policy) throws IOException { + + Request changePolicyRequest = new Request("PUT", "/" + indexName + "/_settings"); + final StringEntity changePolicyEntity = new StringEntity("{ \"index.lifecycle.name\": \"" + policy + "\" }", + ContentType.APPLICATION_JSON); + changePolicyRequest.setEntity(changePolicyEntity); + assertOK(client().performRequest(changePolicyRequest)); + } + + private String getSnapshotState(String snapshot) throws IOException { + Response response = client().performRequest(new Request("GET", "/_snapshot/repo/" + snapshot)); + Map responseMap; + try (InputStream is = response.getEntity().getContent()) { + responseMap = XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true); + } + @SuppressWarnings("unchecked") Map snapResponse = ((List>) responseMap.get("snapshots")).get(0); + assertThat(snapResponse.get("snapshot"), equalTo(snapshot)); + return (String) snapResponse.get("state"); + } } diff --git a/x-pack/plugin/ilm/qa/multi-node/build.gradle b/x-pack/plugin/ilm/qa/multi-node/build.gradle index edd7f3aad472e..5f033626932df 100644 --- a/x-pack/plugin/ilm/qa/multi-node/build.gradle +++ b/x-pack/plugin/ilm/qa/multi-node/build.gradle @@ -5,6 +5,11 @@ dependencies { testCompile project(path: xpackProject('plugin').path, configuration: 'testArtifacts') } +integTestRunner { + /* To support taking index snapshots, we have to set path.repo setting */ + systemProperty 'tests.path.repo', new File(buildDir, "cluster/shared/repo") +} + integTestCluster { numNodes = 4 clusterName = 'ilm' @@ -16,5 +21,4 @@ integTestCluster { setting 'xpack.ml.enabled', 'false' setting 'xpack.license.self_generated.type', 'trial' setting 'indices.lifecycle.poll_interval', '1000ms' - } diff --git a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java index 01eba362711b3..878ddec5e5fe9 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.FrozenEngine; import org.elasticsearch.test.rest.ESRestTestCase; @@ -358,6 +359,44 @@ public void testDeleteOnlyShouldNotMakeIndexReadonly() throws Exception { indexDocument(); } + public void testDeleteDuringSnapshot() throws Exception { + // Create the repository before taking the snapshot. + Request request = new Request("PUT", "/_snapshot/repo"); + request.setJsonEntity(Strings + .toString(JsonXContent.contentBuilder() + .startObject() + .field("type", "fs") + .startObject("settings") + .field("compress", randomBoolean()) + .field("location", System.getProperty("tests.path.repo")) + .field("max_snapshot_bytes_per_sec", "256b") + .endObject() + .endObject())); + assertOK(client().performRequest(request)); + // create delete policy + createNewSingletonPolicy("delete", new DeleteAction(), TimeValue.timeValueMillis(0)); + // create index without policy + createIndexWithSettings(index, Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)); + // index document so snapshot actually does something + indexDocument(); + // start snapshot + request = new Request("PUT", "/_snapshot/repo/snapshot"); + request.addParameter("wait_for_completion", "false"); + request.setJsonEntity("{\"indices\": \"" + index + "\"}"); + assertOK(client().performRequest(request)); + // add policy and expect it to trigger delete immediately (while snapshot in progress) + updatePolicy(index, policy); + // assert that index was deleted + assertBusy(() -> assertFalse(indexExists(index)), 2, TimeUnit.MINUTES); + // assert that snapshot is still in progress and clean up + assertThat(getSnapshotState("snapshot"), equalTo("SUCCESS")); + assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/snapshot"))); + ResponseException e = expectThrows(ResponseException.class, + () -> client().performRequest(new Request("GET", "/_snapshot/repo/snapshot"))); + assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + } + public void testReadOnly() throws Exception { createIndexWithSettings(index, Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)); @@ -427,6 +466,56 @@ public void testShrinkAction() throws Exception { expectThrows(ResponseException.class, this::indexDocument); } + public void testShrinkDuringSnapshot() throws Exception { + String shrunkenIndex = ShrinkAction.SHRUNKEN_INDEX_PREFIX + index; + // Create the repository before taking the snapshot. + Request request = new Request("PUT", "/_snapshot/repo"); + request.setJsonEntity(Strings + .toString(JsonXContent.contentBuilder() + .startObject() + .field("type", "fs") + .startObject("settings") + .field("compress", randomBoolean()) + .field("location", System.getProperty("tests.path.repo")) + .field("max_snapshot_bytes_per_sec", "256b") + .endObject() + .endObject())); + assertOK(client().performRequest(request)); + // create delete policy + createNewSingletonPolicy("warm", new ShrinkAction(1), TimeValue.timeValueMillis(0)); + // create index without policy + createIndexWithSettings(index, Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + // required so the shrink doesn't wait on SetSingleNodeAllocateStep + .put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_name", "node-0")); + // index document so snapshot actually does something + indexDocument(); + // start snapshot + request = new Request("PUT", "/_snapshot/repo/snapshot"); + request.addParameter("wait_for_completion", "false"); + request.setJsonEntity("{\"indices\": \"" + index + "\"}"); + assertOK(client().performRequest(request)); + // add policy and expect it to trigger shrink immediately (while snapshot in progress) + updatePolicy(index, policy); + // assert that index was shrunk and original index was deleted + assertBusy(() -> { + assertTrue(indexExists(shrunkenIndex)); + assertTrue(aliasExists(shrunkenIndex, index)); + Map settings = getOnlyIndexSettings(shrunkenIndex); + assertThat(getStepKeyForIndex(shrunkenIndex), equalTo(TerminalPolicyStep.KEY)); + assertThat(settings.get(IndexMetaData.SETTING_NUMBER_OF_SHARDS), equalTo(String.valueOf(1))); + assertThat(settings.get(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.getKey()), equalTo("true")); + }, 2, TimeUnit.MINUTES); + expectThrows(ResponseException.class, this::indexDocument); + // assert that snapshot succeeded + assertThat(getSnapshotState("snapshot"), equalTo("SUCCESS")); + assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/snapshot"))); + ResponseException e = expectThrows(ResponseException.class, + () -> client().performRequest(new Request("GET", "/_snapshot/repo/snapshot"))); + assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + } + public void testFreezeAction() throws Exception { createIndexWithSettings(index, Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)); @@ -441,6 +530,50 @@ public void testFreezeAction() throws Exception { }); } + public void testFreezeDuringSnapshot() throws Exception { + // Create the repository before taking the snapshot. + Request request = new Request("PUT", "/_snapshot/repo"); + request.setJsonEntity(Strings + .toString(JsonXContent.contentBuilder() + .startObject() + .field("type", "fs") + .startObject("settings") + .field("compress", randomBoolean()) + .field("location", System.getProperty("tests.path.repo")) + .field("max_snapshot_bytes_per_sec", "256b") + .endObject() + .endObject())); + assertOK(client().performRequest(request)); + // create delete policy + createNewSingletonPolicy("cold", new FreezeAction(), TimeValue.timeValueMillis(0)); + // create index without policy + createIndexWithSettings(index, Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)); + // index document so snapshot actually does something + indexDocument(); + // start snapshot + request = new Request("PUT", "/_snapshot/repo/snapshot"); + request.addParameter("wait_for_completion", "false"); + request.setJsonEntity("{\"indices\": \"" + index + "\"}"); + assertOK(client().performRequest(request)); + // add policy and expect it to trigger delete immediately (while snapshot in progress) + updatePolicy(index, policy); + // assert that the index froze + assertBusy(() -> { + Map settings = getOnlyIndexSettings(index); + assertThat(getStepKeyForIndex(index), equalTo(TerminalPolicyStep.KEY)); + assertThat(settings.get(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.getKey()), equalTo("true")); + assertThat(settings.get(IndexSettings.INDEX_SEARCH_THROTTLED.getKey()), equalTo("true")); + assertThat(settings.get(FrozenEngine.INDEX_FROZEN.getKey()), equalTo("true")); + }, 2, TimeUnit.MINUTES); + // assert that snapshot is still in progress and clean up + assertThat(getSnapshotState("snapshot"), equalTo("SUCCESS")); + assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/snapshot"))); + ResponseException e = expectThrows(ResponseException.class, + () -> client().performRequest(new Request("GET", "/_snapshot/repo/snapshot"))); + assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + } + public void testSetPriority() throws Exception { createIndexWithSettings(index, Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.INDEX_PRIORITY_SETTING.getKey(), 100)); @@ -763,4 +896,15 @@ private void indexDocument() throws IOException { Response response = client().performRequest(indexRequest); logger.info(response.getStatusLine()); } + + private String getSnapshotState(String snapshot) throws IOException { + Response response = client().performRequest(new Request("GET", "/_snapshot/repo/" + snapshot)); + Map responseMap; + try (InputStream is = response.getEntity().getContent()) { + responseMap = XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true); + } + Map snapResponse = ((List>) responseMap.get("snapshots")).get(0); + assertThat(snapResponse.get("snapshot"), equalTo(snapshot)); + return (String) snapResponse.get("state"); + } } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java index baa1d8facd958..2e7d2fbbc555d 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java @@ -135,8 +135,8 @@ public Collection createComponents(Client client, ClusterService cluster if (enabled == false || transportClientMode) { return emptyList(); } - indexLifecycleInitialisationService - .set(new IndexLifecycleService(settings, client, clusterService, getClock(), System::currentTimeMillis, xContentRegistry)); + indexLifecycleInitialisationService.set(new IndexLifecycleService(settings, client, clusterService, threadPool, + getClock(), System::currentTimeMillis, xContentRegistry)); return Collections.singletonList(indexLifecycleInitialisationService.get()); } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java index 7ba7bcafe55f8..f6c068d945d79 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java @@ -11,6 +11,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; @@ -23,6 +24,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.Index; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.indexlifecycle.AsyncActionStep; import org.elasticsearch.xpack.core.indexlifecycle.AsyncWaitStep; import org.elasticsearch.xpack.core.indexlifecycle.ClusterStateActionStep; @@ -53,14 +55,17 @@ public class IndexLifecycleRunner { private static final Logger logger = LogManager.getLogger(IndexLifecycleRunner.class); private static final ToXContent.Params STACKTRACE_PARAMS = new ToXContent.MapParams(Collections.singletonMap(REST_EXCEPTION_SKIP_STACK_TRACE, "false")); + private final ThreadPool threadPool; private PolicyStepsRegistry stepRegistry; private ClusterService clusterService; private LongSupplier nowSupplier; - public IndexLifecycleRunner(PolicyStepsRegistry stepRegistry, ClusterService clusterService, LongSupplier nowSupplier) { + public IndexLifecycleRunner(PolicyStepsRegistry stepRegistry, ClusterService clusterService, + ThreadPool threadPool, LongSupplier nowSupplier) { this.stepRegistry = stepRegistry; this.clusterService = clusterService; this.nowSupplier = nowSupplier; + this.threadPool = threadPool; } /** @@ -168,7 +173,8 @@ public void maybeRunAsyncAction(ClusterState currentState, IndexMetaData indexMe } if (currentStep instanceof AsyncActionStep) { logger.debug("[{}] running policy with async action step [{}]", index, currentStep.getKey()); - ((AsyncActionStep) currentStep).performAction(indexMetaData, currentState, new AsyncActionStep.Listener() { + ((AsyncActionStep) currentStep).performAction(indexMetaData, currentState, + new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext()), new AsyncActionStep.Listener() { @Override public void onResponse(boolean complete) { diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java index b77997a94a3d6..d143e80340cc0 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java @@ -53,13 +53,14 @@ public class IndexLifecycleService private final PolicyStepsRegistry policyRegistry; private final IndexLifecycleRunner lifecycleRunner; private final Settings settings; + private final ThreadPool threadPool; private Client client; private ClusterService clusterService; private LongSupplier nowSupplier; private SchedulerEngine.Job scheduledJob; - public IndexLifecycleService(Settings settings, Client client, ClusterService clusterService, Clock clock, LongSupplier nowSupplier, - NamedXContentRegistry xContentRegistry) { + public IndexLifecycleService(Settings settings, Client client, ClusterService clusterService, ThreadPool threadPool, Clock clock, + LongSupplier nowSupplier, NamedXContentRegistry xContentRegistry) { super(); this.settings = settings; this.client = client; @@ -68,7 +69,8 @@ public IndexLifecycleService(Settings settings, Client client, ClusterService cl this.nowSupplier = nowSupplier; this.scheduledJob = null; this.policyRegistry = new PolicyStepsRegistry(xContentRegistry, client); - this.lifecycleRunner = new IndexLifecycleRunner(policyRegistry, clusterService, nowSupplier); + this.threadPool = threadPool; + this.lifecycleRunner = new IndexLifecycleRunner(policyRegistry, clusterService, threadPool, nowSupplier); this.pollInterval = LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING.get(settings); clusterService.addStateApplier(this); clusterService.addListener(this); diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/RestPutLifecycleAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/RestPutLifecycleAction.java index 586c3c683264e..aad85426fc338 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/RestPutLifecycleAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/RestPutLifecycleAction.java @@ -32,11 +32,12 @@ public String getName() { @Override protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { String lifecycleName = restRequest.param("name"); - XContentParser parser = restRequest.contentParser(); - PutLifecycleAction.Request putLifecycleRequest = PutLifecycleAction.Request.parseRequest(lifecycleName, parser); - putLifecycleRequest.timeout(restRequest.paramAsTime("timeout", putLifecycleRequest.timeout())); - putLifecycleRequest.masterNodeTimeout(restRequest.paramAsTime("master_timeout", putLifecycleRequest.masterNodeTimeout())); + try (XContentParser parser = restRequest.contentParser()) { + PutLifecycleAction.Request putLifecycleRequest = PutLifecycleAction.Request.parseRequest(lifecycleName, parser); + putLifecycleRequest.timeout(restRequest.paramAsTime("timeout", putLifecycleRequest.timeout())); + putLifecycleRequest.masterNodeTimeout(restRequest.paramAsTime("master_timeout", putLifecycleRequest.masterNodeTimeout())); - return channel -> client.execute(PutLifecycleAction.INSTANCE, putLifecycleRequest, new RestToXContentListener<>(channel)); + return channel -> client.execute(PutLifecycleAction.INSTANCE, putLifecycleRequest, new RestToXContentListener<>(channel)); + } } } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java index da51bb68962d7..61f9be3558fa7 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java @@ -87,7 +87,12 @@ public ClusterState execute(ClusterState currentState) throws Exception { SortedMap newPolicies = new TreeMap<>(currentMetadata.getPolicyMetadatas()); LifecyclePolicyMetadata lifecyclePolicyMetadata = new LifecyclePolicyMetadata(request.getPolicy(), filteredHeaders, nextVersion, Instant.now().toEpochMilli()); - newPolicies.put(lifecyclePolicyMetadata.getName(), lifecyclePolicyMetadata); + LifecyclePolicyMetadata oldPolicy = newPolicies.put(lifecyclePolicyMetadata.getName(), lifecyclePolicyMetadata); + if (oldPolicy == null) { + logger.info("adding index lifecycle policy [{}]", request.getPolicy().getName()); + } else { + logger.info("updating index lifecycle policy [{}]", request.getPolicy().getName()); + } IndexLifecycleMetadata newMetadata = new IndexLifecycleMetadata(newPolicies, OperationMode.RUNNING); newState.metaData(MetaData.builder(currentState.getMetaData()) .putCustom(IndexLifecycleMetadata.TYPE, newMetadata).build()); diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java index f4ab15a30e880..565a9723c8c6c 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -52,6 +53,8 @@ import org.elasticsearch.xpack.core.indexlifecycle.Step; import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey; import org.elasticsearch.xpack.core.indexlifecycle.TerminalPolicyStep; +import org.junit.After; +import org.junit.Before; import org.mockito.ArgumentMatcher; import org.mockito.Mockito; @@ -80,6 +83,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase { private static final NamedXContentRegistry REGISTRY; + private ThreadPool threadPool; static { try (IndexLifecycle indexLifecycle = new IndexLifecycle(Settings.EMPTY)) { @@ -88,6 +92,16 @@ public class IndexLifecycleRunnerTests extends ESTestCase { } } + @Before + public void prepareThreadPool() { + threadPool = new TestThreadPool("test"); + } + + @After + public void shutdown() { + threadPool.shutdownNow(); + } + /** A real policy steps registry where getStep can be overridden so that JSON doesn't have to be parsed */ private class MockPolicyStepsRegistry extends PolicyStepsRegistry { private BiFunction fn = null; @@ -142,7 +156,7 @@ public void testRunPolicyTerminalPolicyStep() { TerminalPolicyStep step = TerminalPolicyStep.INSTANCE; PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); IndexMetaData indexMetaData = IndexMetaData.builder("my_index").settings(settings(Version.CURRENT)) .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build(); @@ -157,7 +171,7 @@ public void testRunPolicyErrorStep() { MockClusterStateWaitStep step = new MockClusterStateWaitStep(stepKey, null); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); LifecycleExecutionState.Builder newState = LifecycleExecutionState.builder(); newState.setPhase(stepKey.getPhase()); newState.setAction(stepKey.getAction()); @@ -197,7 +211,7 @@ public void testRunStateChangePolicyWithNoNextStep() throws Exception { .localNodeId(node.getId())) .build(); ClusterServiceUtils.setState(clusterService, state); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); ClusterState before = clusterService.state(); CountDownLatch latch = new CountDownLatch(1); @@ -258,7 +272,7 @@ public void testRunStateChangePolicyWithNextStep() throws Exception { .build(); ClusterServiceUtils.setState(clusterService, state); long stepTime = randomLong(); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> stepTime); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> stepTime); ClusterState before = clusterService.state(); CountDownLatch latch = new CountDownLatch(1); @@ -307,7 +321,7 @@ public void testRunAsyncActionDoesNotRun() { .localNodeId(node.getId())) .build(); ClusterServiceUtils.setState(clusterService, state); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); ClusterState before = clusterService.state(); // State changes should not run AsyncAction steps @@ -366,7 +380,7 @@ public void testRunStateChangePolicyWithAsyncActionNextStep() throws Exception { .build(); logger.info("--> state: {}", state); ClusterServiceUtils.setState(clusterService, state); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); ClusterState before = clusterService.state(); CountDownLatch latch = new CountDownLatch(1); @@ -435,7 +449,7 @@ public void testRunPeriodicStep() throws Exception { .build(); logger.info("--> state: {}", state); ClusterServiceUtils.setState(clusterService, state); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); ClusterState before = clusterService.state(); CountDownLatch latch = new CountDownLatch(1); @@ -458,7 +472,7 @@ public void testRunPolicyClusterStateActionStep() { MockClusterStateActionStep step = new MockClusterStateActionStep(stepKey, null); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); IndexMetaData indexMetaData = IndexMetaData.builder("my_index").settings(settings(Version.CURRENT)) .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build(); @@ -476,7 +490,7 @@ public void testRunPolicyClusterStateWaitStep() { step.setWillComplete(true); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); IndexMetaData indexMetaData = IndexMetaData.builder("my_index").settings(settings(Version.CURRENT)) .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build(); @@ -495,7 +509,7 @@ public void testRunPolicyAsyncActionStepClusterStateChangeIgnored() { step.setException(expectedException); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); IndexMetaData indexMetaData = IndexMetaData.builder("my_index").settings(settings(Version.CURRENT)) .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build(); @@ -513,7 +527,7 @@ public void testRunPolicyAsyncWaitStepClusterStateChangeIgnored() { step.setException(expectedException); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); - IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, clusterService, threadPool, () -> 0L); IndexMetaData indexMetaData = IndexMetaData.builder("my_index").settings(settings(Version.CURRENT)) .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build(); @@ -527,7 +541,7 @@ public void testRunPolicyThatDoesntExist() { String policyName = "cluster_state_action_policy"; ClusterService clusterService = mock(ClusterService.class); IndexLifecycleRunner runner = new IndexLifecycleRunner(new PolicyStepsRegistry(NamedXContentRegistry.EMPTY, null), - clusterService, () -> 0L); + clusterService, threadPool, () -> 0L); IndexMetaData indexMetaData = IndexMetaData.builder("my_index").settings(settings(Version.CURRENT)) .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build(); // verify that no exception is thrown @@ -889,7 +903,7 @@ public void testMoveClusterStateToFailedStep() { ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, lifecycleState.build(), Collections.singletonList(policyMetadata)); Index index = clusterState.metaData().index(indexName).getIndex(); - IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, () -> now); + IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, threadPool, () -> now); ClusterState nextClusterState = runner.moveClusterStateToFailedStep(clusterState, indices); IndexLifecycleRunnerTests.assertClusterStateOnNextStep(clusterState, index, errorStepKey, failedStepKey, nextClusterState, now); @@ -923,7 +937,7 @@ public void testMoveClusterStateToFailedStepWithUnknownStep() { lifecycleState.setFailedStep(failedStepKey.getName()); ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, lifecycleState.build(), Collections.singletonList(policyMetadata)); - IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, () -> now); + IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, threadPool, () -> now); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> runner.moveClusterStateToFailedStep(clusterState, indices)); assertThat(exception.getMessage(), equalTo("step [" + failedStepKey @@ -935,7 +949,7 @@ public void testMoveClusterStateToFailedStepIndexNotFound() { String invalidIndexName = "does_not_exist"; ClusterState clusterState = buildClusterState(existingIndexName, Settings.builder(), LifecycleExecutionState.builder().build(), Collections.emptyList()); - IndexLifecycleRunner runner = new IndexLifecycleRunner(null, null, () -> 0L); + IndexLifecycleRunner runner = new IndexLifecycleRunner(null, null, threadPool, () -> 0L); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> runner.moveClusterStateToFailedStep(clusterState, new String[] { invalidIndexName })); assertThat(exception.getMessage(), equalTo("index [" + invalidIndexName + "] does not exist")); @@ -958,7 +972,7 @@ public void testMoveClusterStateToFailedStepInvalidPolicySetting() { lifecycleState.setStep(errorStepKey.getName()); lifecycleState.setFailedStep(failedStepKey.getName()); ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, lifecycleState.build(), Collections.emptyList()); - IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, () -> now); + IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, threadPool, () -> now); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> runner.moveClusterStateToFailedStep(clusterState, indices)); assertThat(exception.getMessage(), equalTo("index [" + indexName + "] is not associated with an Index Lifecycle Policy")); @@ -979,7 +993,7 @@ public void testMoveClusterStateToFailedNotOnError() { lifecycleState.setAction(failedStepKey.getAction()); lifecycleState.setStep(failedStepKey.getName()); ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, lifecycleState.build(), Collections.emptyList()); - IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, () -> now); + IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, threadPool, () -> now); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> runner.moveClusterStateToFailedStep(clusterState, indices)); assertThat(exception.getMessage(), equalTo("cannot retry an action for an index [" + indices[0] @@ -1176,7 +1190,7 @@ public void testIsReadyToTransition() { stepMap, NamedXContentRegistry.EMPTY, null); ClusterService clusterService = mock(ClusterService.class); final AtomicLong now = new AtomicLong(5); - IndexLifecycleRunner runner = new IndexLifecycleRunner(policyStepsRegistry, clusterService, now::get); + IndexLifecycleRunner runner = new IndexLifecycleRunner(policyStepsRegistry, clusterService, threadPool, now::get); IndexMetaData indexMetaData = IndexMetaData.builder("my_index").settings(settings(Version.CURRENT)) .numberOfShards(randomIntBetween(1, 5)) .numberOfReplicas(randomIntBetween(0, 5)) @@ -1360,7 +1374,8 @@ public void setLatch(CountDownLatch latch) { } @Override - public void performAction(IndexMetaData indexMetaData, ClusterState currentState, Listener listener) { + public void performAction(IndexMetaData indexMetaData, ClusterState currentState, + ClusterStateObserver observer, Listener listener) { executeCount++; if (latch != null) { latch.countDown(); diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java index 13fe9c1c69002..810f913775e3b 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java @@ -25,8 +25,10 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.core.indexlifecycle.LifecycleExecutionState; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata; +import org.elasticsearch.xpack.core.indexlifecycle.LifecycleExecutionState; import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy; import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicyMetadata; import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings; @@ -67,6 +69,7 @@ public class IndexLifecycleServiceTests extends ESTestCase { private DiscoveryNode masterNode; private IndicesAdminClient indicesClient; private long now; + private ThreadPool threadPool; @Before public void prepareServices() { @@ -96,7 +99,9 @@ public void prepareServices() { when(adminClient.indices()).thenReturn(indicesClient); when(client.settings()).thenReturn(Settings.EMPTY); - indexLifecycleService = new IndexLifecycleService(Settings.EMPTY, client, clusterService, clock, () -> now, null); + threadPool = new TestThreadPool("test"); + indexLifecycleService = new IndexLifecycleService(Settings.EMPTY, client, clusterService, threadPool, + clock, () -> now, null); Mockito.verify(clusterService).addListener(indexLifecycleService); Mockito.verify(clusterService).addStateApplier(indexLifecycleService); } @@ -104,6 +109,7 @@ public void prepareServices() { @After public void cleanup() { indexLifecycleService.close(); + threadPool.shutdownNow(); } diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/build.gradle b/x-pack/plugin/ml/qa/native-multi-node-tests/build.gradle index 188086afcf14a..95a2f8db2c49c 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/build.gradle +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/build.gradle @@ -36,6 +36,7 @@ integTestCluster { dependsOn copyKeyCerts setting 'xpack.security.enabled', 'true' setting 'xpack.ml.enabled', 'true' + setting 'xpack.watcher.enabled', 'false' setting 'logger.org.elasticsearch.xpack.ml.datafeed', 'TRACE' setting 'xpack.monitoring.enabled', 'false' setting 'xpack.security.authc.token.enabled', 'true' diff --git a/x-pack/plugin/watcher/build.gradle b/x-pack/plugin/watcher/build.gradle index fe6885b1c9d72..dd26db984cdf5 100644 --- a/x-pack/plugin/watcher/build.gradle +++ b/x-pack/plugin/watcher/build.gradle @@ -31,6 +31,7 @@ dependencies { compileOnly project(path: ':plugins:transport-nio', configuration: 'runtime') testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') + testCompile "org.elasticsearch.plugin:x-pack-ilm:${version}" // watcher deps compile 'com.googlecode.owasp-java-html-sanitizer:owasp-java-html-sanitizer:r239' diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java index 7b17d7f9973d1..d5eea54a80c85 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java @@ -270,7 +270,7 @@ public Collection createComponents(Client client, ClusterService cluster throw new UncheckedIOException(e); } - new WatcherIndexTemplateRegistry(clusterService, threadPool, client); + new WatcherIndexTemplateRegistry(clusterService, threadPool, client, xContentRegistry); // http client httpClient = new HttpClient(settings, getSslService(), cryptoService, clusterService); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java index 3258a7a481bb1..0fdb2b3a17d13 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java @@ -18,13 +18,20 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.XPackClient; +import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata; +import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy; +import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicyUtils; +import org.elasticsearch.xpack.core.indexlifecycle.action.PutLifecycleAction; import org.elasticsearch.xpack.core.template.TemplateUtils; import org.elasticsearch.xpack.core.watcher.support.WatcherIndexTemplateRegistryField; import java.nio.charset.StandardCharsets; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; @@ -46,17 +53,23 @@ public class WatcherIndexTemplateRegistry implements ClusterStateListener { TEMPLATE_CONFIG_TRIGGERED_WATCHES, TEMPLATE_CONFIG_WATCH_HISTORY, TEMPLATE_CONFIG_WATCHES }; + public static final PolicyConfig POLICY_WATCH_HISTORY = new PolicyConfig("watch-history-ilm-policy", "/watch-history-ilm-policy.json"); + private static final Logger logger = LogManager.getLogger(WatcherIndexTemplateRegistry.class); private final Client client; private final ThreadPool threadPool; private final TemplateConfig[] indexTemplates; + private final NamedXContentRegistry xContentRegistry; private final ConcurrentMap templateCreationsInProgress = new ConcurrentHashMap<>(); + private final AtomicBoolean historyPolicyCreationInProgress = new AtomicBoolean(); - public WatcherIndexTemplateRegistry(ClusterService clusterService, ThreadPool threadPool, Client client) { + public WatcherIndexTemplateRegistry(ClusterService clusterService, ThreadPool threadPool, Client client, + NamedXContentRegistry xContentRegistry) { this.client = client; this.threadPool = threadPool; this.indexTemplates = TEMPLATE_CONFIGS; + this.xContentRegistry = xContentRegistry; clusterService.addListener(this); } @@ -82,6 +95,7 @@ public void clusterChanged(ClusterChangedEvent event) { if (event.localNodeMaster() || localNodeVersionAfterMaster) { addTemplatesIfMissing(state); + addIndexLifecyclePolicyIfMissing(state); } } @@ -127,6 +141,54 @@ public void onFailure(Exception e) { }); } + // Package visible for testing + LifecyclePolicy loadWatcherHistoryPolicy() { + return LifecyclePolicyUtils.loadPolicy(POLICY_WATCH_HISTORY.policyName, POLICY_WATCH_HISTORY.fileName, xContentRegistry); + } + + private void addIndexLifecyclePolicyIfMissing(ClusterState state) { + if (historyPolicyCreationInProgress.compareAndSet(false, true)) { + final LifecyclePolicy policyOnDisk = loadWatcherHistoryPolicy(); + + Optional maybeMeta = Optional.ofNullable(state.metaData().custom(IndexLifecycleMetadata.TYPE)); + final boolean needsUpdating = maybeMeta + .flatMap(ilmMeta -> Optional.ofNullable(ilmMeta.getPolicies().get(policyOnDisk.getName()))) + .isPresent() == false; // If there is no policy then one needs to be put; + + if (needsUpdating) { + putPolicy(policyOnDisk, historyPolicyCreationInProgress); + } else { + historyPolicyCreationInProgress.set(false); + } + } + } + + private void putPolicy(final LifecyclePolicy policy, final AtomicBoolean creationCheck) { + final Executor executor = threadPool.generic(); + executor.execute(() -> { + PutLifecycleAction.Request request = new PutLifecycleAction.Request(policy); + request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), WATCHER_ORIGIN, request, + new ActionListener() { + @Override + public void onResponse(PutLifecycleAction.Response response) { + creationCheck.set(false); + if (response.isAcknowledged() == false) { + logger.error("error adding watcher index lifecycle policy [{}], request was not acknowledged", + policy.getName()); + } + } + + @Override + public void onFailure(Exception e) { + creationCheck.set(false); + logger.error(new ParameterizedMessage("error adding watcher index lifecycle policy [{}]", + policy.getName()), e); + } + }, (req, listener) -> new XPackClient(client).ilmClient().putLifecyclePolicy(req, listener)); + }); + } + public static boolean validate(ClusterState state) { return state.getMetaData().getTemplates().containsKey(WatcherIndexTemplateRegistryField.HISTORY_TEMPLATE_NAME) && state.getMetaData().getTemplates().containsKey(WatcherIndexTemplateRegistryField.TRIGGERED_TEMPLATE_NAME) && @@ -153,9 +215,19 @@ public String getTemplateName() { public byte[] load() { String template = TemplateUtils.loadTemplate("/" + fileName + ".json", WatcherIndexTemplateRegistryField.INDEX_TEMPLATE_VERSION, - Pattern.quote("${xpack.watcher.template.version}")); + Pattern.quote("${xpack.watcher.template.version}")); assert template != null && template.length() > 0; return template.getBytes(StandardCharsets.UTF_8); } } + public static class PolicyConfig { + + private final String policyName; + private String fileName; + + PolicyConfig(String templateName, String fileName) { + this.policyName = templateName; + this.fileName = fileName; + } + } } diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java index e93a86b93eb23..60ca2b83b2f85 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.client.IndicesAdminClient; import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlocks; @@ -21,20 +22,36 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.ParseField; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.indexlifecycle.DeleteAction; +import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata; +import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction; +import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy; +import org.elasticsearch.xpack.core.indexlifecycle.LifecycleType; +import org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType; +import org.elasticsearch.xpack.core.indexlifecycle.action.PutLifecycleAction; import org.elasticsearch.xpack.core.watcher.support.WatcherIndexTemplateRegistryField; import org.junit.Before; import org.mockito.ArgumentCaptor; +import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.elasticsearch.mock.orig.Mockito.verify; import static org.elasticsearch.mock.orig.Mockito.when; @@ -50,6 +67,7 @@ public class WatcherIndexTemplateRegistryTests extends ESTestCase { private WatcherIndexTemplateRegistry registry; + private NamedXContentRegistry xContentRegistry; private Client client; @Before @@ -72,7 +90,13 @@ public void createRegistryAndClient() { }).when(indicesAdminClient).putTemplate(any(PutIndexTemplateRequest.class), any(ActionListener.class)); ClusterService clusterService = mock(ClusterService.class); - registry = new WatcherIndexTemplateRegistry(clusterService, threadPool, client); + List entries = new ArrayList<>(ClusterModule.getNamedXWriteables()); + entries.addAll(Arrays.asList( + new NamedXContentRegistry.Entry(LifecycleType.class, new ParseField(TimeseriesLifecycleType.TYPE), + (p) -> TimeseriesLifecycleType.INSTANCE), + new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(DeleteAction.NAME), DeleteAction::parse))); + xContentRegistry = new NamedXContentRegistry(entries); + registry = new WatcherIndexTemplateRegistry(clusterService, threadPool, client, xContentRegistry); } public void testThatNonExistingTemplatesAreAddedImmediately() { @@ -91,6 +115,44 @@ public void testThatNonExistingTemplatesAreAddedImmediately() { verify(client.admin().indices(), times(4)).putTemplate(argumentCaptor.capture(), anyObject()); } + public void testThatNonExistingPoliciesAreAddedImmediately() { + DiscoveryNode node = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build(); + + ClusterChangedEvent event = createClusterChangedEvent(Collections.emptyList(), nodes); + registry.clusterChanged(event); + verify(client, times(1)).execute(eq(PutLifecycleAction.INSTANCE), anyObject(), anyObject()); + } + + public void testPolicyAlreadyExists() { + DiscoveryNode node = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build(); + + Map policyMap = new HashMap<>(); + LifecyclePolicy policy = registry.loadWatcherHistoryPolicy(); + policyMap.put(policy.getName(), policy); + ClusterChangedEvent event = createClusterChangedEvent(Collections.emptyList(), policyMap, nodes); + registry.clusterChanged(event); + verify(client, times(0)).execute(eq(PutLifecycleAction.INSTANCE), anyObject(), anyObject()); + } + + public void testPolicyAlreadyExistsButDiffers() throws IOException { + DiscoveryNode node = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build(); + + Map policyMap = new HashMap<>(); + String policyStr = "{\"phases\":{\"delete\":{\"min_age\":\"1m\",\"actions\":{\"delete\":{}}}}}"; + LifecyclePolicy policy = registry.loadWatcherHistoryPolicy(); + try (XContentParser parser = XContentType.JSON.xContent() + .createParser(xContentRegistry, LoggingDeprecationHandler.THROW_UNSUPPORTED_OPERATION, policyStr)) { + LifecyclePolicy different = LifecyclePolicy.parse(parser, policy.getName()); + policyMap.put(policy.getName(), different); + ClusterChangedEvent event = createClusterChangedEvent(Collections.emptyList(), policyMap, nodes); + registry.clusterChanged(event); + verify(client, times(0)).execute(eq(PutLifecycleAction.INSTANCE), anyObject(), anyObject()); + } + } + public void testThatTemplatesExist() { assertThat(WatcherIndexTemplateRegistry.validate(createClusterState(".watch-history")), is(false)); assertThat(WatcherIndexTemplateRegistry.validate(createClusterState(".watch-history", ".triggered_watches", ".watches")), @@ -141,6 +203,12 @@ public void testThatMissingMasterNodeDoesNothing() { } private ClusterChangedEvent createClusterChangedEvent(List existingTemplateNames, DiscoveryNodes nodes) { + return createClusterChangedEvent(existingTemplateNames, Collections.emptyMap(), nodes); + } + + private ClusterChangedEvent createClusterChangedEvent(List existingTemplateNames, + Map existingPolicies, + DiscoveryNodes nodes) { ClusterChangedEvent event = mock(ClusterChangedEvent.class); when(event.localNodeMaster()).thenReturn(nodes.isLocalNodeElectedMaster()); ClusterState cs = mock(ClusterState.class); @@ -158,6 +226,10 @@ private ClusterChangedEvent createClusterChangedEvent(List existingTempl } when(metaData.getTemplates()).thenReturn(indexTemplates.build()); + + IndexLifecycleMetadata ilmMeta = mock(IndexLifecycleMetadata.class); + when(ilmMeta.getPolicies()).thenReturn(existingPolicies); + when(metaData.custom(anyObject())).thenReturn(ilmMeta); when(cs.metaData()).thenReturn(metaData); return event; diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/AbstractWatcherIntegrationTestCase.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/AbstractWatcherIntegrationTestCase.java index 615fc8fbd08f9..6737de19baa13 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/AbstractWatcherIntegrationTestCase.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/AbstractWatcherIntegrationTestCase.java @@ -49,6 +49,7 @@ import org.elasticsearch.xpack.core.watcher.transport.actions.stats.WatcherStatsResponse; import org.elasticsearch.xpack.core.watcher.watch.ClockMock; import org.elasticsearch.xpack.core.watcher.watch.Watch; +import org.elasticsearch.xpack.indexlifecycle.IndexLifecycle; import org.elasticsearch.xpack.watcher.history.HistoryStore; import org.elasticsearch.xpack.watcher.notification.email.Authentication; import org.elasticsearch.xpack.watcher.notification.email.Email; @@ -161,6 +162,8 @@ protected List> pluginTypes() { } types.add(CommonAnalysisPlugin.class); + // ILM is required for watcher template index settings + types.add(IndexLifecycle.class); return types; }