From 75b77f82ed9871597e4d785c1d5be37e53b4bf4b Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 21 Aug 2018 15:52:29 +0700 Subject: [PATCH 01/19] [CCR] Added auto follow patterns feature Auto Following Patterns is a cross cluster replication feature that keeps track whether in the leader cluster indices are being created with names that match with a specific pattern and if so automatically let the follower cluster follow these newly created indices. This change adds an `AutoFollowCoordinator` component that is only active on the elected master node. Periodically this component checks the the cluster state of remote clusters if there new leader indices that match with configured auto follow patterns that have been defined in `AutoFollowMetadata` custom metadata. This change also adds two new APIs to manage auto follow patterns. A put auto follow pattern api: ``` PUT /_ccr/_autofollow/{{remote_cluster}} { "leader_index_pattern": ["logs-*", ...], "follow_index_pattern": "{{leader_index}}-copy", "max_concurrent_read_batches": 2 ... // other optional parameters } ``` and delete auto follow pattern api: ``` DELETE /_ccr/_autofollow/{{remote_cluster_alias}} ``` The auto follow patterns are directly tied to the remote cluster aliases configured in the follow cluster. Relates to #33007 --- .../xpack/ccr/FollowIndexIT.java | 29 ++ .../java/org/elasticsearch/xpack/ccr/Ccr.java | 25 +- .../elasticsearch/xpack/ccr/CcrSettings.java | 10 +- .../ccr/action/AutoFollowCoordinator.java | 261 ++++++++++++++ .../action/DeleteAutoFollowPatternAction.java | 81 +++++ .../action/PutAutoFollowPatternAction.java | 275 ++++++++++++++ ...ransportDeleteAutoFollowPatternAction.java | 105 ++++++ .../TransportPutAutoFollowPatternAction.java | 148 ++++++++ .../RestDeleteAutoFollowPatternAction.java | 39 ++ .../rest/RestPutAutoFollowPatternAction.java | 44 +++ .../action/AutoFollowCoordinatorTests.java | 223 ++++++++++++ .../xpack/ccr/action/AutoFollowTests.java | 189 ++++++++++ .../DeleteAutoFollowPatternRequestTests.java | 23 ++ .../PutAutoFollowPatternRequestTests.java | 63 ++++ ...nsportPutAutoFollowPatternActionTests.java | 133 +++++++ .../xpack/core/XPackClientPlugin.java | 5 +- .../xpack/core/ccr/AutoFollowMetadata.java | 336 ++++++++++++++++++ .../core/ccr/AutoFollowMetadataTests.java | 53 +++ .../api/ccr.delete_auto_follow_pattern.json | 17 + .../api/ccr.put_auto_follow_pattern.json | 21 ++ .../rest-api-spec/test/ccr/auto_follow.yml | 13 + 21 files changed, 2088 insertions(+), 5 deletions(-) create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadataTests.java create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index c14e13e7bb050..7c207cf6aa545 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -78,6 +78,35 @@ public void testFollowIndex() throws Exception { } } + public void testAutoFollowPatterns() throws Exception { + assumeFalse("Test should only run when both clusters are running", runningAgainstLeaderCluster); + + Request request = new Request("PUT", "/_ccr/_autofollow/leader_cluster"); + request.setJsonEntity("{\"leader_index_patterns\": [\"logs-*\"]}"); + assertOK(client().performRequest(request)); + + try (RestClient leaderClient = buildLeaderClient()) { + Settings settings = Settings.builder() + .put("index.soft_deletes.enabled", true) + .build(); + request = new Request("PUT", "/logs-20190101"); + request.setJsonEntity("{\"settings\": " + Strings.toString(settings) + + ", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }"); + assertOK(leaderClient.performRequest(request)); + + for (int i = 0; i < 5; i++) { + String id = Integer.toString(i); + index(leaderClient, "logs-20190101", id, "field", i, "filtered_field", "true"); + } + } + + assertBusy(() -> { + Request indexExists = new Request("HEAD", "/logs-20190101"); + assertOK(client().performRequest(indexExists)); + verifyDocuments("logs-20190101", 5); + }); + } + private static void index(RestClient client, String index, String id, Object... fields) throws IOException { XContentBuilder document = jsonBuilder().startObject(); for (int i = 0; i < fields.length; i += 2) { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java index d76af9f3c5352..896dbcab8a38b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java @@ -39,21 +39,28 @@ import org.elasticsearch.threadpool.FixedExecutorBuilder; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; +import org.elasticsearch.xpack.ccr.action.AutoFollowCoordinator; import org.elasticsearch.xpack.ccr.action.CcrStatsAction; import org.elasticsearch.xpack.ccr.action.CreateAndFollowIndexAction; +import org.elasticsearch.xpack.ccr.action.DeleteAutoFollowPatternAction; import org.elasticsearch.xpack.ccr.action.FollowIndexAction; +import org.elasticsearch.xpack.ccr.action.PutAutoFollowPatternAction; import org.elasticsearch.xpack.ccr.action.ShardChangesAction; import org.elasticsearch.xpack.ccr.action.ShardFollowNodeTask; import org.elasticsearch.xpack.ccr.action.ShardFollowTask; import org.elasticsearch.xpack.ccr.action.ShardFollowTasksExecutor; import org.elasticsearch.xpack.ccr.action.TransportCcrStatsAction; +import org.elasticsearch.xpack.ccr.action.TransportDeleteAutoFollowPatternAction; +import org.elasticsearch.xpack.ccr.action.TransportPutAutoFollowPatternAction; import org.elasticsearch.xpack.ccr.action.UnfollowIndexAction; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; import org.elasticsearch.xpack.ccr.action.bulk.TransportBulkShardOperationsAction; import org.elasticsearch.xpack.ccr.index.engine.FollowingEngineFactory; import org.elasticsearch.xpack.ccr.rest.RestCcrStatsAction; import org.elasticsearch.xpack.ccr.rest.RestCreateAndFollowIndexAction; +import org.elasticsearch.xpack.ccr.rest.RestDeleteAutoFollowPatternAction; import org.elasticsearch.xpack.ccr.rest.RestFollowIndexAction; +import org.elasticsearch.xpack.ccr.rest.RestPutAutoFollowPatternAction; import org.elasticsearch.xpack.ccr.rest.RestUnfollowIndexAction; import org.elasticsearch.xpack.core.XPackPlugin; @@ -113,7 +120,14 @@ public Collection createComponents( final Environment environment, final NodeEnvironment nodeEnvironment, final NamedWriteableRegistry namedWriteableRegistry) { - return Collections.singleton(ccrLicenseChecker); + if (enabled == false) { + return emptyList(); + } + + return Arrays.asList( + ccrLicenseChecker, + new AutoFollowCoordinator(settings, client, threadPool, clusterService) + ); } @Override @@ -133,7 +147,9 @@ public List> getPersistentTasksExecutor(ClusterServic new ActionHandler<>(CreateAndFollowIndexAction.INSTANCE, CreateAndFollowIndexAction.TransportAction.class), new ActionHandler<>(FollowIndexAction.INSTANCE, FollowIndexAction.TransportAction.class), new ActionHandler<>(ShardChangesAction.INSTANCE, ShardChangesAction.TransportAction.class), - new ActionHandler<>(UnfollowIndexAction.INSTANCE, UnfollowIndexAction.TransportAction.class)); + new ActionHandler<>(UnfollowIndexAction.INSTANCE, UnfollowIndexAction.TransportAction.class), + new ActionHandler<>(PutAutoFollowPatternAction.INSTANCE, TransportPutAutoFollowPatternAction.class), + new ActionHandler<>(DeleteAutoFollowPatternAction.INSTANCE, TransportDeleteAutoFollowPatternAction.class)); } public List getRestHandlers(Settings settings, RestController restController, ClusterSettings clusterSettings, @@ -144,7 +160,10 @@ public List getRestHandlers(Settings settings, RestController restC new RestCcrStatsAction(settings, restController), new RestCreateAndFollowIndexAction(settings, restController), new RestFollowIndexAction(settings, restController), - new RestUnfollowIndexAction(settings, restController)); + new RestUnfollowIndexAction(settings, restController), + new RestPutAutoFollowPatternAction(settings, restController), + new RestDeleteAutoFollowPatternAction(settings, restController) + ); } public List getNamedWriteables() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java index 6960766bad0cb..a942990ea5a74 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java @@ -7,6 +7,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; +import org.elasticsearch.common.unit.TimeValue; import java.util.Arrays; import java.util.List; @@ -32,6 +33,12 @@ private CcrSettings() { public static final Setting CCR_FOLLOWING_INDEX_SETTING = Setting.boolSetting("index.xpack.ccr.following_index", false, Setting.Property.IndexScope); + /** + * Setting for controlling the interval in between polling leader clusters to check whether there are indices to follow + */ + public static final Setting CCR_AUTO_FOLLOW_POLL_INTERVAL = + Setting.timeSetting("xpack.ccr.auto_follow.poll_interval", TimeValue.timeValueMillis(2500), Property.NodeScope); + /** * The settings defined by CCR. * @@ -40,7 +47,8 @@ private CcrSettings() { static List> getSettings() { return Arrays.asList( CCR_ENABLED_SETTING, - CCR_FOLLOWING_INDEX_SETTING); + CCR_FOLLOWING_INDEX_SETTING, + CCR_AUTO_FOLLOW_POLL_INTERVAL); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java new file mode 100644 index 0000000000000..a93835933a15d --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -0,0 +1,261 @@ +/* + * 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.ccr.action; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateApplier; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.ccr.CcrSettings; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; + +/** + * A component that runs only on the elected master node and follows leader indices automatically + * if they match with a auto follow pattern that is defined in {@link AutoFollowMetadata}. + */ +public class AutoFollowCoordinator implements ClusterStateApplier { + + private static final Logger LOGGER = LogManager.getLogger(AutoFollowCoordinator.class); + + private final Client client; + private final TimeValue pollInterval; + private final ThreadPool threadPool; + private final ClusterService clusterService; + + private volatile boolean localNodeMaster = false; + + public AutoFollowCoordinator(Settings settings, + Client client, + ThreadPool threadPool, + ClusterService clusterService) { + this.client = client; + this.threadPool = threadPool; + this.clusterService = clusterService; + + this.pollInterval = CcrSettings.CCR_AUTO_FOLLOW_POLL_INTERVAL.get(settings); + clusterService.addStateApplier(this); + } + + void doAutoFollow() { + if (localNodeMaster == false) { + return; + } + ClusterState localClusterState = clusterService.state(); + AutoFollowMetadata autoFollowMetadata = localClusterState.getMetaData().custom(AutoFollowMetadata.TYPE); + if (autoFollowMetadata == null) { + threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); + return; + } + + if (autoFollowMetadata.getPatterns().isEmpty()) { + threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); + return; + } + + Consumer handler = e -> { + if (e != null) { + LOGGER.error("Failure occurred during auto following indices", e); + } + threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); + }; + AutoFollower operation = new AutoFollower(client, handler, autoFollowMetadata) { + + void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + ClusterStateRequest request = new ClusterStateRequest(); + request.clear(); + request.metaData(true); + remoteClient.admin().cluster().state(request, + ActionListener.wrap( + r -> handler.accept(r.getState(), null), + e -> handler.accept(null, e) + ) + ); + } + + void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + client.execute(CreateAndFollowIndexAction.INSTANCE, new CreateAndFollowIndexAction.Request(followRequest), + ActionListener.wrap(r -> handler.accept(null), handler::accept)); + } + + void updateAutoMetadata(Function updateFunction, Consumer handler) { + clusterService.submitStateUpdateTask("update_auto_follow_metadata", new ClusterStateUpdateTask() { + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + return updateFunction.apply(currentState); + } + + @Override + public void onFailure(String source, Exception e) { + handler.accept(e); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + handler.accept(null); + } + }); + } + + }; + operation.autoFollowIndices(); + } + + @Override + public void applyClusterState(ClusterChangedEvent event) { + final boolean beforeLocalMasterNode = localNodeMaster; + localNodeMaster = event.localNodeMaster(); + if (beforeLocalMasterNode == false && localNodeMaster) { + threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); + } + } + + abstract static class AutoFollower { + + private final Client client; + private final Consumer handler; + private final AutoFollowMetadata autoFollowMetadata; + + private final AtomicInteger executedRequests = new AtomicInteger(0); + private final AtomicReference errorHolder = new AtomicReference<>(); + + AutoFollower(Client client, Consumer handler, AutoFollowMetadata autoFollowMetadata) { + this.client = client; + this.handler = handler; + this.autoFollowMetadata = autoFollowMetadata; + } + + void autoFollowIndices() { + for (Map.Entry entry : autoFollowMetadata.getPatterns().entrySet()) { + String clusterAlias = entry.getKey(); + AutoFollowMetadata.AutoFollowPattern autoFollowPattern = entry.getValue(); + Client remoteClient = clusterAlias.equals("_local_") ? client : client.getRemoteClusterClient(clusterAlias); + List followedIndices = autoFollowMetadata.getFollowedLeaderIndexUUIDS().get(clusterAlias); + + clusterStateApiCall(remoteClient, (remoteClusterState, e) -> { + if (remoteClusterState != null) { + handleClusterAlias(clusterAlias, autoFollowPattern, followedIndices, remoteClusterState); + } else { + finalise(e); + } + }); + } + } + + private void handleClusterAlias(String clusterAlias, AutoFollowMetadata.AutoFollowPattern autoFollowPattern, + List followedIndexUUIDS, ClusterState remoteClusterState) { + List leaderIndicesToFollow = new ArrayList<>(); + String[] patterns = autoFollowPattern.getLeaderIndexPatterns().toArray(new String[0]); + for (IndexMetaData indexMetaData : remoteClusterState.getMetaData()) { + if (Regex.simpleMatch(patterns, indexMetaData.getIndex().getName())) { + if (followedIndexUUIDS.contains(indexMetaData.getIndex().getUUID()) == false) { + leaderIndicesToFollow.add(indexMetaData); + } + } + } + if (leaderIndicesToFollow.isEmpty()) { + finalise(null); + } else { + AtomicInteger numRequests = new AtomicInteger(); + for (IndexMetaData indexToFollow : leaderIndicesToFollow) { + String leaderIndexName = indexToFollow.getIndex().getName(); + String followIndexName = leaderIndexName; + if (autoFollowPattern.getFollowIndexPattern() != null) { + followIndexName = autoFollowPattern.getFollowIndexPattern().replace("{{leader_index}}", leaderIndexName); + } + + String leaderIndexNameWithClusterAliasPrefix = clusterAlias.equals("_local_") ? leaderIndexName : + clusterAlias + ":" + leaderIndexName; + FollowIndexAction.Request followRequest = + new FollowIndexAction.Request(leaderIndexNameWithClusterAliasPrefix, followIndexName, + autoFollowPattern.getMaxBatchOperationCount(), autoFollowPattern.getMaxConcurrentReadBatches(), + autoFollowPattern.getMaxOperationSizeInBytes(), autoFollowPattern.getMaxConcurrentWriteBatches(), + autoFollowPattern.getMaxWriteBufferSize(), autoFollowPattern.getRetryTimeout(), + autoFollowPattern.getIdleShardRetryDelay()); + + // This runs on the elected master node, so we can update cluster state here: + Consumer handler = followError -> { + if (followError != null) { + LOGGER.error("Failed to auto follow leader index [" + leaderIndexName + "]", followError); + if (numRequests.incrementAndGet() == leaderIndicesToFollow.size()) { + finalise(followError); + } + return; + } + Function clusterStateUpdateFunction = currentState -> { + AutoFollowMetadata currentAutoFollowMetadata = currentState.metaData().custom(AutoFollowMetadata.TYPE); + + Map> newFollowedIndexUUIDS = + new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); + newFollowedIndexUUIDS.get(clusterAlias).add(indexToFollow.getIndexUUID()); + + ClusterState.Builder newState = ClusterState.builder(currentState); + AutoFollowMetadata newAutoFollowMetadata = + new AutoFollowMetadata(currentAutoFollowMetadata.getPatterns(), newFollowedIndexUUIDS); + newState.metaData(MetaData.builder(currentState.getMetaData()) + .putCustom(AutoFollowMetadata.TYPE, newAutoFollowMetadata) + .build()); + return newState.build(); + }; + updateAutoMetadata(clusterStateUpdateFunction, updateError -> { + if (updateError != null) { + LOGGER.error("Failed to mark leader index [" + leaderIndexName + "] as auto followed", updateError); + } else { + LOGGER.debug("Successfully marked leader index [{}] as auto followed", leaderIndexName); + } + if (numRequests.incrementAndGet() == leaderIndicesToFollow.size()) { + finalise(updateError); + } + }); + }; + LOGGER.info("Auto following leader index [{}] as follow index [{}]", leaderIndexName, followIndexName); + createAndFollowApiCall(followRequest, handler); + } + } + } + + private void finalise(Exception failure) { + if (errorHolder.compareAndSet(null, failure) == false) { + errorHolder.get().addSuppressed(failure); + } + + if (executedRequests.incrementAndGet() == autoFollowMetadata.getPatterns().size()) { + handler.accept(errorHolder.get()); + } + } + + // abstract methods to make unit testing possible: + + abstract void clusterStateApiCall(Client remoteClient, BiConsumer handler); + + abstract void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler); + + abstract void updateAutoMetadata(Function updateFunction, Consumer handler); + + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java new file mode 100644 index 0000000000000..0355ca525267b --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java @@ -0,0 +1,81 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class DeleteAutoFollowPatternAction extends Action { + + public static final String NAME = "cluster:admin/xpack/ccr/auto_follow_pattern/delete"; + public static final DeleteAutoFollowPatternAction INSTANCE = new DeleteAutoFollowPatternAction(); + + private DeleteAutoFollowPatternAction() { + super(NAME); + } + + @Override + public AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + public static class Request extends AcknowledgedRequest { + + private String remoteClusterAlias; + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (remoteClusterAlias == null) { + validationException = addValidationError("remoteClusterAlias is missing", validationException); + } + return validationException; + } + + public String getRemoteClusterAlias() { + return remoteClusterAlias; + } + + public void setRemoteClusterAlias(String remoteClusterAlias) { + this.remoteClusterAlias = remoteClusterAlias; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + remoteClusterAlias = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(remoteClusterAlias); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return Objects.equals(remoteClusterAlias, request.remoteClusterAlias); + } + + @Override + public int hashCode() { + return Objects.hash(remoteClusterAlias); + } + } + +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java new file mode 100644 index 0000000000000..14079686c3ab5 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java @@ -0,0 +1,275 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class PutAutoFollowPatternAction extends Action { + + public static final String NAME = "cluster:admin/xpack/ccr/auto_follow_pattern/put"; + public static final PutAutoFollowPatternAction INSTANCE = new PutAutoFollowPatternAction(); + + private PutAutoFollowPatternAction() { + super(NAME); + } + + @Override + public AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + public static class Request extends AcknowledgedRequest implements ToXContentObject { + + static final ParseField REMOTE_CLUSTER_ALIAS_FIELD = new ParseField("remote_cluster_alias"); + static final ParseField LEADER_INDEX_PATTERNS_FIELD = new ParseField("leader_index_patterns"); + static final ParseField FOLLOW_INDEX_NAME_PATTERN_FIELD = new ParseField("follow_index_name_pattern"); + + private static final ObjectParser PARSER = new ObjectParser<>("put_auto_follow_pattern_request", Request::new); + + static { + PARSER.declareString(Request::setRemoteClusterAlias, REMOTE_CLUSTER_ALIAS_FIELD); + PARSER.declareStringArray(Request::setLeaderIndexPatterns, LEADER_INDEX_PATTERNS_FIELD); + PARSER.declareString(Request::setFollowIndexNamePattern, FOLLOW_INDEX_NAME_PATTERN_FIELD); + PARSER.declareInt(Request::setMaxBatchOperationCount, AutoFollowPattern.MAX_BATCH_OPERATION_COUNT); + PARSER.declareInt(Request::setMaxConcurrentReadBatches, AutoFollowPattern.MAX_CONCURRENT_READ_BATCHES); + PARSER.declareLong(Request::setMaxOperationSizeInBytes, AutoFollowPattern.MAX_BATCH_SIZE_IN_BYTES); + PARSER.declareInt(Request::setMaxConcurrentWriteBatches, AutoFollowPattern.MAX_CONCURRENT_WRITE_BATCHES); + PARSER.declareInt(Request::setMaxWriteBufferSize, AutoFollowPattern.MAX_WRITE_BUFFER_SIZE); + PARSER.declareField(Request::setRetryTimeout, + (p, c) -> TimeValue.parseTimeValue(p.text(), AutoFollowPattern.RETRY_TIMEOUT.getPreferredName()), + ShardFollowTask.RETRY_TIMEOUT, ObjectParser.ValueType.STRING); + PARSER.declareField(Request::setIdleShardRetryDelay, + (p, c) -> TimeValue.parseTimeValue(p.text(), AutoFollowPattern.IDLE_SHARD_RETRY_DELAY.getPreferredName()), + ShardFollowTask.IDLE_SHARD_RETRY_DELAY, ObjectParser.ValueType.STRING); + } + + public static Request fromXContent(XContentParser parser, String remoteClusterAlias) throws IOException { + Request request = PARSER.parse(parser, null); + if (remoteClusterAlias != null) { + if (request.remoteClusterAlias == null) { + request.remoteClusterAlias = remoteClusterAlias; + } else { + if (request.remoteClusterAlias.equals(remoteClusterAlias) == false) { + throw new IllegalArgumentException("provided remoteClusterAlias is not equal"); + } + } + } + return request; + } + + private String remoteClusterAlias; + private List leaderIndexPatterns; + private String followIndexNamePattern; + + private Integer maxBatchOperationCount; + private Integer maxConcurrentReadBatches; + private Long maxOperationSizeInBytes; + private Integer maxConcurrentWriteBatches; + private Integer maxWriteBufferSize; + private TimeValue retryTimeout; + private TimeValue idleShardRetryDelay; + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (remoteClusterAlias == null) { + validationException = addValidationError("remoteClusterAlias is missing", validationException); + } + if (leaderIndexPatterns == null || leaderIndexPatterns.isEmpty()) { + validationException = addValidationError("leaderIndexPatterns is missing", validationException); + } + return validationException; + } + + public String getRemoteClusterAlias() { + return remoteClusterAlias; + } + + public void setRemoteClusterAlias(String remoteClusterAlias) { + this.remoteClusterAlias = remoteClusterAlias; + } + + public List getLeaderIndexPatterns() { + return leaderIndexPatterns; + } + + public void setLeaderIndexPatterns(List leaderIndexPatterns) { + this.leaderIndexPatterns = leaderIndexPatterns; + } + + public String getFollowIndexNamePattern() { + return followIndexNamePattern; + } + + public void setFollowIndexNamePattern(String followIndexNamePattern) { + this.followIndexNamePattern = followIndexNamePattern; + } + + public Integer getMaxBatchOperationCount() { + return maxBatchOperationCount; + } + + public void setMaxBatchOperationCount(Integer maxBatchOperationCount) { + this.maxBatchOperationCount = maxBatchOperationCount; + } + + public Integer getMaxConcurrentReadBatches() { + return maxConcurrentReadBatches; + } + + public void setMaxConcurrentReadBatches(Integer maxConcurrentReadBatches) { + this.maxConcurrentReadBatches = maxConcurrentReadBatches; + } + + public Long getMaxOperationSizeInBytes() { + return maxOperationSizeInBytes; + } + + public void setMaxOperationSizeInBytes(Long maxOperationSizeInBytes) { + this.maxOperationSizeInBytes = maxOperationSizeInBytes; + } + + public Integer getMaxConcurrentWriteBatches() { + return maxConcurrentWriteBatches; + } + + public void setMaxConcurrentWriteBatches(Integer maxConcurrentWriteBatches) { + this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; + } + + public Integer getMaxWriteBufferSize() { + return maxWriteBufferSize; + } + + public void setMaxWriteBufferSize(Integer maxWriteBufferSize) { + this.maxWriteBufferSize = maxWriteBufferSize; + } + + public TimeValue getRetryTimeout() { + return retryTimeout; + } + + public void setRetryTimeout(TimeValue retryTimeout) { + this.retryTimeout = retryTimeout; + } + + public TimeValue getIdleShardRetryDelay() { + return idleShardRetryDelay; + } + + public void setIdleShardRetryDelay(TimeValue idleShardRetryDelay) { + this.idleShardRetryDelay = idleShardRetryDelay; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + remoteClusterAlias = in.readString(); + leaderIndexPatterns = in.readList(StreamInput::readString); + followIndexNamePattern = in.readOptionalString(); + maxBatchOperationCount = in.readOptionalVInt(); + maxConcurrentReadBatches = in.readOptionalVInt(); + maxOperationSizeInBytes = in.readOptionalLong(); + maxConcurrentWriteBatches = in.readOptionalVInt(); + maxWriteBufferSize = in.readOptionalVInt(); + retryTimeout = in.readOptionalTimeValue(); + idleShardRetryDelay = in.readOptionalTimeValue(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(remoteClusterAlias); + out.writeStringList(leaderIndexPatterns); + out.writeOptionalString(followIndexNamePattern); + out.writeOptionalVInt(maxBatchOperationCount); + out.writeOptionalVInt(maxConcurrentReadBatches); + out.writeOptionalLong(maxOperationSizeInBytes); + out.writeOptionalVInt(maxConcurrentWriteBatches); + out.writeOptionalVInt(maxWriteBufferSize); + out.writeOptionalTimeValue(retryTimeout); + out.writeOptionalTimeValue(idleShardRetryDelay); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + { + builder.field(REMOTE_CLUSTER_ALIAS_FIELD.getPreferredName(), remoteClusterAlias); + builder.field(LEADER_INDEX_PATTERNS_FIELD.getPreferredName(), leaderIndexPatterns); + if (followIndexNamePattern != null) { + builder.field(FOLLOW_INDEX_NAME_PATTERN_FIELD.getPreferredName(), followIndexNamePattern); + } + if (maxBatchOperationCount != null) { + builder.field(ShardFollowTask.MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); + } + if (maxOperationSizeInBytes != null) { + builder.field(ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); + } + if (maxWriteBufferSize != null) { + builder.field(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize); + } + if (maxConcurrentReadBatches != null) { + builder.field(ShardFollowTask.MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); + } + if (maxConcurrentWriteBatches != null) { + builder.field(ShardFollowTask.MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + } + if (retryTimeout != null) { + builder.field(ShardFollowTask.RETRY_TIMEOUT.getPreferredName(), retryTimeout.getStringRep()); + } + if (idleShardRetryDelay != null) { + builder.field(ShardFollowTask.IDLE_SHARD_RETRY_DELAY.getPreferredName(), idleShardRetryDelay.getStringRep()); + } + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return Objects.equals(remoteClusterAlias, request.remoteClusterAlias) && + Objects.equals(leaderIndexPatterns, request.leaderIndexPatterns) && + Objects.equals(followIndexNamePattern, request.followIndexNamePattern) && + Objects.equals(maxBatchOperationCount, request.maxBatchOperationCount) && + Objects.equals(maxConcurrentReadBatches, request.maxConcurrentReadBatches) && + Objects.equals(maxOperationSizeInBytes, request.maxOperationSizeInBytes) && + Objects.equals(maxConcurrentWriteBatches, request.maxConcurrentWriteBatches) && + Objects.equals(maxWriteBufferSize, request.maxWriteBufferSize) && + Objects.equals(retryTimeout, request.retryTimeout) && + Objects.equals(idleShardRetryDelay, request.idleShardRetryDelay); + } + + @Override + public int hashCode() { + return Objects.hash(remoteClusterAlias, leaderIndexPatterns, followIndexNamePattern, maxBatchOperationCount, + maxConcurrentReadBatches, maxOperationSizeInBytes, maxConcurrentWriteBatches, maxWriteBufferSize, + retryTimeout, idleShardRetryDelay); + } + } + +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java new file mode 100644 index 0000000000000..657e2cd83da78 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java @@ -0,0 +1,105 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public class TransportDeleteAutoFollowPatternAction extends + TransportMasterNodeAction { + + @Inject + public TransportDeleteAutoFollowPatternAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver) { + super(settings, DeleteAutoFollowPatternAction.NAME, transportService, clusterService, threadPool, actionFilters, + indexNameExpressionResolver, DeleteAutoFollowPatternAction.Request::new); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + @Override + protected void masterOperation(DeleteAutoFollowPatternAction.Request request, + ClusterState state, + ActionListener listener) throws Exception { + clusterService.submitStateUpdateTask("put_auto_follow_pattern-" + request.getRemoteClusterAlias(), + new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected AcknowledgedResponse newResponse(boolean acknowledged) { + return new AcknowledgedResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + return innerDelete(request, currentState); + } + }); + } + + static ClusterState innerDelete(DeleteAutoFollowPatternAction.Request request, ClusterState currentState) { + AutoFollowMetadata currentAutoFollowMetadata = currentState.metaData().custom(AutoFollowMetadata.TYPE); + if (currentAutoFollowMetadata == null) { + return currentState; + } + Map configurations = currentAutoFollowMetadata.getPatterns(); + Set toRemove = new HashSet<>(); + for (String configurationKey : configurations.keySet()) { + if (request.getRemoteClusterAlias().equals(configurationKey)) { + toRemove.add(configurationKey); + } + } + if (toRemove.isEmpty()) { + throw new ResourceNotFoundException("auto follow pattern [{}] is missing", request.getRemoteClusterAlias()); + } else if (toRemove.isEmpty()) { + return currentState; + } + final Map configurationsCopy = new HashMap<>(configurations); + for (String key : toRemove) { + configurationsCopy.remove(key); + } + AutoFollowMetadata newAutoFollowMetadata = + new AutoFollowMetadata(configurationsCopy, currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); + ClusterState.Builder newState = ClusterState.builder(currentState); + newState.metaData(MetaData.builder(currentState.getMetaData()) + .putCustom(AutoFollowMetadata.TYPE, newAutoFollowMetadata) + .build()); + return newState.build(); + } + + @Override + protected ClusterBlockException checkBlock(DeleteAutoFollowPatternAction.Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java new file mode 100644 index 0000000000000..77ab019b554e9 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java @@ -0,0 +1,148 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TransportPutAutoFollowPatternAction extends + TransportMasterNodeAction { + + private final Client client; + + @Inject + public TransportPutAutoFollowPatternAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, Client client, + IndexNameExpressionResolver indexNameExpressionResolver) { + super(settings, PutAutoFollowPatternAction.NAME, transportService, clusterService, threadPool, actionFilters, + indexNameExpressionResolver, PutAutoFollowPatternAction.Request::new); + this.client = client; + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + @Override + protected void masterOperation(PutAutoFollowPatternAction.Request request, + ClusterState state, + ActionListener listener) throws Exception { + final Client remoteClient; + if (request.getRemoteClusterAlias().equals("_local_")) { + remoteClient = client; + } else { + remoteClient = client.getRemoteClusterClient(request.getRemoteClusterAlias()); + } + + final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.clear(); + clusterStateRequest.metaData(true); + + remoteClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(resp -> { + clusterService.submitStateUpdateTask("put_auto_follow_pattern-" + request.getRemoteClusterAlias(), + new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected AcknowledgedResponse newResponse(boolean acknowledged) { + return new AcknowledgedResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + return innerPut(request, currentState, resp.getState()); + } + }); + }, listener::onFailure)); + } + + static ClusterState innerPut(PutAutoFollowPatternAction.Request request, ClusterState localState, ClusterState remoteClusterState) { + // auto patterns are always overwritten + // only already followed index uuids are updated + + AutoFollowMetadata currentAutoFollowMetadata = localState.metaData().custom(AutoFollowMetadata.TYPE); + Map> followedLeaderIndices; + Map configurations; + if (currentAutoFollowMetadata != null) { + configurations = new HashMap<>(currentAutoFollowMetadata.getPatterns()); + followedLeaderIndices = new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); + } else { + configurations = new HashMap<>(); + followedLeaderIndices = new HashMap<>(); + } + + AutoFollowMetadata.AutoFollowPattern previousPattern = configurations.get(request.getRemoteClusterAlias()); + List followedIndexUUIDS = followedLeaderIndices.get(request.getRemoteClusterAlias()); + if (followedIndexUUIDS == null) { + followedIndexUUIDS = new ArrayList<>(); + followedLeaderIndices.put(request.getRemoteClusterAlias(), followedIndexUUIDS); + } + + // Mark existing leader indices as already auto followed: + if (previousPattern != null) { + for (String newPattern : request.getLeaderIndexPatterns()) { + if (previousPattern.getLeaderIndexPatterns().contains(newPattern) == false) { + for (IndexMetaData indexMetaData : remoteClusterState.getMetaData()) { + if (Regex.simpleMatch(newPattern, indexMetaData.getIndex().getName())) { + followedIndexUUIDS.add(indexMetaData.getIndexUUID()); + } + } + } + } + } else { + for (IndexMetaData indexMetaData : remoteClusterState.getMetaData()) { + String[] patterns = request.getLeaderIndexPatterns().toArray(new String[0]); + if (Regex.simpleMatch(patterns, indexMetaData.getIndex().getName())) { + followedIndexUUIDS.add(indexMetaData.getIndexUUID()); + } + } + } + + AutoFollowMetadata.AutoFollowPattern autoFollowPattern = new AutoFollowMetadata.AutoFollowPattern(request.getLeaderIndexPatterns(), + request.getFollowIndexNamePattern(), request.getMaxBatchOperationCount(), request.getMaxConcurrentReadBatches(), + request.getMaxOperationSizeInBytes(), request.getMaxConcurrentWriteBatches(), request.getMaxWriteBufferSize(), + request.getRetryTimeout(), request.getIdleShardRetryDelay()); + configurations.put(request.getRemoteClusterAlias(), autoFollowPattern); + ClusterState.Builder newState = ClusterState.builder(localState); + newState.metaData(MetaData.builder(localState.getMetaData()) + .putCustom(AutoFollowMetadata.TYPE, new AutoFollowMetadata(configurations, followedLeaderIndices)) + .build()); + return newState.build(); + } + + @Override + protected ClusterBlockException checkBlock(PutAutoFollowPatternAction.Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java new file mode 100644 index 0000000000000..7070017d20940 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java @@ -0,0 +1,39 @@ +/* + * 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.ccr.rest; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.ccr.action.DeleteAutoFollowPatternAction.Request; + +import java.io.IOException; + +import static org.elasticsearch.xpack.ccr.action.DeleteAutoFollowPatternAction.INSTANCE; + +public class RestDeleteAutoFollowPatternAction extends BaseRestHandler { + + public RestDeleteAutoFollowPatternAction(Settings settings, RestController controller) { + super(settings); + controller.registerHandler(RestRequest.Method.DELETE, "/_ccr/_autofollow/{remote_cluster_alias}", this); + } + + @Override + public String getName() { + return "ccr_delete_auto_folow_pattern_action"; + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + Request request = new Request(); + request.setRemoteClusterAlias(restRequest.param("remote_cluster_alias")); + return channel -> client.execute(INSTANCE, request, new RestToXContentListener<>(channel)); + } + +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java new file mode 100644 index 0000000000000..cb32117e3c854 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java @@ -0,0 +1,44 @@ +/* + * 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.ccr.rest; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.ccr.action.PutAutoFollowPatternAction.Request; + +import java.io.IOException; + +import static org.elasticsearch.xpack.ccr.action.PutAutoFollowPatternAction.INSTANCE; + +public class RestPutAutoFollowPatternAction extends BaseRestHandler { + + public RestPutAutoFollowPatternAction(Settings settings, RestController controller) { + super(settings); + controller.registerHandler(RestRequest.Method.PUT, "/_ccr/_autofollow/{remote_cluster_alias}", this); + } + + @Override + public String getName() { + return "ccr_put_auto_folow_pattern_action"; + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + Request request = createRequest(restRequest); + return channel -> client.execute(INSTANCE, request, new RestToXContentListener<>(channel)); + } + + static Request createRequest(RestRequest restRequest) throws IOException { + try (XContentParser parser = restRequest.contentOrSourceParamParser()) { + return Request.fromXContent(parser, restRequest.param("remote_cluster_alias")); + } + } +} diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java new file mode 100644 index 0000000000000..d27d92cea2821 --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java @@ -0,0 +1,223 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.Version; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class AutoFollowCoordinatorTests extends ESTestCase { + + public void testAutoFollower() { + Client client = mock(Client.class); + when(client.getRemoteClusterClient(anyString())).thenReturn(client); + + ClusterState remoteState = ClusterState.builder(new ClusterName("remote")) + .metaData(MetaData.builder().put(IndexMetaData.builder("logs-20190101") + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0))) + .build(); + + AutoFollowPattern autoFollowPattern = + new AutoFollowPattern(Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null); + Map patterns = new HashMap<>(); + patterns.put("remote", autoFollowPattern); + Map> followedLeaderIndexUUIDS = new HashMap<>(); + followedLeaderIndexUUIDS.put("remote", new ArrayList<>()); + AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata(patterns, followedLeaderIndexUUIDS); + + ClusterState currentState = ClusterState.builder(new ClusterName("name")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, autoFollowMetadata)) + .build(); + + boolean[] invoked = new boolean[]{false}; + Consumer handler = e -> { + invoked[0] = true; + assertThat(e, nullValue()); + }; + AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + @Override + void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + handler.accept(remoteState, null); + } + + @Override + void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); + assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); + handler.accept(null); + } + + @Override + void updateAutoMetadata(Function updateFunction, Consumer handler) { + ClusterState resultCs = updateFunction.apply(currentState); + AutoFollowMetadata result = resultCs.metaData().custom(AutoFollowMetadata.TYPE); + assertThat(result.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); + assertThat(result.getFollowedLeaderIndexUUIDS().get("remote").size(), equalTo(1)); + handler.accept(null); + } + }; + autoFollower.autoFollowIndices(); + assertThat(invoked[0], is(true)); + } + + public void testAutoFollowerClusterStateApiFailure() { + Client client = mock(Client.class); + when(client.getRemoteClusterClient(anyString())).thenReturn(client); + + AutoFollowPattern autoFollowPattern = + new AutoFollowPattern(Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null); + Map patterns = new HashMap<>(); + patterns.put("remote", autoFollowPattern); + Map> followedLeaderIndexUUIDS = new HashMap<>(); + followedLeaderIndexUUIDS.put("remote", new ArrayList<>()); + AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata(patterns, followedLeaderIndexUUIDS); + + Exception failure = new RuntimeException("failure"); + boolean[] invoked = new boolean[]{false}; + Consumer handler = e -> { + invoked[0] = true; + assertThat(e, sameInstance(failure)); + }; + AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + @Override + void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + handler.accept(null, failure); + } + + @Override + void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + fail("should not get here"); + } + + @Override + void updateAutoMetadata(Function updateFunction, Consumer handler) { + fail("should not get here"); + } + }; + autoFollower.autoFollowIndices(); + assertThat(invoked[0], is(true)); + } + + public void testAutoFollowerUpdateClusterStateFailure() { + Client client = mock(Client.class); + when(client.getRemoteClusterClient(anyString())).thenReturn(client); + + ClusterState remoteState = ClusterState.builder(new ClusterName("remote")) + .metaData(MetaData.builder().put(IndexMetaData.builder("logs-20190101") + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0))) + .build(); + + AutoFollowPattern autoFollowPattern = + new AutoFollowPattern(Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null); + Map patterns = new HashMap<>(); + patterns.put("remote", autoFollowPattern); + Map> followedLeaderIndexUUIDS = new HashMap<>(); + followedLeaderIndexUUIDS.put("remote", new ArrayList<>()); + AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata(patterns, followedLeaderIndexUUIDS); + + Exception failure = new RuntimeException("failure"); + boolean[] invoked = new boolean[]{false}; + Consumer handler = e -> { + invoked[0] = true; + assertThat(e, sameInstance(failure)); + }; + AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + @Override + void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + handler.accept(remoteState, null); + } + + @Override + void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); + assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); + handler.accept(null); + } + + @Override + void updateAutoMetadata(Function updateFunction, Consumer handler) { + handler.accept(failure); + } + }; + autoFollower.autoFollowIndices(); + assertThat(invoked[0], is(true)); + } + + public void testAutoFollowerCreateAndFollowApiCallFailure() { + Client client = mock(Client.class); + when(client.getRemoteClusterClient(anyString())).thenReturn(client); + + ClusterState remoteState = ClusterState.builder(new ClusterName("remote")) + .metaData(MetaData.builder().put(IndexMetaData.builder("logs-20190101") + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0))) + .build(); + + AutoFollowPattern autoFollowPattern = + new AutoFollowPattern(Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null); + Map patterns = new HashMap<>(); + patterns.put("remote", autoFollowPattern); + Map> followedLeaderIndexUUIDS = new HashMap<>(); + followedLeaderIndexUUIDS.put("remote", new ArrayList<>()); + AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata(patterns, followedLeaderIndexUUIDS); + + Exception failure = new RuntimeException("failure"); + boolean[] invoked = new boolean[]{false}; + Consumer handler = e -> { + invoked[0] = true; + assertThat(e, sameInstance(failure)); + }; + AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + @Override + void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + handler.accept(remoteState, null); + } + + @Override + void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); + assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); + handler.accept(failure); + } + + @Override + void updateAutoMetadata(Function updateFunction, Consumer handler) { + fail("should not get here"); + } + }; + autoFollower.autoFollowIndices(); + assertThat(invoked[0], is(true)); + } + +} diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java new file mode 100644 index 0000000000000..f3682a93a2202 --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java @@ -0,0 +1,189 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.persistent.PersistentTasksCustomMetaData; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xpack.ccr.LocalStateCcr; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; + +public class AutoFollowTests extends ESSingleNodeTestCase { + + @Override + protected Collection> getPlugins() { + return Collections.singleton(LocalStateCcr.class); + } + + @Override + protected boolean resetNodeAfterTest() { + return true; + } + + public void testAutoFollow() throws Exception { + Settings leaderIndexSettings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) + .build(); + + createIndex("logs-201812", leaderIndexSettings, "_doc"); + + // Enabling auto following: + putAutoFollowPatterns("logs-*", "transactions-*"); + + createIndex("metrics-201901", leaderIndexSettings, "_doc"); + + createIndex("logs-201901", leaderIndexSettings, "_doc"); + assertBusy(() -> { + IndicesExistsRequest request = new IndicesExistsRequest("copy-logs-201901"); + assertTrue(client().admin().indices().exists(request).actionGet().isExists()); + }); + createIndex("transactions-201901", leaderIndexSettings, "_doc"); + assertBusy(() -> { + IndicesExistsRequest request = new IndicesExistsRequest("copy-transactions-201901"); + assertTrue(client().admin().indices().exists(request).actionGet().isExists()); + }); + + IndicesExistsRequest request = new IndicesExistsRequest("copy-metrics-201901"); + assertFalse(client().admin().indices().exists(request).actionGet().isExists()); + request = new IndicesExistsRequest("copy-logs-201812"); + assertFalse(client().admin().indices().exists(request).actionGet().isExists()); + } + + public void testAutoFollowManyIndices() throws Exception { + Settings leaderIndexSettings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) + .build(); + + putAutoFollowPatterns("logs-*"); + int numIndices = randomIntBetween(4, 32); + for (int i = 0; i < numIndices; i++) { + createIndex("logs-" + i, leaderIndexSettings, "_doc"); + } + int expectedVal1 = numIndices; + assertBusy(() -> { + MetaData metaData = client().admin().cluster().prepareState().get().getState().metaData(); + int count = (int) Arrays.stream(metaData.getConcreteAllIndices()).filter(s -> s.startsWith("copy-")).count(); + assertThat(count, equalTo(expectedVal1)); + }); + + deleteAutoFollowPatternSetting(); + createIndex("logs-does-not-count", leaderIndexSettings, "_doc"); + + putAutoFollowPatterns("logs-*"); + int i = numIndices; + numIndices = numIndices + randomIntBetween(4, 32); + for (; i < numIndices; i++) { + createIndex("logs-" + i, leaderIndexSettings, "_doc"); + } + int expectedVal2 = numIndices; + assertBusy(() -> { + MetaData metaData = client().admin().cluster().prepareState().get().getState().metaData(); + int count = (int) Arrays.stream(metaData.getConcreteAllIndices()).filter(s -> s.startsWith("copy-")).count(); + assertThat(count, equalTo(expectedVal2)); + }); + } + + public void testAutoFollowParameterAreDelegated() throws Exception { + Settings leaderIndexSettings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) + .build(); + + // Enabling auto following: + PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias("_local_"); + request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); + // Need to set this, because following an index in the same cluster + request.setFollowIndexNamePattern("copy-{{leader_index}}"); + if (randomBoolean()) { + request.setMaxWriteBufferSize(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxConcurrentReadBatches(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxConcurrentWriteBatches(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxBatchOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxOperationSizeInBytes(randomNonNegativeLong()); + } + if (randomBoolean()) { + request.setRetryTimeout(TimeValue.timeValueMillis(500)); + } + if (randomBoolean()) { + request.setIdleShardRetryDelay(TimeValue.timeValueMillis(500)); + } + assertTrue(client().execute(PutAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); + + createIndex("logs-201901", leaderIndexSettings, "_doc"); + assertBusy(() -> { + PersistentTasksCustomMetaData persistentTasksMetaData = + client().admin().cluster().prepareState().get().getState().getMetaData().custom(PersistentTasksCustomMetaData.TYPE); + assertThat(persistentTasksMetaData, notNullValue()); + assertThat(persistentTasksMetaData.tasks().size(), equalTo(1)); + ShardFollowTask shardFollowTask = (ShardFollowTask) persistentTasksMetaData.tasks().iterator().next().getParams(); + assertThat(shardFollowTask.getLeaderShardId().getIndexName(), equalTo("logs-201901")); + assertThat(shardFollowTask.getFollowShardId().getIndexName(), equalTo("copy-logs-201901")); + if (request.getMaxWriteBufferSize() != null) { + assertThat(shardFollowTask.getMaxWriteBufferSize(), equalTo(request.getMaxWriteBufferSize())); + } + if (request.getMaxConcurrentReadBatches() != null) { + assertThat(shardFollowTask.getMaxConcurrentReadBatches(), equalTo(request.getMaxConcurrentReadBatches())); + } + if (request.getMaxConcurrentWriteBatches() != null) { + assertThat(shardFollowTask.getMaxConcurrentWriteBatches(), equalTo(request.getMaxConcurrentWriteBatches())); + } + if (request.getMaxBatchOperationCount() != null) { + assertThat(shardFollowTask.getMaxBatchOperationCount(), equalTo(request.getMaxBatchOperationCount())); + } + if (request.getMaxOperationSizeInBytes() != null) { + assertThat(shardFollowTask.getMaxBatchSizeInBytes(), equalTo(request.getMaxOperationSizeInBytes())); + } + if (request.getRetryTimeout() != null) { + assertThat(shardFollowTask.getRetryTimeout(), equalTo(request.getRetryTimeout())); + } + if (request.getIdleShardRetryDelay() != null) { + assertThat(shardFollowTask.getIdleShardRetryDelay(), equalTo(request.getIdleShardRetryDelay())); + } + }); + } + + private void putAutoFollowPatterns(String... patterns) { + PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias("_local_"); + request.setLeaderIndexPatterns(Arrays.asList(patterns)); + // Need to set this, because following an index in the same cluster + request.setFollowIndexNamePattern("copy-{{leader_index}}"); + assertTrue(client().execute(PutAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); + } + + private void deleteAutoFollowPatternSetting() { + DeleteAutoFollowPatternAction.Request request = new DeleteAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias("_local_"); + assertTrue(client().execute(DeleteAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); + } + +} diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java new file mode 100644 index 0000000000000..1b5192f253fab --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java @@ -0,0 +1,23 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.test.AbstractStreamableTestCase; + +public class DeleteAutoFollowPatternRequestTests extends AbstractStreamableTestCase { + + @Override + protected DeleteAutoFollowPatternAction.Request createBlankInstance() { + return new DeleteAutoFollowPatternAction.Request(); + } + + @Override + protected DeleteAutoFollowPatternAction.Request createTestInstance() { + DeleteAutoFollowPatternAction.Request request = new DeleteAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias(randomAlphaOfLength(4)); + return request; + } +} diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java new file mode 100644 index 0000000000000..ae0cb68cbfd3f --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java @@ -0,0 +1,63 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractStreamableXContentTestCase; + +import java.io.IOException; +import java.util.Arrays; + +public class PutAutoFollowPatternRequestTests extends AbstractStreamableXContentTestCase { + + @Override + protected boolean supportsUnknownFields() { + return false; + } + + @Override + protected PutAutoFollowPatternAction.Request doParseInstance(XContentParser parser) throws IOException { + return PutAutoFollowPatternAction.Request.fromXContent(parser, null); + } + + @Override + protected PutAutoFollowPatternAction.Request createBlankInstance() { + return new PutAutoFollowPatternAction.Request(); + } + + @Override + protected PutAutoFollowPatternAction.Request createTestInstance() { + PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias(randomAlphaOfLength(4)); + request.setLeaderIndexPatterns(Arrays.asList(generateRandomStringArray(4, 4, false))); + if (randomBoolean()) { + request.setFollowIndexNamePattern(randomAlphaOfLength(4)); + } + if (randomBoolean()) { + request.setIdleShardRetryDelay(TimeValue.timeValueMillis(500)); + } + if (randomBoolean()) { + request.setRetryTimeout(TimeValue.timeValueMillis(500)); + } + if (randomBoolean()) { + request.setMaxBatchOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxConcurrentReadBatches(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxConcurrentWriteBatches(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxOperationSizeInBytes(randomNonNegativeLong()); + } + if (randomBoolean()) { + request.setMaxWriteBufferSize(randomIntBetween(0, Integer.MAX_VALUE)); + } + return request; + } +} diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java new file mode 100644 index 0000000000000..0f298e75aa9be --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java @@ -0,0 +1,133 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; + +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.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; + +public class TransportPutAutoFollowPatternActionTests extends ESTestCase { + + public void testInnerPut() { + PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias("eu_cluster"); + request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); + + ClusterState localState = ClusterState.builder(new ClusterName("us_cluster")) + .metaData(MetaData.builder()) + .build(); + + ClusterState remoteState = ClusterState.builder(new ClusterName("eu_cluster")) + .metaData(MetaData.builder()) + .build(); + + ClusterState result = TransportPutAutoFollowPatternAction.innerPut(request, localState, remoteState); + AutoFollowMetadata autoFollowMetadata = result.metaData().custom(AutoFollowMetadata.TYPE); + assertThat(autoFollowMetadata, notNullValue()); + assertThat(autoFollowMetadata.getPatterns().size(), equalTo(1)); + assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().size(), equalTo(1)); + assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(0), equalTo("logs-*")); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().get("eu_cluster").size(), equalTo(0)); + } + + public void testInnerPut_existingLeaderIndices() { + PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias("eu_cluster"); + request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); + + ClusterState localState = ClusterState.builder(new ClusterName("us_cluster")) + .metaData(MetaData.builder()) + .build(); + + int numLeaderIndices = randomIntBetween(1, 8); + int numMatchingLeaderIndices = randomIntBetween(1, 8); + MetaData.Builder mdBuilder = MetaData.builder(); + for (int i = 0; i < numLeaderIndices; i++) { + mdBuilder.put(IndexMetaData.builder("transactions-" + i) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0)); + } + for (int i = 0; i < numMatchingLeaderIndices; i++) { + mdBuilder.put(IndexMetaData.builder("logs-" + i) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0)); + } + + ClusterState remoteState = ClusterState.builder(new ClusterName("eu_cluster")) + .metaData(mdBuilder) + .build(); + + ClusterState result = TransportPutAutoFollowPatternAction.innerPut(request, localState, remoteState); + AutoFollowMetadata autoFollowMetadata = result.metaData().custom(AutoFollowMetadata.TYPE); + assertThat(autoFollowMetadata, notNullValue()); + assertThat(autoFollowMetadata.getPatterns().size(), equalTo(1)); + assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().size(), equalTo(1)); + assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(0), equalTo("logs-*")); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().get("eu_cluster").size(), equalTo(numMatchingLeaderIndices)); + } + + public void testInnerPut_existingLeaderIndicesAndAutoFollowMetadata() { + PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); + request.setRemoteClusterAlias("eu_cluster"); + request.setLeaderIndexPatterns(Arrays.asList("logs-*", "transactions-*")); + + Map existingAutoFollowPatterns = new HashMap<>(); + List existingPatterns = new ArrayList<>(); + existingPatterns.add("transactions-*"); + existingAutoFollowPatterns.put("eu_cluster", + new AutoFollowMetadata.AutoFollowPattern(existingPatterns, null, null, null, null, null, null, null, null)); + Map> existingAlreadyFollowedIndexUUIDS = new HashMap<>(); + List existingUUIDS = new ArrayList<>(); + existingUUIDS.add("_val"); + existingAlreadyFollowedIndexUUIDS.put("eu_cluster", existingUUIDS); + ClusterState localState = ClusterState.builder(new ClusterName("us_cluster")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, + new AutoFollowMetadata(existingAutoFollowPatterns, existingAlreadyFollowedIndexUUIDS))) + .build(); + + int numLeaderIndices = randomIntBetween(1, 8); + MetaData.Builder mdBuilder = MetaData.builder(); + for (int i = 0; i < numLeaderIndices; i++) { + mdBuilder.put(IndexMetaData.builder("logs-" + i) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0)); + } + + ClusterState remoteState = ClusterState.builder(new ClusterName("eu_cluster")) + .metaData(mdBuilder) + .build(); + + ClusterState result = TransportPutAutoFollowPatternAction.innerPut(request, localState, remoteState); + AutoFollowMetadata autoFollowMetadata = result.metaData().custom(AutoFollowMetadata.TYPE); + assertThat(autoFollowMetadata, notNullValue()); + assertThat(autoFollowMetadata.getPatterns().size(), equalTo(1)); + assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().size(), equalTo(2)); + assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(0), equalTo("logs-*")); + assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(1), equalTo("transactions-*")); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().get("eu_cluster").size(), equalTo(numLeaderIndices + 1)); + } + +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index 0389ceffbc3d6..05f159e242186 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -38,6 +38,7 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.xpack.core.action.XPackInfoAction; import org.elasticsearch.xpack.core.action.XPackUsageAction; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction; import org.elasticsearch.xpack.core.graph.GraphFeatureSetUsage; import org.elasticsearch.xpack.core.graph.action.GraphExploreAction; @@ -364,7 +365,9 @@ public List getNamedWriteables() { new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.ROLLUP, RollupFeatureSetUsage::new), new NamedWriteableRegistry.Entry(PersistentTaskParams.class, RollupJob.NAME, RollupJob::new), new NamedWriteableRegistry.Entry(Task.Status.class, RollupJobStatus.NAME, RollupJobStatus::new), - new NamedWriteableRegistry.Entry(PersistentTaskState.class, RollupJobStatus.NAME, RollupJobStatus::new) + new NamedWriteableRegistry.Entry(PersistentTaskState.class, RollupJobStatus.NAME, RollupJobStatus::new), + // ccr + new NamedWriteableRegistry.Entry(AutoFollowMetadata.class, AutoFollowMetadata.TYPE, AutoFollowMetadata::new) ); } 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 new file mode 100644 index 0000000000000..c5008ff1d2e66 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java @@ -0,0 +1,336 @@ +/* + * 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.ccr; + +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.AbstractNamedDiffable; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.xpack.core.security.xcontent.XContentUtils; + +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Custom metadata that contains auto follow patterns and what leader indices an auto follow pattern has already followed. + */ +public class AutoFollowMetadata extends AbstractNamedDiffable implements MetaData.Custom { + + public static final String TYPE = "ccr_auto_follow"; + + private static final ParseField PATTERNS_FIELD = new ParseField("patterns"); + private static final ParseField FOLLOWED_LEADER_INDICES_FIELD = new ParseField("followed_leader_indices"); + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("autofollow", + args -> new AutoFollowMetadata((Map) args[0], (Map>) args[1])); + + static { + PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> { + Map patterns = new HashMap<>(); + String fieldName = null; + for (XContentParser.Token token = p.nextToken(); token != XContentParser.Token.END_OBJECT; token = p.nextToken()) { + if (token == XContentParser.Token.FIELD_NAME) { + fieldName = p.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + patterns.put(fieldName, AutoFollowPattern.PARSER.parse(p, c)); + } else { + throw new ElasticsearchParseException("unexpected token [" + token + "]"); + } + } + return patterns; + }, PATTERNS_FIELD); + PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> { + Map> alreadyFollowedIndexUUIDS = new HashMap<>(); + String fieldName = null; + for (XContentParser.Token token = p.nextToken(); token != XContentParser.Token.END_OBJECT; token = p.nextToken()) { + if (token == XContentParser.Token.FIELD_NAME) { + fieldName = p.currentName(); + } else if (token == XContentParser.Token.START_ARRAY) { + alreadyFollowedIndexUUIDS.put(fieldName, Arrays.asList(XContentUtils.readStringArray(p, false))); + } else { + throw new ElasticsearchParseException("unexpected token [" + token + "]"); + } + } + return alreadyFollowedIndexUUIDS; + }, FOLLOWED_LEADER_INDICES_FIELD); + } + + public static AutoFollowMetadata fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + private final Map patterns; + private final Map> followedLeaderIndexUUIDS; + + public AutoFollowMetadata(Map patterns, Map> followedLeaderIndexUUIDS) { + this.patterns = patterns; + this.followedLeaderIndexUUIDS = followedLeaderIndexUUIDS; + } + + public AutoFollowMetadata(StreamInput in) throws IOException { + patterns = in.readMap(StreamInput::readString, AutoFollowPattern::new); + followedLeaderIndexUUIDS = in.readMapOfLists(StreamInput::readString, StreamInput::readString); + } + + public Map getPatterns() { + return patterns; + } + + public Map> getFollowedLeaderIndexUUIDS() { + return followedLeaderIndexUUIDS; + } + + @Override + public EnumSet context() { + // TODO: When a snapshot is restored do we want to restore this? + // (Otherwise we could start following indices automatically immediately) + return MetaData.ALL_CONTEXTS; + } + + @Override + public String getWriteableName() { + return TYPE; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT.minimumCompatibilityVersion(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeMap(patterns, StreamOutput::writeString, (out1, value) -> value.writeTo(out1)); + out.writeMapOfLists(followedLeaderIndexUUIDS, StreamOutput::writeString, StreamOutput::writeString); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(PATTERNS_FIELD.getPreferredName()); + for (Map.Entry entry : patterns.entrySet()) { + builder.startObject(entry.getKey()); + builder.value(entry.getValue()); + builder.endObject(); + } + builder.endObject(); + + builder.startObject(FOLLOWED_LEADER_INDICES_FIELD.getPreferredName()); + for (Map.Entry> entry : followedLeaderIndexUUIDS.entrySet()) { + builder.field(entry.getKey(), entry.getValue()); + } + builder.endObject(); + return builder; + } + + @Override + public boolean isFragment() { + return true; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AutoFollowMetadata that = (AutoFollowMetadata) o; + return Objects.equals(patterns, that.patterns); + } + + @Override + public int hashCode() { + return Objects.hash(patterns); + } + + public static class AutoFollowPattern implements Writeable, ToXContentObject { + + private static final ParseField LEADER_PATTERNS_FIELD = new ParseField("leader_patterns"); + private static final ParseField FOLLOW_PATTERN_FIELD = new ParseField("follow_pattern"); + public static final ParseField MAX_BATCH_OPERATION_COUNT = new ParseField("max_batch_operation_count"); + public static final ParseField MAX_CONCURRENT_READ_BATCHES = new ParseField("max_concurrent_read_batches"); + public static final ParseField MAX_BATCH_SIZE_IN_BYTES = new ParseField("max_batch_size_in_bytes"); + public static final ParseField MAX_CONCURRENT_WRITE_BATCHES = new ParseField("max_concurrent_write_batches"); + public static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); + public static final ParseField RETRY_TIMEOUT = new ParseField("retry_timeout"); + public static final ParseField IDLE_SHARD_RETRY_DELAY = new ParseField("idle_shard_retry_delay"); + + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("auto_follow_pattern", + args -> new AutoFollowPattern((List) args[0], (String) args[1], (Integer) args[2], (Integer) args[3], + (Long) args[4], (Integer) args[5], (Integer) args[6], (TimeValue) args[7], (TimeValue) args[8])); + + static { + PARSER.declareStringArray(ConstructingObjectParser.constructorArg(), LEADER_PATTERNS_FIELD); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), FOLLOW_PATTERN_FIELD); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_BATCH_OPERATION_COUNT); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_CONCURRENT_READ_BATCHES); + PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), MAX_BATCH_SIZE_IN_BYTES); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_CONCURRENT_WRITE_BATCHES); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_WRITE_BUFFER_SIZE); + PARSER.declareField(ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> TimeValue.parseTimeValue(p.text(), RETRY_TIMEOUT.getPreferredName()), + RETRY_TIMEOUT, ObjectParser.ValueType.STRING); + PARSER.declareField(ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> TimeValue.parseTimeValue(p.text(), IDLE_SHARD_RETRY_DELAY.getPreferredName()), + IDLE_SHARD_RETRY_DELAY, ObjectParser.ValueType.STRING); + } + + private final List leaderIndexPatterns; + private final String followIndexPattern; + private final Integer maxBatchOperationCount; + private final Integer maxConcurrentReadBatches; + private final Long maxOperationSizeInBytes; + private final Integer maxConcurrentWriteBatches; + private final Integer maxWriteBufferSize; + private final TimeValue retryTimeout; + private final TimeValue idleShardRetryDelay; + + public AutoFollowPattern(List leaderIndexPatterns, String followIndexPattern, Integer maxBatchOperationCount, + Integer maxConcurrentReadBatches, Long maxOperationSizeInBytes, Integer maxConcurrentWriteBatches, + Integer maxWriteBufferSize, TimeValue retryTimeout, TimeValue idleShardRetryDelay) { + this.leaderIndexPatterns = leaderIndexPatterns; + this.followIndexPattern = followIndexPattern; + this.maxBatchOperationCount = maxBatchOperationCount; + this.maxConcurrentReadBatches = maxConcurrentReadBatches; + this.maxOperationSizeInBytes = maxOperationSizeInBytes; + this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; + this.maxWriteBufferSize = maxWriteBufferSize; + this.retryTimeout = retryTimeout; + this.idleShardRetryDelay = idleShardRetryDelay; + } + + public AutoFollowPattern(StreamInput in) throws IOException { + leaderIndexPatterns = in.readList(StreamInput::readString); + followIndexPattern = in.readOptionalString(); + maxBatchOperationCount = in.readOptionalVInt(); + maxConcurrentReadBatches = in.readOptionalVInt(); + maxOperationSizeInBytes = in.readOptionalLong(); + maxConcurrentWriteBatches = in.readOptionalVInt(); + maxWriteBufferSize = in.readOptionalVInt(); + retryTimeout = in.readOptionalTimeValue(); + idleShardRetryDelay = in.readOptionalTimeValue(); + } + + public List getLeaderIndexPatterns() { + return leaderIndexPatterns; + } + + public String getFollowIndexPattern() { + return followIndexPattern; + } + + public Integer getMaxBatchOperationCount() { + return maxBatchOperationCount; + } + + public Integer getMaxConcurrentReadBatches() { + return maxConcurrentReadBatches; + } + + public Long getMaxOperationSizeInBytes() { + return maxOperationSizeInBytes; + } + + public Integer getMaxConcurrentWriteBatches() { + return maxConcurrentWriteBatches; + } + + public Integer getMaxWriteBufferSize() { + return maxWriteBufferSize; + } + + public TimeValue getRetryTimeout() { + return retryTimeout; + } + + public TimeValue getIdleShardRetryDelay() { + return idleShardRetryDelay; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeStringList(leaderIndexPatterns); + out.writeOptionalString(followIndexPattern); + out.writeOptionalVInt(maxBatchOperationCount); + out.writeOptionalVInt(maxConcurrentReadBatches); + out.writeOptionalLong(maxOperationSizeInBytes); + out.writeOptionalVInt(maxConcurrentWriteBatches); + out.writeOptionalVInt(maxWriteBufferSize); + out.writeOptionalTimeValue(retryTimeout); + out.writeOptionalTimeValue(idleShardRetryDelay); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.array(LEADER_PATTERNS_FIELD.getPreferredName(), leaderIndexPatterns.toArray(new String[0])); + if (followIndexPattern != null) { + builder.field(FOLLOW_PATTERN_FIELD.getPreferredName(), followIndexPattern); + } + if (maxBatchOperationCount != null) { + builder.field(MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); + } + if (maxConcurrentReadBatches != null) { + builder.field(MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); + } + if (maxOperationSizeInBytes != null) { + builder.field(MAX_BATCH_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); + } + if (maxConcurrentWriteBatches != null) { + builder.field(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + } + if (maxWriteBufferSize != null){ + builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize); + } + if (retryTimeout != null) { + builder.field(RETRY_TIMEOUT.getPreferredName(), retryTimeout); + } + if (idleShardRetryDelay != null) { + builder.field(IDLE_SHARD_RETRY_DELAY.getPreferredName(), idleShardRetryDelay); + } + return builder; + } + + @Override + public boolean isFragment() { + return true; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AutoFollowPattern that = (AutoFollowPattern) o; + return Objects.equals(leaderIndexPatterns, that.leaderIndexPatterns) && + Objects.equals(followIndexPattern, that.followIndexPattern) && + Objects.equals(maxBatchOperationCount, that.maxBatchOperationCount) && + Objects.equals(maxConcurrentReadBatches, that.maxConcurrentReadBatches) && + Objects.equals(maxOperationSizeInBytes, that.maxOperationSizeInBytes) && + Objects.equals(maxConcurrentWriteBatches, that.maxConcurrentWriteBatches) && + Objects.equals(maxWriteBufferSize, that.maxWriteBufferSize) && + Objects.equals(retryTimeout, that.retryTimeout) && + Objects.equals(idleShardRetryDelay, that.idleShardRetryDelay); + } + + @Override + public int hashCode() { + + return Objects.hash(leaderIndexPatterns, followIndexPattern, maxBatchOperationCount, maxConcurrentReadBatches, maxOperationSizeInBytes, maxConcurrentWriteBatches, maxWriteBufferSize, retryTimeout, idleShardRetryDelay); + } + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadataTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadataTests.java new file mode 100644 index 0000000000000..5227c04962a7b --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadataTests.java @@ -0,0 +1,53 @@ +/* + * 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.ccr; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractSerializingTestCase; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; + +public class AutoFollowMetadataTests extends AbstractSerializingTestCase { + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + return s -> true; + } + + @Override + protected AutoFollowMetadata doParseInstance(XContentParser parser) throws IOException { + return AutoFollowMetadata.fromXContent(parser); + } + + @Override + protected AutoFollowMetadata createTestInstance() { + int numEntries = randomIntBetween(0, 32); + Map configs = new HashMap<>(numEntries); + Map> followedLeaderIndices = new HashMap<>(numEntries); + for (int i = 0; i < numEntries; i++) { + List leaderPatterns = Arrays.asList(generateRandomStringArray(4, 4, false)); + AutoFollowMetadata.AutoFollowPattern autoFollowPattern = + new AutoFollowMetadata.AutoFollowPattern(leaderPatterns, randomAlphaOfLength(4), randomIntBetween(0, Integer.MAX_VALUE), + randomIntBetween(0, Integer.MAX_VALUE), randomNonNegativeLong(), randomIntBetween(0, Integer.MAX_VALUE), + randomIntBetween(0, Integer.MAX_VALUE), TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(500)); + configs.put(Integer.toString(i), autoFollowPattern); + followedLeaderIndices.put(Integer.toString(i), Arrays.asList(generateRandomStringArray(4, 4, false))); + } + return new AutoFollowMetadata(configs, followedLeaderIndices); + } + + @Override + protected Writeable.Reader instanceReader() { + return AutoFollowMetadata::new; + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json new file mode 100644 index 0000000000000..9b3e46d4bddce --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json @@ -0,0 +1,17 @@ +{ + "ccr.delete_auto_follow_pattern": { + "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current", + "methods": [ "DELETE" ], + "url": { + "path": "/_ccr/_autofollow/{remote_cluster_alias}", + "paths": [ "/_ccr/_autofollow/{remote_cluster_alias}" ], + "parts": { + "remote_cluster_alias": { + "type": "string", + "required": true, + "description": "The name of the remote cluster alias." + } + } + } + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json new file mode 100644 index 0000000000000..ee28109f53b96 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json @@ -0,0 +1,21 @@ +{ + "ccr.put_auto_follow_pattern": { + "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current", + "methods": [ "PUT" ], + "url": { + "path": "/_ccr/_autofollow/{remote_cluster_alias}", + "paths": [ "/_ccr/_autofollow/{remote_cluster_alias}" ], + "parts": { + "remote_cluster_alias": { + "type": "string", + "required": true, + "description": "The name of the remote cluster alias." + } + } + }, + "body": { + "description" : "The specification of the auto follow pattern", + "required" : true + } + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml new file mode 100644 index 0000000000000..c6974e9f37bb6 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml @@ -0,0 +1,13 @@ +--- +"Test put and delete auto follow pattern": + - do: + ccr.put_auto_follow_pattern: + remote_cluster_alias: _local_ + body: + leader_index_patterns: ['logs-*'] + - is_true: acknowledged + + - do: + ccr.delete_auto_follow_pattern: + remote_cluster_alias: _local_ + - is_true: acknowledged From 78c178a543e74d9f14c2f896ba26710bb2883eb5 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 27 Aug 2018 10:56:28 +0700 Subject: [PATCH 02/19] iter --- .../xpack/core/ccr/AutoFollowMetadata.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) 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 c5008ff1d2e66..1ba7c919ed738 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 @@ -19,6 +19,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.security.xcontent.XContentUtils; import java.io.IOException; @@ -32,14 +33,15 @@ /** * Custom metadata that contains auto follow patterns and what leader indices an auto follow pattern has already followed. */ -public class AutoFollowMetadata extends AbstractNamedDiffable implements MetaData.Custom { +public class AutoFollowMetadata extends AbstractNamedDiffable implements XPackPlugin.XPackMetaDataCustom { public static final String TYPE = "ccr_auto_follow"; private static final ParseField PATTERNS_FIELD = new ParseField("patterns"); private static final ParseField FOLLOWED_LEADER_INDICES_FIELD = new ParseField("followed_leader_indices"); - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("autofollow", + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("auto_follow", args -> new AutoFollowMetadata((Map) args[0], (Map>) args[1])); static { @@ -112,7 +114,7 @@ public String getWriteableName() { @Override public Version getMinimalSupportedVersion() { - return Version.CURRENT.minimumCompatibilityVersion(); + return Version.V_6_5_0.minimumCompatibilityVersion(); } @Override @@ -169,6 +171,7 @@ public static class AutoFollowPattern implements Writeable, ToXContentObject { public static final ParseField RETRY_TIMEOUT = new ParseField("retry_timeout"); public static final ParseField IDLE_SHARD_RETRY_DELAY = new ParseField("idle_shard_retry_delay"); + @SuppressWarnings("unchecked") private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("auto_follow_pattern", args -> new AutoFollowPattern((List) args[0], (String) args[1], (Integer) args[2], (Integer) args[3], @@ -214,7 +217,7 @@ public AutoFollowPattern(List leaderIndexPatterns, String followIndexPat this.idleShardRetryDelay = idleShardRetryDelay; } - public AutoFollowPattern(StreamInput in) throws IOException { + AutoFollowPattern(StreamInput in) throws IOException { leaderIndexPatterns = in.readList(StreamInput::readString); followIndexPattern = in.readOptionalString(); maxBatchOperationCount = in.readOptionalVInt(); @@ -328,8 +331,8 @@ public boolean equals(Object o) { @Override public int hashCode() { - - return Objects.hash(leaderIndexPatterns, followIndexPattern, maxBatchOperationCount, maxConcurrentReadBatches, maxOperationSizeInBytes, maxConcurrentWriteBatches, maxWriteBufferSize, retryTimeout, idleShardRetryDelay); + return Objects.hash(leaderIndexPatterns, followIndexPattern, maxBatchOperationCount, maxConcurrentReadBatches, + maxOperationSizeInBytes, maxConcurrentWriteBatches, maxWriteBufferSize, retryTimeout, idleShardRetryDelay); } } From 80b517edde6f344b6b6e61593f1cbf991645476f Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 27 Aug 2018 11:15:43 +0700 Subject: [PATCH 03/19] single line for each field in hashcode method --- .../xpack/core/ccr/AutoFollowMetadata.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) 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 1ba7c919ed738..09a26bf896606 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 @@ -103,7 +103,7 @@ public Map> getFollowedLeaderIndexUUIDS() { @Override public EnumSet context() { // TODO: When a snapshot is restored do we want to restore this? - // (Otherwise we could start following indices automatically immediately) + // (Otherwise we would start following indices automatically immediately) return MetaData.ALL_CONTEXTS; } @@ -331,8 +331,17 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(leaderIndexPatterns, followIndexPattern, maxBatchOperationCount, maxConcurrentReadBatches, - maxOperationSizeInBytes, maxConcurrentWriteBatches, maxWriteBufferSize, retryTimeout, idleShardRetryDelay); + return Objects.hash( + leaderIndexPatterns, + followIndexPattern, + maxBatchOperationCount, + maxConcurrentReadBatches, + maxOperationSizeInBytes, + maxConcurrentWriteBatches, + maxWriteBufferSize, + retryTimeout, + idleShardRetryDelay + ); } } From f818a5d21296358b145c45aa32d5fef0de736dc4 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 27 Aug 2018 11:52:03 +0700 Subject: [PATCH 04/19] fixed delete auto follow pattern and added unit tests for it --- ...ransportDeleteAutoFollowPatternAction.java | 17 ++-- ...ortDeleteAutoFollowPatternActionTests.java | 98 +++++++++++++++++++ 2 files changed, 109 insertions(+), 6 deletions(-) create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java index 657e2cd83da78..f6ab3914d47b3 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -71,7 +72,8 @@ public ClusterState execute(ClusterState currentState) throws Exception { static ClusterState innerDelete(DeleteAutoFollowPatternAction.Request request, ClusterState currentState) { AutoFollowMetadata currentAutoFollowMetadata = currentState.metaData().custom(AutoFollowMetadata.TYPE); if (currentAutoFollowMetadata == null) { - return currentState; + throw new ResourceNotFoundException("auto follow patterns for [{}] cluster alias are missing", + request.getRemoteClusterAlias()); } Map configurations = currentAutoFollowMetadata.getPatterns(); Set toRemove = new HashSet<>(); @@ -81,16 +83,19 @@ static ClusterState innerDelete(DeleteAutoFollowPatternAction.Request request, C } } if (toRemove.isEmpty()) { - throw new ResourceNotFoundException("auto follow pattern [{}] is missing", request.getRemoteClusterAlias()); - } else if (toRemove.isEmpty()) { - return currentState; + throw new ResourceNotFoundException("auto follow patterns for [{}] cluster alias are missing", + request.getRemoteClusterAlias()); } + final Map configurationsCopy = new HashMap<>(configurations); + final Map> followedLeaderIndexUUIDSCopy = + new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); for (String key : toRemove) { configurationsCopy.remove(key); + followedLeaderIndexUUIDSCopy.remove(key); } - AutoFollowMetadata newAutoFollowMetadata = - new AutoFollowMetadata(configurationsCopy, currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); + + AutoFollowMetadata newAutoFollowMetadata = new AutoFollowMetadata(configurationsCopy, followedLeaderIndexUUIDSCopy); ClusterState.Builder newState = ClusterState.builder(currentState); newState.metaData(MetaData.builder(currentState.getMetaData()) .putCustom(AutoFollowMetadata.TYPE, newAutoFollowMetadata) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java new file mode 100644 index 0000000000000..390b7a460c278 --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java @@ -0,0 +1,98 @@ +/* + * 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.ccr.action; + +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.ccr.action.DeleteAutoFollowPatternAction.Request; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; + +public class TransportDeleteAutoFollowPatternActionTests extends ESTestCase { + + public void testInnerDelete() { + Map> existingAlreadyFollowedIndexUUIDS = new HashMap<>(); + Map existingAutoFollowPatterns = new HashMap<>(); + { + List existingPatterns = new ArrayList<>(); + existingPatterns.add("transactions-*"); + existingAutoFollowPatterns.put("eu_cluster", + new AutoFollowMetadata.AutoFollowPattern(existingPatterns, null, null, null, null, null, null, null, null)); + + List existingUUIDS = new ArrayList<>(); + existingUUIDS.add("_val"); + existingAlreadyFollowedIndexUUIDS.put("eu_cluster", existingUUIDS); + } + { + List existingPatterns = new ArrayList<>(); + existingPatterns.add("logs-*"); + existingAutoFollowPatterns.put("asia_cluster", + new AutoFollowMetadata.AutoFollowPattern(existingPatterns, null, null, null, null, null, null, null, null)); + + List existingUUIDS = new ArrayList<>(); + existingUUIDS.add("_val"); + existingAlreadyFollowedIndexUUIDS.put("asia_cluster", existingUUIDS); + } + ClusterState clusterState = ClusterState.builder(new ClusterName("us_cluster")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, + new AutoFollowMetadata(existingAutoFollowPatterns, existingAlreadyFollowedIndexUUIDS))) + .build(); + + Request request = new Request(); + request.setRemoteClusterAlias("eu_cluster"); + AutoFollowMetadata result = TransportDeleteAutoFollowPatternAction.innerDelete(request, clusterState) + .getMetaData() + .custom(AutoFollowMetadata.TYPE); + assertThat(result.getPatterns().size(), equalTo(1)); + assertThat(result.getPatterns().get("asia_cluster"), notNullValue()); + assertThat(result.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); + assertThat(result.getFollowedLeaderIndexUUIDS().get("asia_cluster"), notNullValue()); + } + + public void testInnerDeleteDoesNotExist() { + Map> existingAlreadyFollowedIndexUUIDS = new HashMap<>(); + Map existingAutoFollowPatterns = new HashMap<>(); + { + List existingPatterns = new ArrayList<>(); + existingPatterns.add("transactions-*"); + existingAutoFollowPatterns.put("eu_cluster", + new AutoFollowMetadata.AutoFollowPattern(existingPatterns, null, null, null, null, null, null, null, null)); + } + ClusterState clusterState = ClusterState.builder(new ClusterName("us_cluster")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, + new AutoFollowMetadata(existingAutoFollowPatterns, existingAlreadyFollowedIndexUUIDS))) + .build(); + + Request request = new Request(); + request.setRemoteClusterAlias("asia_cluster"); + Exception e = expectThrows(ResourceNotFoundException.class, + () -> TransportDeleteAutoFollowPatternAction.innerDelete(request, clusterState)); + assertThat(e.getMessage(), equalTo("auto follow patterns for [asia_cluster] cluster alias are missing")); + } + + public void testInnerDeleteNoAutoFollowMetadata() { + ClusterState clusterState = ClusterState.builder(new ClusterName("us_cluster")) + .metaData(MetaData.builder()) + .build(); + + Request request = new Request(); + request.setRemoteClusterAlias("asia_cluster"); + Exception e = expectThrows(ResourceNotFoundException.class, + () -> TransportDeleteAutoFollowPatternAction.innerDelete(request, clusterState)); + assertThat(e.getMessage(), equalTo("auto follow patterns for [asia_cluster] cluster alias are missing")); + } + +} From c7c2287813f61a924c3fb6ef3708e4de5ac5d683 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 28 Aug 2018 11:51:40 +0700 Subject: [PATCH 05/19] iter and moved logic to determine leader indices to follow and logic to determine the follow index name into testable methods. --- .../ccr/action/AutoFollowCoordinator.java | 110 ++++++++++-------- ...ransportDeleteAutoFollowPatternAction.java | 2 +- .../TransportPutAutoFollowPatternAction.java | 2 +- .../action/AutoFollowCoordinatorTests.java | 108 +++++++++++++---- ...ortDeleteAutoFollowPatternActionTests.java | 4 +- ...nsportPutAutoFollowPatternActionTests.java | 12 +- .../xpack/core/ccr/AutoFollowMetadata.java | 16 +-- 7 files changed, 165 insertions(+), 89 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index a93835933a15d..e567a3cc30e45 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -20,15 +20,17 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.index.Index; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.CcrSettings; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -61,7 +63,7 @@ public AutoFollowCoordinator(Settings settings, clusterService.addStateApplier(this); } - void doAutoFollow() { + private void doAutoFollow() { if (localNodeMaster == false) { return; } @@ -79,17 +81,17 @@ void doAutoFollow() { Consumer handler = e -> { if (e != null) { - LOGGER.error("Failure occurred during auto following indices", e); + LOGGER.warn("Failure occurred during auto following indices", e); } threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); }; AutoFollower operation = new AutoFollower(client, handler, autoFollowMetadata) { - void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + void getLeaderClusterState(Client leaderClient, BiConsumer handler) { ClusterStateRequest request = new ClusterStateRequest(); request.clear(); request.metaData(true); - remoteClient.admin().cluster().state(request, + leaderClient.admin().cluster().state(request, ActionListener.wrap( r -> handler.accept(r.getState(), null), e -> handler.accept(null, e) @@ -97,12 +99,12 @@ void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Consumer handler) { client.execute(CreateAndFollowIndexAction.INSTANCE, new CreateAndFollowIndexAction.Request(followRequest), - ActionListener.wrap(r -> handler.accept(null), handler::accept)); + ActionListener.wrap(r -> handler.accept(null), handler)); } - void updateAutoMetadata(Function updateFunction, Consumer handler) { + void updateAutoFollowMetadata(Function updateFunction, Consumer handler) { clusterService.submitStateUpdateTask("update_auto_follow_metadata", new ClusterStateUpdateTask() { @Override @@ -141,25 +143,27 @@ abstract static class AutoFollower { private final Consumer handler; private final AutoFollowMetadata autoFollowMetadata; - private final AtomicInteger executedRequests = new AtomicInteger(0); + private final CountDown countDown; private final AtomicReference errorHolder = new AtomicReference<>(); AutoFollower(Client client, Consumer handler, AutoFollowMetadata autoFollowMetadata) { this.client = client; this.handler = handler; this.autoFollowMetadata = autoFollowMetadata; + this.countDown = new CountDown(autoFollowMetadata.getPatterns().size()); } void autoFollowIndices() { - for (Map.Entry entry : autoFollowMetadata.getPatterns().entrySet()) { + for (Map.Entry entry : autoFollowMetadata.getPatterns().entrySet()) { String clusterAlias = entry.getKey(); - AutoFollowMetadata.AutoFollowPattern autoFollowPattern = entry.getValue(); - Client remoteClient = clusterAlias.equals("_local_") ? client : client.getRemoteClusterClient(clusterAlias); - List followedIndices = autoFollowMetadata.getFollowedLeaderIndexUUIDS().get(clusterAlias); - - clusterStateApiCall(remoteClient, (remoteClusterState, e) -> { - if (remoteClusterState != null) { - handleClusterAlias(clusterAlias, autoFollowPattern, followedIndices, remoteClusterState); + AutoFollowPattern autoFollowPattern = entry.getValue(); + Client leaderClient = clusterAlias.equals("_local_") ? client : client.getRemoteClusterClient(clusterAlias); + List followedIndices = autoFollowMetadata.getFollowedLeaderIndexUUIDs().get(clusterAlias); + + getLeaderClusterState(leaderClient, (leaderClusterState, e) -> { + if (leaderClusterState != null) { + assert e == null; + handleClusterAlias(clusterAlias, autoFollowPattern, followedIndices, leaderClusterState); } else { finalise(e); } @@ -167,27 +171,16 @@ void autoFollowIndices() { } } - private void handleClusterAlias(String clusterAlias, AutoFollowMetadata.AutoFollowPattern autoFollowPattern, - List followedIndexUUIDS, ClusterState remoteClusterState) { - List leaderIndicesToFollow = new ArrayList<>(); - String[] patterns = autoFollowPattern.getLeaderIndexPatterns().toArray(new String[0]); - for (IndexMetaData indexMetaData : remoteClusterState.getMetaData()) { - if (Regex.simpleMatch(patterns, indexMetaData.getIndex().getName())) { - if (followedIndexUUIDS.contains(indexMetaData.getIndex().getUUID()) == false) { - leaderIndicesToFollow.add(indexMetaData); - } - } - } + private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollowPattern, + List followedIndexUUIDs, ClusterState leaderClusterState) { + List leaderIndicesToFollow = getLeaderIndicesToFollow(autoFollowPattern, leaderClusterState, followedIndexUUIDs); if (leaderIndicesToFollow.isEmpty()) { finalise(null); } else { - AtomicInteger numRequests = new AtomicInteger(); - for (IndexMetaData indexToFollow : leaderIndicesToFollow) { - String leaderIndexName = indexToFollow.getIndex().getName(); - String followIndexName = leaderIndexName; - if (autoFollowPattern.getFollowIndexPattern() != null) { - followIndexName = autoFollowPattern.getFollowIndexPattern().replace("{{leader_index}}", leaderIndexName); - } + CountDown countDown = new CountDown(leaderIndicesToFollow.size()); + for (Index indexToFollow : leaderIndicesToFollow) { + final String leaderIndexName = indexToFollow.getName(); + final String followIndexName = getFollowerIndexName(autoFollowPattern, leaderIndexName); String leaderIndexNameWithClusterAliasPrefix = clusterAlias.equals("_local_") ? leaderIndexName : clusterAlias + ":" + leaderIndexName; @@ -201,18 +194,19 @@ private void handleClusterAlias(String clusterAlias, AutoFollowMetadata.AutoFoll // This runs on the elected master node, so we can update cluster state here: Consumer handler = followError -> { if (followError != null) { - LOGGER.error("Failed to auto follow leader index [" + leaderIndexName + "]", followError); - if (numRequests.incrementAndGet() == leaderIndicesToFollow.size()) { + LOGGER.warn("Failed to auto follow leader index [" + leaderIndexName + "]", followError); + if (countDown.countDown()) { finalise(followError); } return; } + LOGGER.info("Auto followed leader index [{}] as follow index [{}]", leaderIndexName, followIndexName); Function clusterStateUpdateFunction = currentState -> { AutoFollowMetadata currentAutoFollowMetadata = currentState.metaData().custom(AutoFollowMetadata.TYPE); Map> newFollowedIndexUUIDS = - new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); - newFollowedIndexUUIDS.get(clusterAlias).add(indexToFollow.getIndexUUID()); + new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); + newFollowedIndexUUIDS.get(clusterAlias).add(indexToFollow.getUUID()); ClusterState.Builder newState = ClusterState.builder(currentState); AutoFollowMetadata newAutoFollowMetadata = @@ -222,19 +216,18 @@ private void handleClusterAlias(String clusterAlias, AutoFollowMetadata.AutoFoll .build()); return newState.build(); }; - updateAutoMetadata(clusterStateUpdateFunction, updateError -> { + updateAutoFollowMetadata(clusterStateUpdateFunction, updateError -> { if (updateError != null) { LOGGER.error("Failed to mark leader index [" + leaderIndexName + "] as auto followed", updateError); } else { LOGGER.debug("Successfully marked leader index [{}] as auto followed", leaderIndexName); } - if (numRequests.incrementAndGet() == leaderIndicesToFollow.size()) { + if (countDown.countDown()) { finalise(updateError); } }); }; - LOGGER.info("Auto following leader index [{}] as follow index [{}]", leaderIndexName, followIndexName); - createAndFollowApiCall(followRequest, handler); + createAndFollow(followRequest, handler); } } } @@ -244,18 +237,41 @@ private void finalise(Exception failure) { errorHolder.get().addSuppressed(failure); } - if (executedRequests.incrementAndGet() == autoFollowMetadata.getPatterns().size()) { + if (countDown.countDown()) { handler.accept(errorHolder.get()); } } + static List getLeaderIndicesToFollow(AutoFollowPattern autoFollowPattern, + ClusterState leaderClusterState, + List followedIndexUUIDs) { + List leaderIndicesToFollow = new ArrayList<>(); + String[] patterns = autoFollowPattern.getLeaderIndexPatterns().toArray(new String[0]); + for (IndexMetaData indexMetaData : leaderClusterState.getMetaData()) { + if (Regex.simpleMatch(patterns, indexMetaData.getIndex().getName())) { + if (followedIndexUUIDs.contains(indexMetaData.getIndex().getUUID()) == false) { + leaderIndicesToFollow.add(indexMetaData.getIndex()); + } + } + } + return leaderIndicesToFollow; + } + + static String getFollowerIndexName(AutoFollowPattern autoFollowPattern, String leaderIndexName) { + if (autoFollowPattern.getFollowIndexPattern() != null) { + return autoFollowPattern.getFollowIndexPattern().replace("{{leader_index}}", leaderIndexName); + } else { + return leaderIndexName; + } + } + // abstract methods to make unit testing possible: - abstract void clusterStateApiCall(Client remoteClient, BiConsumer handler); + abstract void getLeaderClusterState(Client leaderClient, BiConsumer handler); - abstract void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler); + abstract void createAndFollow(FollowIndexAction.Request followRequest, Consumer handler); - abstract void updateAutoMetadata(Function updateFunction, Consumer handler); + abstract void updateAutoFollowMetadata(Function updateFunction, Consumer handler); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java index f6ab3914d47b3..24d4d2574c9c5 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java @@ -89,7 +89,7 @@ static ClusterState innerDelete(DeleteAutoFollowPatternAction.Request request, C final Map configurationsCopy = new HashMap<>(configurations); final Map> followedLeaderIndexUUIDSCopy = - new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); + new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); for (String key : toRemove) { configurationsCopy.remove(key); followedLeaderIndexUUIDSCopy.remove(key); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java index 77ab019b554e9..bff2809f321fc 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java @@ -96,7 +96,7 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, Cluster Map configurations; if (currentAutoFollowMetadata != null) { configurations = new HashMap<>(currentAutoFollowMetadata.getPatterns()); - followedLeaderIndices = new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDS()); + followedLeaderIndices = new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); } else { configurations = new HashMap<>(); followedLeaderIndices = new HashMap<>(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java index d27d92cea2821..cb1029f24fccc 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java @@ -11,12 +11,16 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.ccr.action.AutoFollowCoordinator.AutoFollower; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -38,7 +42,7 @@ public void testAutoFollower() { Client client = mock(Client.class); when(client.getRemoteClusterClient(anyString())).thenReturn(client); - ClusterState remoteState = ClusterState.builder(new ClusterName("remote")) + ClusterState leaderState = ClusterState.builder(new ClusterName("remote")) .metaData(MetaData.builder().put(IndexMetaData.builder("logs-20190101") .settings(settings(Version.CURRENT)) .numberOfShards(1) @@ -62,25 +66,25 @@ public void testAutoFollower() { invoked[0] = true; assertThat(e, nullValue()); }; - AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { @Override - void clusterStateApiCall(Client remoteClient, BiConsumer handler) { - handler.accept(remoteState, null); + void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + handler.accept(leaderState, null); } @Override - void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Consumer handler) { assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); handler.accept(null); } @Override - void updateAutoMetadata(Function updateFunction, Consumer handler) { + void updateAutoFollowMetadata(Function updateFunction, Consumer handler) { ClusterState resultCs = updateFunction.apply(currentState); AutoFollowMetadata result = resultCs.metaData().custom(AutoFollowMetadata.TYPE); - assertThat(result.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); - assertThat(result.getFollowedLeaderIndexUUIDS().get("remote").size(), equalTo(1)); + assertThat(result.getFollowedLeaderIndexUUIDs().size(), equalTo(1)); + assertThat(result.getFollowedLeaderIndexUUIDs().get("remote").size(), equalTo(1)); handler.accept(null); } }; @@ -106,19 +110,19 @@ public void testAutoFollowerClusterStateApiFailure() { invoked[0] = true; assertThat(e, sameInstance(failure)); }; - AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { @Override - void clusterStateApiCall(Client remoteClient, BiConsumer handler) { + void getLeaderClusterState(Client leaderClient, BiConsumer handler) { handler.accept(null, failure); } @Override - void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Consumer handler) { fail("should not get here"); } @Override - void updateAutoMetadata(Function updateFunction, Consumer handler) { + void updateAutoFollowMetadata(Function updateFunction, Consumer handler) { fail("should not get here"); } }; @@ -130,7 +134,7 @@ public void testAutoFollowerUpdateClusterStateFailure() { Client client = mock(Client.class); when(client.getRemoteClusterClient(anyString())).thenReturn(client); - ClusterState remoteState = ClusterState.builder(new ClusterName("remote")) + ClusterState leaderState = ClusterState.builder(new ClusterName("remote")) .metaData(MetaData.builder().put(IndexMetaData.builder("logs-20190101") .settings(settings(Version.CURRENT)) .numberOfShards(1) @@ -151,21 +155,21 @@ public void testAutoFollowerUpdateClusterStateFailure() { invoked[0] = true; assertThat(e, sameInstance(failure)); }; - AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { @Override - void clusterStateApiCall(Client remoteClient, BiConsumer handler) { - handler.accept(remoteState, null); + void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + handler.accept(leaderState, null); } @Override - void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Consumer handler) { assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); handler.accept(null); } @Override - void updateAutoMetadata(Function updateFunction, Consumer handler) { + void updateAutoFollowMetadata(Function updateFunction, Consumer handler) { handler.accept(failure); } }; @@ -177,7 +181,7 @@ public void testAutoFollowerCreateAndFollowApiCallFailure() { Client client = mock(Client.class); when(client.getRemoteClusterClient(anyString())).thenReturn(client); - ClusterState remoteState = ClusterState.builder(new ClusterName("remote")) + ClusterState leaderState = ClusterState.builder(new ClusterName("remote")) .metaData(MetaData.builder().put(IndexMetaData.builder("logs-20190101") .settings(settings(Version.CURRENT)) .numberOfShards(1) @@ -198,21 +202,21 @@ public void testAutoFollowerCreateAndFollowApiCallFailure() { invoked[0] = true; assertThat(e, sameInstance(failure)); }; - AutoFollowCoordinator.AutoFollower autoFollower = new AutoFollowCoordinator.AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { @Override - void clusterStateApiCall(Client remoteClient, BiConsumer handler) { - handler.accept(remoteState, null); + void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + handler.accept(leaderState, null); } @Override - void createAndFollowApiCall(FollowIndexAction.Request followRequest, Consumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Consumer handler) { assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); handler.accept(failure); } @Override - void updateAutoMetadata(Function updateFunction, Consumer handler) { + void updateAutoFollowMetadata(Function updateFunction, Consumer handler) { fail("should not get here"); } }; @@ -220,4 +224,60 @@ void updateAutoMetadata(Function updateFunction, Con assertThat(invoked[0], is(true)); } + public void testGetLeaderIndicesToFollow() { + AutoFollowPattern autoFollowPattern = + new AutoFollowPattern(Collections.singletonList("metrics-*"), null, null, null, null, null, null, null, null); + + MetaData.Builder imdBuilder = MetaData.builder(); + for (int i = 0; i < 5; i++) { + Settings.Builder builder = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_INDEX_UUID, "metrics-" + i); + imdBuilder.put(IndexMetaData.builder("metrics-" + i) + .settings(builder) + .numberOfShards(1) + .numberOfReplicas(0)); + } + imdBuilder.put(IndexMetaData.builder("logs-0") + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0)); + + ClusterState clusterState = ClusterState.builder(new ClusterName("remote")) + .metaData(imdBuilder) + .build(); + + List result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, clusterState, Collections.emptyList()); + result.sort(Comparator.comparing(Index::getName)); + assertThat(result.size(), equalTo(5)); + assertThat(result.get(0).getName(), equalTo("metrics-0")); + assertThat(result.get(1).getName(), equalTo("metrics-1")); + assertThat(result.get(2).getName(), equalTo("metrics-2")); + assertThat(result.get(3).getName(), equalTo("metrics-3")); + assertThat(result.get(4).getName(), equalTo("metrics-4")); + + List followedIndexUUIDs = Collections.singletonList(clusterState.metaData().index("metrics-2").getIndexUUID()); + result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, clusterState, followedIndexUUIDs); + result.sort(Comparator.comparing(Index::getName)); + assertThat(result.size(), equalTo(4)); + assertThat(result.get(0).getName(), equalTo("metrics-0")); + assertThat(result.get(1).getName(), equalTo("metrics-1")); + assertThat(result.get(2).getName(), equalTo("metrics-3")); + assertThat(result.get(3).getName(), equalTo("metrics-4")); + } + + public void testGetFollowerIndexName() { + AutoFollowPattern autoFollowPattern = new AutoFollowPattern(Collections.singletonList("metrics-*"), null, null, + null, null, null, null, null, null); + assertThat(AutoFollower.getFollowerIndexName(autoFollowPattern, "metrics-0"), equalTo("metrics-0")); + + autoFollowPattern = new AutoFollowPattern(Collections.singletonList("metrics-*"), "eu-metrics-0", null, null, + null, null, null, null, null); + assertThat(AutoFollower.getFollowerIndexName(autoFollowPattern, "metrics-0"), equalTo("eu-metrics-0")); + + autoFollowPattern = new AutoFollowPattern(Collections.singletonList("metrics-*"), "eu-{{leader_index}}", null, + null, null, null, null, null, null); + assertThat(AutoFollower.getFollowerIndexName(autoFollowPattern, "metrics-0"), equalTo("eu-metrics-0")); + } + } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java index 390b7a460c278..8876d4b24c9b0 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java @@ -58,8 +58,8 @@ public void testInnerDelete() { .custom(AutoFollowMetadata.TYPE); assertThat(result.getPatterns().size(), equalTo(1)); assertThat(result.getPatterns().get("asia_cluster"), notNullValue()); - assertThat(result.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); - assertThat(result.getFollowedLeaderIndexUUIDS().get("asia_cluster"), notNullValue()); + assertThat(result.getFollowedLeaderIndexUUIDs().size(), equalTo(1)); + assertThat(result.getFollowedLeaderIndexUUIDs().get("asia_cluster"), notNullValue()); } public void testInnerDeleteDoesNotExist() { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java index 0f298e75aa9be..0ab03918f1a4e 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java @@ -44,8 +44,8 @@ public void testInnerPut() { assertThat(autoFollowMetadata.getPatterns().size(), equalTo(1)); assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().size(), equalTo(1)); assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(0), equalTo("logs-*")); - assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); - assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().get("eu_cluster").size(), equalTo(0)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDs().size(), equalTo(1)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDs().get("eu_cluster").size(), equalTo(0)); } public void testInnerPut_existingLeaderIndices() { @@ -83,8 +83,8 @@ public void testInnerPut_existingLeaderIndices() { assertThat(autoFollowMetadata.getPatterns().size(), equalTo(1)); assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().size(), equalTo(1)); assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(0), equalTo("logs-*")); - assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); - assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().get("eu_cluster").size(), equalTo(numMatchingLeaderIndices)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDs().size(), equalTo(1)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDs().get("eu_cluster").size(), equalTo(numMatchingLeaderIndices)); } public void testInnerPut_existingLeaderIndicesAndAutoFollowMetadata() { @@ -126,8 +126,8 @@ public void testInnerPut_existingLeaderIndicesAndAutoFollowMetadata() { assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().size(), equalTo(2)); assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(0), equalTo("logs-*")); assertThat(autoFollowMetadata.getPatterns().get("eu_cluster").getLeaderIndexPatterns().get(1), equalTo("transactions-*")); - assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().size(), equalTo(1)); - assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDS().get("eu_cluster").size(), equalTo(numLeaderIndices + 1)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDs().size(), equalTo(1)); + assertThat(autoFollowMetadata.getFollowedLeaderIndexUUIDs().get("eu_cluster").size(), equalTo(numLeaderIndices + 1)); } } 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 09a26bf896606..eaf05f88d9659 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 @@ -80,24 +80,24 @@ public static AutoFollowMetadata fromXContent(XContentParser parser) throws IOEx } private final Map patterns; - private final Map> followedLeaderIndexUUIDS; + private final Map> followedLeaderIndexUUIDs; - public AutoFollowMetadata(Map patterns, Map> followedLeaderIndexUUIDS) { + public AutoFollowMetadata(Map patterns, Map> followedLeaderIndexUUIDs) { this.patterns = patterns; - this.followedLeaderIndexUUIDS = followedLeaderIndexUUIDS; + this.followedLeaderIndexUUIDs = followedLeaderIndexUUIDs; } public AutoFollowMetadata(StreamInput in) throws IOException { patterns = in.readMap(StreamInput::readString, AutoFollowPattern::new); - followedLeaderIndexUUIDS = in.readMapOfLists(StreamInput::readString, StreamInput::readString); + followedLeaderIndexUUIDs = in.readMapOfLists(StreamInput::readString, StreamInput::readString); } public Map getPatterns() { return patterns; } - public Map> getFollowedLeaderIndexUUIDS() { - return followedLeaderIndexUUIDS; + public Map> getFollowedLeaderIndexUUIDs() { + return followedLeaderIndexUUIDs; } @Override @@ -120,7 +120,7 @@ public Version getMinimalSupportedVersion() { @Override public void writeTo(StreamOutput out) throws IOException { out.writeMap(patterns, StreamOutput::writeString, (out1, value) -> value.writeTo(out1)); - out.writeMapOfLists(followedLeaderIndexUUIDS, StreamOutput::writeString, StreamOutput::writeString); + out.writeMapOfLists(followedLeaderIndexUUIDs, StreamOutput::writeString, StreamOutput::writeString); } @Override @@ -134,7 +134,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); builder.startObject(FOLLOWED_LEADER_INDICES_FIELD.getPreferredName()); - for (Map.Entry> entry : followedLeaderIndexUUIDS.entrySet()) { + for (Map.Entry> entry : followedLeaderIndexUUIDs.entrySet()) { builder.field(entry.getKey(), entry.getValue()); } builder.endObject(); From 3baa21095d7e870fe1bc13795221d6402a08ef30 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 28 Aug 2018 12:10:53 +0700 Subject: [PATCH 06/19] make handleClusterAlias() method more readable --- .../ccr/action/AutoFollowCoordinator.java | 78 +++++++++++-------- .../action/AutoFollowCoordinatorTests.java | 14 ++-- 2 files changed, 54 insertions(+), 38 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index e567a3cc30e45..5decf4bf1d6dd 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -99,9 +99,11 @@ void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, + Runnable successHandler, + Consumer failureHandler) { client.execute(CreateAndFollowIndexAction.INSTANCE, new CreateAndFollowIndexAction.Request(followRequest), - ActionListener.wrap(r -> handler.accept(null), handler)); + ActionListener.wrap(r -> successHandler.run(), failureHandler)); } void updateAutoFollowMetadata(Function updateFunction, Consumer handler) { @@ -191,32 +193,15 @@ private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollo autoFollowPattern.getMaxWriteBufferSize(), autoFollowPattern.getRetryTimeout(), autoFollowPattern.getIdleShardRetryDelay()); - // This runs on the elected master node, so we can update cluster state here: - Consumer handler = followError -> { - if (followError != null) { - LOGGER.warn("Failed to auto follow leader index [" + leaderIndexName + "]", followError); - if (countDown.countDown()) { - finalise(followError); - } - return; - } + // Execute if the create and follow api call succeeds: + Runnable successHandler = () -> { LOGGER.info("Auto followed leader index [{}] as follow index [{}]", leaderIndexName, followIndexName); - Function clusterStateUpdateFunction = currentState -> { - AutoFollowMetadata currentAutoFollowMetadata = currentState.metaData().custom(AutoFollowMetadata.TYPE); - - Map> newFollowedIndexUUIDS = - new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); - newFollowedIndexUUIDS.get(clusterAlias).add(indexToFollow.getUUID()); - - ClusterState.Builder newState = ClusterState.builder(currentState); - AutoFollowMetadata newAutoFollowMetadata = - new AutoFollowMetadata(currentAutoFollowMetadata.getPatterns(), newFollowedIndexUUIDS); - newState.metaData(MetaData.builder(currentState.getMetaData()) - .putCustom(AutoFollowMetadata.TYPE, newAutoFollowMetadata) - .build()); - return newState.build(); - }; - updateAutoFollowMetadata(clusterStateUpdateFunction, updateError -> { + + // This function updates the auto follow metadata in the cluster to record that the leader index has been followed: + // (so that we do not try to follow it in subsequent auto follow runs) + Function function = recordLeaderIndexAsFollowFunction(clusterAlias, indexToFollow); + // The coordinator always runs on the elected master node, so we can update cluster state here: + updateAutoFollowMetadata(function, updateError -> { if (updateError != null) { LOGGER.error("Failed to mark leader index [" + leaderIndexName + "] as auto followed", updateError); } else { @@ -227,7 +212,15 @@ private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollo } }); }; - createAndFollow(followRequest, handler); + // Execute if the create and follow apu call fails: + Consumer failureHandler = followError -> { + assert followError != null; + LOGGER.warn("Failed to auto follow leader index [" + leaderIndexName + "]", followError); + if (countDown.countDown()) { + finalise(followError); + } + }; + createAndFollow(followRequest, successHandler, failureHandler); } } } @@ -265,13 +258,36 @@ static String getFollowerIndexName(AutoFollowPattern autoFollowPattern, String l } } + static Function recordLeaderIndexAsFollowFunction(String clusterAlias, Index indexToFollow) { + return currentState -> { + AutoFollowMetadata currentAutoFollowMetadata = currentState.metaData().custom(AutoFollowMetadata.TYPE); + + Map> newFollowedIndexUUIDS = + new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); + newFollowedIndexUUIDS.get(clusterAlias).add(indexToFollow.getUUID()); + + ClusterState.Builder newState = ClusterState.builder(currentState); + AutoFollowMetadata newAutoFollowMetadata = + new AutoFollowMetadata(currentAutoFollowMetadata.getPatterns(), newFollowedIndexUUIDS); + newState.metaData(MetaData.builder(currentState.getMetaData()) + .putCustom(AutoFollowMetadata.TYPE, newAutoFollowMetadata) + .build()); + return newState.build(); + }; + } + // abstract methods to make unit testing possible: - abstract void getLeaderClusterState(Client leaderClient, BiConsumer handler); + abstract void getLeaderClusterState(Client leaderClient, + BiConsumer handler); - abstract void createAndFollow(FollowIndexAction.Request followRequest, Consumer handler); + abstract void createAndFollow(FollowIndexAction.Request followRequest, + Runnable successHandler, + Consumer failureHandler); - abstract void updateAutoFollowMetadata(Function updateFunction, Consumer handler); + abstract void updateAutoFollowMetadata(Function updateFunction, + Consumer handler); } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java index cb1029f24fccc..2374ae3ef563a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java @@ -73,10 +73,10 @@ void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Runnable successHandler, Consumer failureHandler) { assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); - handler.accept(null); + successHandler.run(); } @Override @@ -117,7 +117,7 @@ void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Runnable successHandler, Consumer failureHandler) { fail("should not get here"); } @@ -162,10 +162,10 @@ void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Runnable successHandler, Consumer failureHandler) { assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); - handler.accept(null); + successHandler.run(); } @Override @@ -209,10 +209,10 @@ void getLeaderClusterState(Client leaderClient, BiConsumer handler) { + void createAndFollow(FollowIndexAction.Request followRequest, Runnable successHandler, Consumer failureHandler) { assertThat(followRequest.getLeaderIndex(), equalTo("remote:logs-20190101")); assertThat(followRequest.getFollowerIndex(), equalTo("logs-20190101")); - handler.accept(failure); + failureHandler.accept(failure); } @Override From 812ebb7d6fda6823f3629b2d1bdff447420f71e3 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 28 Aug 2018 16:46:57 +0700 Subject: [PATCH 07/19] since the test does not explicitly execute create_and_follow we need to make sure that al least one shard is available to avoid 'all shards failed' error in search api. so instead of checking if index exists in cluster state a wait for yellow check is done. --- .../org/elasticsearch/xpack/ccr/FollowIndexIT.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 7c207cf6aa545..607d4312611c6 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -101,8 +101,7 @@ public void testAutoFollowPatterns() throws Exception { } assertBusy(() -> { - Request indexExists = new Request("HEAD", "/logs-20190101"); - assertOK(client().performRequest(indexExists)); + ensureYellow("logs-20190101"); verifyDocuments("logs-20190101", 5); }); } @@ -164,6 +163,15 @@ private static Map toMap(String response) { return XContentHelper.convertToMap(JsonXContent.jsonXContent, response, false); } + private static void ensureYellow(String index) throws IOException { + Request request = new Request("GET", "/_cluster/health/" + index); + request.addParameter("wait_for_status", "yellow"); + request.addParameter("wait_for_no_relocating_shards", "true"); + request.addParameter("timeout", "70s"); + request.addParameter("level", "shards"); + client().performRequest(request); + } + private RestClient buildLeaderClient() throws IOException { assert runningAgainstLeaderCluster == false; String leaderUrl = System.getProperty("tests.leader_host"); From 5fcfb3eb4d27aeff4c59e411cb9eb2043efca3b3 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 29 Aug 2018 10:25:53 +0700 Subject: [PATCH 08/19] added missing @Override --- .../elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index 5decf4bf1d6dd..f490e2047ccf0 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -87,6 +87,7 @@ private void doAutoFollow() { }; AutoFollower operation = new AutoFollower(client, handler, autoFollowMetadata) { + @Override void getLeaderClusterState(Client leaderClient, BiConsumer handler) { ClusterStateRequest request = new ClusterStateRequest(); request.clear(); @@ -99,6 +100,7 @@ void getLeaderClusterState(Client leaderClient, BiConsumer failureHandler) { @@ -106,6 +108,7 @@ void createAndFollow(FollowIndexAction.Request followRequest, ActionListener.wrap(r -> successHandler.run(), failureHandler)); } + @Override void updateAutoFollowMetadata(Function updateFunction, Consumer handler) { clusterService.submitStateUpdateTask("update_auto_follow_metadata", new ClusterStateUpdateTask() { From d2b06432a69b55c9d10da66464d6a8debdb3196f Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 29 Aug 2018 10:28:39 +0700 Subject: [PATCH 09/19] rename --- .../xpack/ccr/action/AutoFollowCoordinator.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index f490e2047ccf0..bb138e5018a30 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -148,14 +148,14 @@ abstract static class AutoFollower { private final Consumer handler; private final AutoFollowMetadata autoFollowMetadata; - private final CountDown countDown; + private final CountDown autoFollowPatternsCountDown; private final AtomicReference errorHolder = new AtomicReference<>(); AutoFollower(Client client, Consumer handler, AutoFollowMetadata autoFollowMetadata) { this.client = client; this.handler = handler; this.autoFollowMetadata = autoFollowMetadata; - this.countDown = new CountDown(autoFollowMetadata.getPatterns().size()); + this.autoFollowPatternsCountDown = new CountDown(autoFollowMetadata.getPatterns().size()); } void autoFollowIndices() { @@ -182,7 +182,7 @@ private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollo if (leaderIndicesToFollow.isEmpty()) { finalise(null); } else { - CountDown countDown = new CountDown(leaderIndicesToFollow.size()); + CountDown leaderIndicesCountDown = new CountDown(leaderIndicesToFollow.size()); for (Index indexToFollow : leaderIndicesToFollow) { final String leaderIndexName = indexToFollow.getName(); final String followIndexName = getFollowerIndexName(autoFollowPattern, leaderIndexName); @@ -210,7 +210,7 @@ private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollo } else { LOGGER.debug("Successfully marked leader index [{}] as auto followed", leaderIndexName); } - if (countDown.countDown()) { + if (leaderIndicesCountDown.countDown()) { finalise(updateError); } }); @@ -219,7 +219,7 @@ private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollo Consumer failureHandler = followError -> { assert followError != null; LOGGER.warn("Failed to auto follow leader index [" + leaderIndexName + "]", followError); - if (countDown.countDown()) { + if (leaderIndicesCountDown.countDown()) { finalise(followError); } }; @@ -233,7 +233,7 @@ private void finalise(Exception failure) { errorHolder.get().addSuppressed(failure); } - if (countDown.countDown()) { + if (autoFollowPatternsCountDown.countDown()) { handler.accept(errorHolder.get()); } } From 1634592c29bf6025494da77d175d556c87ef4163 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 29 Aug 2018 10:32:52 +0700 Subject: [PATCH 10/19] Keep track of all updateError instances --- .../ccr/action/AutoFollowCoordinator.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index bb138e5018a30..96e495b387a19 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -149,7 +149,7 @@ abstract static class AutoFollower { private final AutoFollowMetadata autoFollowMetadata; private final CountDown autoFollowPatternsCountDown; - private final AtomicReference errorHolder = new AtomicReference<>(); + private final AtomicReference autoFollowPatternsErrorHolder = new AtomicReference<>(); AutoFollower(Client client, Consumer handler, AutoFollowMetadata autoFollowMetadata) { this.client = client; @@ -178,11 +178,12 @@ void autoFollowIndices() { private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollowPattern, List followedIndexUUIDs, ClusterState leaderClusterState) { - List leaderIndicesToFollow = getLeaderIndicesToFollow(autoFollowPattern, leaderClusterState, followedIndexUUIDs); + final List leaderIndicesToFollow = getLeaderIndicesToFollow(autoFollowPattern, leaderClusterState, followedIndexUUIDs); if (leaderIndicesToFollow.isEmpty()) { finalise(null); } else { - CountDown leaderIndicesCountDown = new CountDown(leaderIndicesToFollow.size()); + final CountDown leaderIndicesCountDown = new CountDown(leaderIndicesToFollow.size()); + final AtomicReference leaderIndicesErrorHolder = new AtomicReference<>(); for (Index indexToFollow : leaderIndicesToFollow) { final String leaderIndexName = indexToFollow.getName(); final String followIndexName = getFollowerIndexName(autoFollowPattern, leaderIndexName); @@ -207,11 +208,14 @@ private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollo updateAutoFollowMetadata(function, updateError -> { if (updateError != null) { LOGGER.error("Failed to mark leader index [" + leaderIndexName + "] as auto followed", updateError); + if (leaderIndicesErrorHolder.compareAndSet(null, updateError) == false) { + leaderIndicesErrorHolder.get().addSuppressed(updateError); + } } else { LOGGER.debug("Successfully marked leader index [{}] as auto followed", leaderIndexName); } if (leaderIndicesCountDown.countDown()) { - finalise(updateError); + finalise(leaderIndicesErrorHolder.get()); } }); }; @@ -229,12 +233,12 @@ private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollo } private void finalise(Exception failure) { - if (errorHolder.compareAndSet(null, failure) == false) { - errorHolder.get().addSuppressed(failure); + if (autoFollowPatternsErrorHolder.compareAndSet(null, failure) == false) { + autoFollowPatternsErrorHolder.get().addSuppressed(failure); } if (autoFollowPatternsCountDown.countDown()) { - handler.accept(errorHolder.get()); + handler.accept(autoFollowPatternsErrorHolder.get()); } } From 5edc457ea438de266c9095fb14feeafc0de987cd Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 29 Aug 2018 10:46:19 +0700 Subject: [PATCH 11/19] Added a TODO and passed down follower cluster state so that in the future we can check for indices that are already being followed but we failed to update AutoMetaData#followedLeaderIndexUUIDs --- .../ccr/action/AutoFollowCoordinator.java | 28 ++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index 96e495b387a19..abd54cf7f2988 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -67,8 +67,8 @@ private void doAutoFollow() { if (localNodeMaster == false) { return; } - ClusterState localClusterState = clusterService.state(); - AutoFollowMetadata autoFollowMetadata = localClusterState.getMetaData().custom(AutoFollowMetadata.TYPE); + ClusterState followerClusterState = clusterService.state(); + AutoFollowMetadata autoFollowMetadata = followerClusterState.getMetaData().custom(AutoFollowMetadata.TYPE); if (autoFollowMetadata == null) { threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); return; @@ -85,7 +85,7 @@ private void doAutoFollow() { } threadPool.schedule(pollInterval, ThreadPool.Names.SAME, this::doAutoFollow); }; - AutoFollower operation = new AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower operation = new AutoFollower(client, handler, followerClusterState) { @Override void getLeaderClusterState(Client leaderClient, BiConsumer handler) { @@ -146,15 +146,17 @@ abstract static class AutoFollower { private final Client client; private final Consumer handler; + private final ClusterState followerClusterState; private final AutoFollowMetadata autoFollowMetadata; private final CountDown autoFollowPatternsCountDown; private final AtomicReference autoFollowPatternsErrorHolder = new AtomicReference<>(); - AutoFollower(Client client, Consumer handler, AutoFollowMetadata autoFollowMetadata) { + AutoFollower(Client client, Consumer handler, ClusterState followerClusterState) { this.client = client; this.handler = handler; - this.autoFollowMetadata = autoFollowMetadata; + this.followerClusterState = followerClusterState; + this.autoFollowMetadata = followerClusterState.getMetaData().custom(AutoFollowMetadata.TYPE); this.autoFollowPatternsCountDown = new CountDown(autoFollowMetadata.getPatterns().size()); } @@ -178,7 +180,8 @@ void autoFollowIndices() { private void handleClusterAlias(String clusterAlias, AutoFollowPattern autoFollowPattern, List followedIndexUUIDs, ClusterState leaderClusterState) { - final List leaderIndicesToFollow = getLeaderIndicesToFollow(autoFollowPattern, leaderClusterState, followedIndexUUIDs); + final List leaderIndicesToFollow = + getLeaderIndicesToFollow(autoFollowPattern, leaderClusterState, followerClusterState, followedIndexUUIDs); if (leaderIndicesToFollow.isEmpty()) { finalise(null); } else { @@ -244,13 +247,18 @@ private void finalise(Exception failure) { static List getLeaderIndicesToFollow(AutoFollowPattern autoFollowPattern, ClusterState leaderClusterState, + ClusterState followerClusterState, List followedIndexUUIDs) { List leaderIndicesToFollow = new ArrayList<>(); String[] patterns = autoFollowPattern.getLeaderIndexPatterns().toArray(new String[0]); - for (IndexMetaData indexMetaData : leaderClusterState.getMetaData()) { - if (Regex.simpleMatch(patterns, indexMetaData.getIndex().getName())) { - if (followedIndexUUIDs.contains(indexMetaData.getIndex().getUUID()) == false) { - leaderIndicesToFollow.add(indexMetaData.getIndex()); + for (IndexMetaData leaderIndexMetaData : leaderClusterState.getMetaData()) { + if (Regex.simpleMatch(patterns, leaderIndexMetaData.getIndex().getName())) { + if (followedIndexUUIDs.contains(leaderIndexMetaData.getIndex().getUUID()) == false) { + // TODO: iterate over the indices in the followerClusterState and check whether a IndexMetaData + // has a leader index uuid custom metadata entry that matches with uuid of leaderIndexMetaData variable + // If so then handle it differently: not follow it, but just add an entry to + // AutoFollowMetadata#followedLeaderIndexUUIDs + leaderIndicesToFollow.add(leaderIndexMetaData.getIndex()); } } } From 5045a625f828c66f7c13d5ebe971ac063b659eb0 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 29 Aug 2018 10:53:07 +0700 Subject: [PATCH 12/19] pushed too fast --- .../action/AutoFollowCoordinatorTests.java | 29 ++++++++++++++----- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java index 2374ae3ef563a..dd1376a4d7a73 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java @@ -66,7 +66,7 @@ public void testAutoFollower() { invoked[0] = true; assertThat(e, nullValue()); }; - AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, currentState) { @Override void getLeaderClusterState(Client leaderClient, BiConsumer handler) { handler.accept(leaderState, null); @@ -103,6 +103,9 @@ public void testAutoFollowerClusterStateApiFailure() { Map> followedLeaderIndexUUIDS = new HashMap<>(); followedLeaderIndexUUIDS.put("remote", new ArrayList<>()); AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata(patterns, followedLeaderIndexUUIDS); + ClusterState followerState = ClusterState.builder(new ClusterName("remote")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, autoFollowMetadata)) + .build(); Exception failure = new RuntimeException("failure"); boolean[] invoked = new boolean[]{false}; @@ -110,7 +113,7 @@ public void testAutoFollowerClusterStateApiFailure() { invoked[0] = true; assertThat(e, sameInstance(failure)); }; - AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, followerState) { @Override void getLeaderClusterState(Client leaderClient, BiConsumer handler) { handler.accept(null, failure); @@ -148,6 +151,9 @@ public void testAutoFollowerUpdateClusterStateFailure() { Map> followedLeaderIndexUUIDS = new HashMap<>(); followedLeaderIndexUUIDS.put("remote", new ArrayList<>()); AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata(patterns, followedLeaderIndexUUIDS); + ClusterState followerState = ClusterState.builder(new ClusterName("remote")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, autoFollowMetadata)) + .build(); Exception failure = new RuntimeException("failure"); boolean[] invoked = new boolean[]{false}; @@ -155,7 +161,7 @@ public void testAutoFollowerUpdateClusterStateFailure() { invoked[0] = true; assertThat(e, sameInstance(failure)); }; - AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, followerState) { @Override void getLeaderClusterState(Client leaderClient, BiConsumer handler) { handler.accept(leaderState, null); @@ -195,6 +201,9 @@ public void testAutoFollowerCreateAndFollowApiCallFailure() { Map> followedLeaderIndexUUIDS = new HashMap<>(); followedLeaderIndexUUIDS.put("remote", new ArrayList<>()); AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata(patterns, followedLeaderIndexUUIDS); + ClusterState followerState = ClusterState.builder(new ClusterName("remote")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, autoFollowMetadata)) + .build(); Exception failure = new RuntimeException("failure"); boolean[] invoked = new boolean[]{false}; @@ -202,7 +211,7 @@ public void testAutoFollowerCreateAndFollowApiCallFailure() { invoked[0] = true; assertThat(e, sameInstance(failure)); }; - AutoFollower autoFollower = new AutoFollower(client, handler, autoFollowMetadata) { + AutoFollower autoFollower = new AutoFollower(client, handler, followerState) { @Override void getLeaderClusterState(Client leaderClient, BiConsumer handler) { handler.accept(leaderState, null); @@ -227,6 +236,10 @@ void updateAutoFollowMetadata(Function updateFunctio public void testGetLeaderIndicesToFollow() { AutoFollowPattern autoFollowPattern = new AutoFollowPattern(Collections.singletonList("metrics-*"), null, null, null, null, null, null, null, null); + ClusterState followerState = ClusterState.builder(new ClusterName("remote")) + .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, + new AutoFollowMetadata(Collections.singletonMap("remote", autoFollowPattern), Collections.emptyMap()))) + .build(); MetaData.Builder imdBuilder = MetaData.builder(); for (int i = 0; i < 5; i++) { @@ -243,11 +256,11 @@ public void testGetLeaderIndicesToFollow() { .numberOfShards(1) .numberOfReplicas(0)); - ClusterState clusterState = ClusterState.builder(new ClusterName("remote")) + ClusterState leaderState = ClusterState.builder(new ClusterName("remote")) .metaData(imdBuilder) .build(); - List result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, clusterState, Collections.emptyList()); + List result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, leaderState, followerState, Collections.emptyList()); result.sort(Comparator.comparing(Index::getName)); assertThat(result.size(), equalTo(5)); assertThat(result.get(0).getName(), equalTo("metrics-0")); @@ -256,8 +269,8 @@ public void testGetLeaderIndicesToFollow() { assertThat(result.get(3).getName(), equalTo("metrics-3")); assertThat(result.get(4).getName(), equalTo("metrics-4")); - List followedIndexUUIDs = Collections.singletonList(clusterState.metaData().index("metrics-2").getIndexUUID()); - result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, clusterState, followedIndexUUIDs); + List followedIndexUUIDs = Collections.singletonList(leaderState.metaData().index("metrics-2").getIndexUUID()); + result = AutoFollower.getLeaderIndicesToFollow(autoFollowPattern, leaderState, followerState, followedIndexUUIDs); result.sort(Comparator.comparing(Index::getName)); assertThat(result.size(), equalTo(4)); assertThat(result.get(0).getName(), equalTo("metrics-0")); From 91dedcd25a2e2ea2364923b34badc6371890fde9 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 30 Aug 2018 08:37:33 +0700 Subject: [PATCH 13/19] changed api path from /_ccr/_autofollow to /_ccr/_auto_follow --- .../test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java | 2 +- .../xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java | 4 ++-- .../xpack/ccr/rest/RestPutAutoFollowPatternAction.java | 4 ++-- .../rest-api-spec/api/ccr.delete_auto_follow_pattern.json | 4 ++-- .../rest-api-spec/api/ccr.put_auto_follow_pattern.json | 4 ++-- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 607d4312611c6..17a6db286f283 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -81,7 +81,7 @@ public void testFollowIndex() throws Exception { public void testAutoFollowPatterns() throws Exception { assumeFalse("Test should only run when both clusters are running", runningAgainstLeaderCluster); - Request request = new Request("PUT", "/_ccr/_autofollow/leader_cluster"); + Request request = new Request("PUT", "/_ccr/_auto_follow/leader_cluster"); request.setJsonEntity("{\"leader_index_patterns\": [\"logs-*\"]}"); assertOK(client().performRequest(request)); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java index 7070017d20940..9899e88f35aab 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java @@ -21,12 +21,12 @@ public class RestDeleteAutoFollowPatternAction extends BaseRestHandler { public RestDeleteAutoFollowPatternAction(Settings settings, RestController controller) { super(settings); - controller.registerHandler(RestRequest.Method.DELETE, "/_ccr/_autofollow/{remote_cluster_alias}", this); + controller.registerHandler(RestRequest.Method.DELETE, "/_ccr/_auto_follow/{remote_cluster_alias}", this); } @Override public String getName() { - return "ccr_delete_auto_folow_pattern_action"; + return "ccr_delete_auto_follow_pattern_action"; } @Override diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java index cb32117e3c854..93a455d8e4b4b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java @@ -22,12 +22,12 @@ public class RestPutAutoFollowPatternAction extends BaseRestHandler { public RestPutAutoFollowPatternAction(Settings settings, RestController controller) { super(settings); - controller.registerHandler(RestRequest.Method.PUT, "/_ccr/_autofollow/{remote_cluster_alias}", this); + controller.registerHandler(RestRequest.Method.PUT, "/_ccr/_auto_follow/{remote_cluster_alias}", this); } @Override public String getName() { - return "ccr_put_auto_folow_pattern_action"; + return "ccr_put_auto_follow_pattern_action"; } @Override diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json index 9b3e46d4bddce..d883e05153ec5 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json @@ -3,8 +3,8 @@ "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current", "methods": [ "DELETE" ], "url": { - "path": "/_ccr/_autofollow/{remote_cluster_alias}", - "paths": [ "/_ccr/_autofollow/{remote_cluster_alias}" ], + "path": "/_ccr/_auto_follow/{remote_cluster_alias}", + "paths": [ "/_ccr/_auto_follow/{remote_cluster_alias}" ], "parts": { "remote_cluster_alias": { "type": "string", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json index ee28109f53b96..9ff8acafe1466 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json @@ -3,8 +3,8 @@ "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current", "methods": [ "PUT" ], "url": { - "path": "/_ccr/_autofollow/{remote_cluster_alias}", - "paths": [ "/_ccr/_autofollow/{remote_cluster_alias}" ], + "path": "/_ccr/_auto_follow/{remote_cluster_alias}", + "paths": [ "/_ccr/_auto_follow/{remote_cluster_alias}" ], "parts": { "remote_cluster_alias": { "type": "string", From cf792d9376319178c59a949ecf00d3f5172f6d5a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 30 Aug 2018 08:49:19 +0700 Subject: [PATCH 14/19] iter --- .../action/PutAutoFollowPatternAction.java | 49 ++++++++++-------- .../TransportPutAutoFollowPatternAction.java | 51 ++++++++++--------- .../xpack/ccr/action/AutoFollowTests.java | 4 +- .../PutAutoFollowPatternRequestTests.java | 2 +- ...nsportPutAutoFollowPatternActionTests.java | 6 +-- 5 files changed, 62 insertions(+), 50 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java index 14079686c3ab5..a01fd8e3bc209 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternAction.java @@ -41,14 +41,14 @@ public AcknowledgedResponse newResponse() { public static class Request extends AcknowledgedRequest implements ToXContentObject { - static final ParseField REMOTE_CLUSTER_ALIAS_FIELD = new ParseField("remote_cluster_alias"); + static final ParseField LEADER_CLUSTER_ALIAS_FIELD = new ParseField("leader_cluster_alias"); static final ParseField LEADER_INDEX_PATTERNS_FIELD = new ParseField("leader_index_patterns"); static final ParseField FOLLOW_INDEX_NAME_PATTERN_FIELD = new ParseField("follow_index_name_pattern"); private static final ObjectParser PARSER = new ObjectParser<>("put_auto_follow_pattern_request", Request::new); static { - PARSER.declareString(Request::setRemoteClusterAlias, REMOTE_CLUSTER_ALIAS_FIELD); + PARSER.declareString(Request::setLeaderClusterAlias, LEADER_CLUSTER_ALIAS_FIELD); PARSER.declareStringArray(Request::setLeaderIndexPatterns, LEADER_INDEX_PATTERNS_FIELD); PARSER.declareString(Request::setFollowIndexNamePattern, FOLLOW_INDEX_NAME_PATTERN_FIELD); PARSER.declareInt(Request::setMaxBatchOperationCount, AutoFollowPattern.MAX_BATCH_OPERATION_COUNT); @@ -67,18 +67,18 @@ public static class Request extends AcknowledgedRequest implements ToXC public static Request fromXContent(XContentParser parser, String remoteClusterAlias) throws IOException { Request request = PARSER.parse(parser, null); if (remoteClusterAlias != null) { - if (request.remoteClusterAlias == null) { - request.remoteClusterAlias = remoteClusterAlias; + if (request.leaderClusterAlias == null) { + request.leaderClusterAlias = remoteClusterAlias; } else { - if (request.remoteClusterAlias.equals(remoteClusterAlias) == false) { - throw new IllegalArgumentException("provided remoteClusterAlias is not equal"); + if (request.leaderClusterAlias.equals(remoteClusterAlias) == false) { + throw new IllegalArgumentException("provided leaderClusterAlias is not equal"); } } } return request; } - private String remoteClusterAlias; + private String leaderClusterAlias; private List leaderIndexPatterns; private String followIndexNamePattern; @@ -93,8 +93,8 @@ public static Request fromXContent(XContentParser parser, String remoteClusterAl @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; - if (remoteClusterAlias == null) { - validationException = addValidationError("remoteClusterAlias is missing", validationException); + if (leaderClusterAlias == null) { + validationException = addValidationError("leaderClusterAlias is missing", validationException); } if (leaderIndexPatterns == null || leaderIndexPatterns.isEmpty()) { validationException = addValidationError("leaderIndexPatterns is missing", validationException); @@ -102,12 +102,12 @@ public ActionRequestValidationException validate() { return validationException; } - public String getRemoteClusterAlias() { - return remoteClusterAlias; + public String getLeaderClusterAlias() { + return leaderClusterAlias; } - public void setRemoteClusterAlias(String remoteClusterAlias) { - this.remoteClusterAlias = remoteClusterAlias; + public void setLeaderClusterAlias(String leaderClusterAlias) { + this.leaderClusterAlias = leaderClusterAlias; } public List getLeaderIndexPatterns() { @@ -185,7 +185,7 @@ public void setIdleShardRetryDelay(TimeValue idleShardRetryDelay) { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - remoteClusterAlias = in.readString(); + leaderClusterAlias = in.readString(); leaderIndexPatterns = in.readList(StreamInput::readString); followIndexNamePattern = in.readOptionalString(); maxBatchOperationCount = in.readOptionalVInt(); @@ -200,7 +200,7 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(remoteClusterAlias); + out.writeString(leaderClusterAlias); out.writeStringList(leaderIndexPatterns); out.writeOptionalString(followIndexNamePattern); out.writeOptionalVInt(maxBatchOperationCount); @@ -216,7 +216,7 @@ public void writeTo(StreamOutput out) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); { - builder.field(REMOTE_CLUSTER_ALIAS_FIELD.getPreferredName(), remoteClusterAlias); + builder.field(LEADER_CLUSTER_ALIAS_FIELD.getPreferredName(), leaderClusterAlias); builder.field(LEADER_INDEX_PATTERNS_FIELD.getPreferredName(), leaderIndexPatterns); if (followIndexNamePattern != null) { builder.field(FOLLOW_INDEX_NAME_PATTERN_FIELD.getPreferredName(), followIndexNamePattern); @@ -252,7 +252,7 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; - return Objects.equals(remoteClusterAlias, request.remoteClusterAlias) && + return Objects.equals(leaderClusterAlias, request.leaderClusterAlias) && Objects.equals(leaderIndexPatterns, request.leaderIndexPatterns) && Objects.equals(followIndexNamePattern, request.followIndexNamePattern) && Objects.equals(maxBatchOperationCount, request.maxBatchOperationCount) && @@ -266,9 +266,18 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(remoteClusterAlias, leaderIndexPatterns, followIndexNamePattern, maxBatchOperationCount, - maxConcurrentReadBatches, maxOperationSizeInBytes, maxConcurrentWriteBatches, maxWriteBufferSize, - retryTimeout, idleShardRetryDelay); + return Objects.hash( + leaderClusterAlias, + leaderIndexPatterns, + followIndexNamePattern, + maxBatchOperationCount, + maxConcurrentReadBatches, + maxOperationSizeInBytes, + maxConcurrentWriteBatches, + maxWriteBufferSize, + retryTimeout, + idleShardRetryDelay + ); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java index bff2809f321fc..bb1b08f4cee2c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java @@ -59,19 +59,20 @@ protected AcknowledgedResponse newResponse() { protected void masterOperation(PutAutoFollowPatternAction.Request request, ClusterState state, ActionListener listener) throws Exception { - final Client remoteClient; - if (request.getRemoteClusterAlias().equals("_local_")) { - remoteClient = client; + final Client leaderClient; + if (request.getLeaderClusterAlias().equals("_local_")) { + leaderClient = client; } else { - remoteClient = client.getRemoteClusterClient(request.getRemoteClusterAlias()); + leaderClient = client.getRemoteClusterClient(request.getLeaderClusterAlias()); } final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); clusterStateRequest.clear(); clusterStateRequest.metaData(true); - remoteClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(resp -> { - clusterService.submitStateUpdateTask("put_auto_follow_pattern-" + request.getRemoteClusterAlias(), + leaderClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { + final ClusterState leaderClusterState = clusterStateResponse.getState(); + clusterService.submitStateUpdateTask("put-auto-follow-pattern-" + request.getLeaderClusterAlias(), new AckedClusterStateUpdateTask(request, listener) { @Override @@ -81,50 +82,52 @@ protected AcknowledgedResponse newResponse(boolean acknowledged) { @Override public ClusterState execute(ClusterState currentState) throws Exception { - return innerPut(request, currentState, resp.getState()); + return innerPut(request, currentState, leaderClusterState); } }); }, listener::onFailure)); } - static ClusterState innerPut(PutAutoFollowPatternAction.Request request, ClusterState localState, ClusterState remoteClusterState) { + static ClusterState innerPut(PutAutoFollowPatternAction.Request request, + ClusterState localState, + ClusterState leaderClusterState) { // auto patterns are always overwritten // only already followed index uuids are updated AutoFollowMetadata currentAutoFollowMetadata = localState.metaData().custom(AutoFollowMetadata.TYPE); Map> followedLeaderIndices; - Map configurations; + Map patterns; if (currentAutoFollowMetadata != null) { - configurations = new HashMap<>(currentAutoFollowMetadata.getPatterns()); + patterns = new HashMap<>(currentAutoFollowMetadata.getPatterns()); followedLeaderIndices = new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); } else { - configurations = new HashMap<>(); + patterns = new HashMap<>(); followedLeaderIndices = new HashMap<>(); } - AutoFollowMetadata.AutoFollowPattern previousPattern = configurations.get(request.getRemoteClusterAlias()); - List followedIndexUUIDS = followedLeaderIndices.get(request.getRemoteClusterAlias()); - if (followedIndexUUIDS == null) { - followedIndexUUIDS = new ArrayList<>(); - followedLeaderIndices.put(request.getRemoteClusterAlias(), followedIndexUUIDS); + AutoFollowMetadata.AutoFollowPattern previousPattern = patterns.get(request.getLeaderClusterAlias()); + List followedIndexUUIDs = followedLeaderIndices.get(request.getLeaderClusterAlias()); + if (followedIndexUUIDs == null) { + followedIndexUUIDs = new ArrayList<>(); + followedLeaderIndices.put(request.getLeaderClusterAlias(), followedIndexUUIDs); } // Mark existing leader indices as already auto followed: if (previousPattern != null) { for (String newPattern : request.getLeaderIndexPatterns()) { if (previousPattern.getLeaderIndexPatterns().contains(newPattern) == false) { - for (IndexMetaData indexMetaData : remoteClusterState.getMetaData()) { + for (IndexMetaData indexMetaData : leaderClusterState.getMetaData()) { if (Regex.simpleMatch(newPattern, indexMetaData.getIndex().getName())) { - followedIndexUUIDS.add(indexMetaData.getIndexUUID()); + followedIndexUUIDs.add(indexMetaData.getIndexUUID()); } } } } } else { - for (IndexMetaData indexMetaData : remoteClusterState.getMetaData()) { - String[] patterns = request.getLeaderIndexPatterns().toArray(new String[0]); - if (Regex.simpleMatch(patterns, indexMetaData.getIndex().getName())) { - followedIndexUUIDS.add(indexMetaData.getIndexUUID()); + for (IndexMetaData indexMetaData : leaderClusterState.getMetaData()) { + String[] indexPatterns = request.getLeaderIndexPatterns().toArray(new String[0]); + if (Regex.simpleMatch(indexPatterns, indexMetaData.getIndex().getName())) { + followedIndexUUIDs.add(indexMetaData.getIndexUUID()); } } } @@ -133,10 +136,10 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, Cluster request.getFollowIndexNamePattern(), request.getMaxBatchOperationCount(), request.getMaxConcurrentReadBatches(), request.getMaxOperationSizeInBytes(), request.getMaxConcurrentWriteBatches(), request.getMaxWriteBufferSize(), request.getRetryTimeout(), request.getIdleShardRetryDelay()); - configurations.put(request.getRemoteClusterAlias(), autoFollowPattern); + patterns.put(request.getLeaderClusterAlias(), autoFollowPattern); ClusterState.Builder newState = ClusterState.builder(localState); newState.metaData(MetaData.builder(localState.getMetaData()) - .putCustom(AutoFollowMetadata.TYPE, new AutoFollowMetadata(configurations, followedLeaderIndices)) + .putCustom(AutoFollowMetadata.TYPE, new AutoFollowMetadata(patterns, followedLeaderIndices)) .build()); return newState.build(); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java index f3682a93a2202..771d2ea5c32dd 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java @@ -111,7 +111,7 @@ public void testAutoFollowParameterAreDelegated() throws Exception { // Enabling auto following: PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias("_local_"); + request.setLeaderClusterAlias("_local_"); request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); // Need to set this, because following an index in the same cluster request.setFollowIndexNamePattern("copy-{{leader_index}}"); @@ -173,7 +173,7 @@ public void testAutoFollowParameterAreDelegated() throws Exception { private void putAutoFollowPatterns(String... patterns) { PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias("_local_"); + request.setLeaderClusterAlias("_local_"); request.setLeaderIndexPatterns(Arrays.asList(patterns)); // Need to set this, because following an index in the same cluster request.setFollowIndexNamePattern("copy-{{leader_index}}"); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java index ae0cb68cbfd3f..27760578db945 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java @@ -32,7 +32,7 @@ protected PutAutoFollowPatternAction.Request createBlankInstance() { @Override protected PutAutoFollowPatternAction.Request createTestInstance() { PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias(randomAlphaOfLength(4)); + request.setLeaderClusterAlias(randomAlphaOfLength(4)); request.setLeaderIndexPatterns(Arrays.asList(generateRandomStringArray(4, 4, false))); if (randomBoolean()) { request.setFollowIndexNamePattern(randomAlphaOfLength(4)); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java index 0ab03918f1a4e..d894eda0b1186 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java @@ -27,7 +27,7 @@ public class TransportPutAutoFollowPatternActionTests extends ESTestCase { public void testInnerPut() { PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias("eu_cluster"); + request.setLeaderClusterAlias("eu_cluster"); request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); ClusterState localState = ClusterState.builder(new ClusterName("us_cluster")) @@ -50,7 +50,7 @@ public void testInnerPut() { public void testInnerPut_existingLeaderIndices() { PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias("eu_cluster"); + request.setLeaderClusterAlias("eu_cluster"); request.setLeaderIndexPatterns(Collections.singletonList("logs-*")); ClusterState localState = ClusterState.builder(new ClusterName("us_cluster")) @@ -89,7 +89,7 @@ public void testInnerPut_existingLeaderIndices() { public void testInnerPut_existingLeaderIndicesAndAutoFollowMetadata() { PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias("eu_cluster"); + request.setLeaderClusterAlias("eu_cluster"); request.setLeaderIndexPatterns(Arrays.asList("logs-*", "transactions-*")); Map existingAutoFollowPatterns = new HashMap<>(); From 4c4caee56feb0246685c4a468ae67cc58ba9675a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 30 Aug 2018 09:39:23 +0700 Subject: [PATCH 15/19] simplified innerPut() method and added AutoFollowPattern.match() method --- .../org/elasticsearch/common/regex/Regex.java | 15 +++++ .../ccr/action/AutoFollowCoordinator.java | 4 +- .../TransportPutAutoFollowPatternAction.java | 65 +++++++++++++------ .../xpack/core/ccr/AutoFollowMetadata.java | 9 +++ 4 files changed, 69 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/regex/Regex.java b/server/src/main/java/org/elasticsearch/common/regex/Regex.java index bcf2dfba3ef82..a102b869a6238 100644 --- a/server/src/main/java/org/elasticsearch/common/regex/Regex.java +++ b/server/src/main/java/org/elasticsearch/common/regex/Regex.java @@ -138,6 +138,21 @@ public static boolean simpleMatch(String[] patterns, String str) { return false; } + /** + * Similar to {@link #simpleMatch(String[], String)}, but accepts a list of strings instead of an array of strings + * for the patterns to match. + */ + public static boolean simpleMatch(List patterns, String str) { + if (patterns != null) { + for (String pattern : patterns) { + if (simpleMatch(pattern, str)) { + return true; + } + } + } + return false; + } + public static boolean simpleMatch(String[] patterns, String[] types) { if (patterns != null && types != null) { for (String type : types) { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index abd54cf7f2988..234fe32cdd0ee 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -17,7 +17,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.CountDown; @@ -250,9 +249,8 @@ static List getLeaderIndicesToFollow(AutoFollowPattern autoFollowPattern, ClusterState followerClusterState, List followedIndexUUIDs) { List leaderIndicesToFollow = new ArrayList<>(); - String[] patterns = autoFollowPattern.getLeaderIndexPatterns().toArray(new String[0]); for (IndexMetaData leaderIndexMetaData : leaderClusterState.getMetaData()) { - if (Regex.simpleMatch(patterns, leaderIndexMetaData.getIndex().getName())) { + if (autoFollowPattern.match(leaderIndexMetaData.getIndex().getName())) { if (followedIndexUUIDs.contains(leaderIndexMetaData.getIndex().getUUID()) == false) { // TODO: iterate over the indices in the followerClusterState and check whether a IndexMetaData // has a leader index uuid custom metadata entry that matches with uuid of leaderIndexMetaData variable diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java index bb1b08f4cee2c..01bb33cdabae1 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java @@ -25,11 +25,13 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class TransportPutAutoFollowPatternAction extends TransportMasterNodeAction { @@ -96,7 +98,7 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, AutoFollowMetadata currentAutoFollowMetadata = localState.metaData().custom(AutoFollowMetadata.TYPE); Map> followedLeaderIndices; - Map patterns; + Map patterns; if (currentAutoFollowMetadata != null) { patterns = new HashMap<>(currentAutoFollowMetadata.getPatterns()); followedLeaderIndices = new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); @@ -105,7 +107,7 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, followedLeaderIndices = new HashMap<>(); } - AutoFollowMetadata.AutoFollowPattern previousPattern = patterns.get(request.getLeaderClusterAlias()); + AutoFollowPattern previousPattern = patterns.get(request.getLeaderClusterAlias()); List followedIndexUUIDs = followedLeaderIndices.get(request.getLeaderClusterAlias()); if (followedIndexUUIDs == null) { followedIndexUUIDs = new ArrayList<>(); @@ -114,28 +116,24 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, // Mark existing leader indices as already auto followed: if (previousPattern != null) { - for (String newPattern : request.getLeaderIndexPatterns()) { - if (previousPattern.getLeaderIndexPatterns().contains(newPattern) == false) { - for (IndexMetaData indexMetaData : leaderClusterState.getMetaData()) { - if (Regex.simpleMatch(newPattern, indexMetaData.getIndex().getName())) { - followedIndexUUIDs.add(indexMetaData.getIndexUUID()); - } - } - } - } + markExistingIndicesAsAutoFollowedForNewPatterns(request.getLeaderIndexPatterns(), leaderClusterState.metaData(), + previousPattern, followedIndexUUIDs); } else { - for (IndexMetaData indexMetaData : leaderClusterState.getMetaData()) { - String[] indexPatterns = request.getLeaderIndexPatterns().toArray(new String[0]); - if (Regex.simpleMatch(indexPatterns, indexMetaData.getIndex().getName())) { - followedIndexUUIDs.add(indexMetaData.getIndexUUID()); - } - } + markExistingIndicesAsAutoFollowed(request.getLeaderIndexPatterns(), leaderClusterState.metaData(), + followedIndexUUIDs); } - AutoFollowMetadata.AutoFollowPattern autoFollowPattern = new AutoFollowMetadata.AutoFollowPattern(request.getLeaderIndexPatterns(), - request.getFollowIndexNamePattern(), request.getMaxBatchOperationCount(), request.getMaxConcurrentReadBatches(), - request.getMaxOperationSizeInBytes(), request.getMaxConcurrentWriteBatches(), request.getMaxWriteBufferSize(), - request.getRetryTimeout(), request.getIdleShardRetryDelay()); + AutoFollowPattern autoFollowPattern = new AutoFollowPattern( + request.getLeaderIndexPatterns(), + request.getFollowIndexNamePattern(), + request.getMaxBatchOperationCount(), + request.getMaxConcurrentReadBatches(), + request.getMaxOperationSizeInBytes(), + request.getMaxConcurrentWriteBatches(), + request.getMaxWriteBufferSize(), + request.getRetryTimeout(), + request.getIdleShardRetryDelay() + ); patterns.put(request.getLeaderClusterAlias(), autoFollowPattern); ClusterState.Builder newState = ClusterState.builder(localState); newState.metaData(MetaData.builder(localState.getMetaData()) @@ -144,6 +142,31 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, return newState.build(); } + private static void markExistingIndicesAsAutoFollowedForNewPatterns( + List leaderIndexPatterns, + MetaData leaderMetaData, + AutoFollowPattern previousPattern, + List followedIndexUUIDS) { + + final List newPatterns = leaderIndexPatterns + .stream() + .filter(p -> previousPattern.getLeaderIndexPatterns().contains(p) == false) + .collect(Collectors.toList()); + markExistingIndicesAsAutoFollowed(newPatterns, leaderMetaData, followedIndexUUIDS); + } + + private static void markExistingIndicesAsAutoFollowed( + List patterns, + MetaData leaderMetaData, + List followedIndexUUIDS) { + + for (final IndexMetaData indexMetaData : leaderMetaData) { + if (AutoFollowPattern.match(patterns, indexMetaData.getIndex().getName())) { + followedIndexUUIDS.add(indexMetaData.getIndexUUID()); + } + } + } + @Override protected ClusterBlockException checkBlock(PutAutoFollowPatternAction.Request request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); 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 eaf05f88d9659..244a5d441d9bf 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 @@ -13,6 +13,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ObjectParser; @@ -229,6 +230,14 @@ public AutoFollowPattern(List leaderIndexPatterns, String followIndexPat idleShardRetryDelay = in.readOptionalTimeValue(); } + public boolean match(String indexName) { + return match(leaderIndexPatterns, indexName); + } + + public static boolean match(List leaderIndexPatterns, String indexName) { + return Regex.simpleMatch(leaderIndexPatterns, indexName); + } + public List getLeaderIndexPatterns() { return leaderIndexPatterns; } From 01819ab9f0d1252220d8b6f3fc3104244f369b5d Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 30 Aug 2018 09:58:31 +0700 Subject: [PATCH 16/19] iter delete autofollow api --- .../action/DeleteAutoFollowPatternAction.java | 22 ++++++------ ...ransportDeleteAutoFollowPatternAction.java | 34 +++++++------------ .../TransportPutAutoFollowPatternAction.java | 1 - .../RestDeleteAutoFollowPatternAction.java | 4 +-- .../rest/RestPutAutoFollowPatternAction.java | 4 +-- .../xpack/ccr/action/AutoFollowTests.java | 2 +- .../DeleteAutoFollowPatternRequestTests.java | 2 +- ...ortDeleteAutoFollowPatternActionTests.java | 10 +++--- .../api/ccr.delete_auto_follow_pattern.json | 8 ++--- .../api/ccr.put_auto_follow_pattern.json | 8 ++--- 10 files changed, 43 insertions(+), 52 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java index 0355ca525267b..82e142202d26c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternAction.java @@ -33,35 +33,35 @@ public AcknowledgedResponse newResponse() { public static class Request extends AcknowledgedRequest { - private String remoteClusterAlias; + private String leaderClusterAlias; @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; - if (remoteClusterAlias == null) { - validationException = addValidationError("remoteClusterAlias is missing", validationException); + if (leaderClusterAlias == null) { + validationException = addValidationError("leaderClusterAlias is missing", validationException); } return validationException; } - public String getRemoteClusterAlias() { - return remoteClusterAlias; + public String getLeaderClusterAlias() { + return leaderClusterAlias; } - public void setRemoteClusterAlias(String remoteClusterAlias) { - this.remoteClusterAlias = remoteClusterAlias; + public void setLeaderClusterAlias(String leaderClusterAlias) { + this.leaderClusterAlias = leaderClusterAlias; } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - remoteClusterAlias = in.readString(); + leaderClusterAlias = in.readString(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(remoteClusterAlias); + out.writeString(leaderClusterAlias); } @Override @@ -69,12 +69,12 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; - return Objects.equals(remoteClusterAlias, request.remoteClusterAlias); + return Objects.equals(leaderClusterAlias, request.leaderClusterAlias); } @Override public int hashCode() { - return Objects.hash(remoteClusterAlias); + return Objects.hash(leaderClusterAlias); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java index 24d4d2574c9c5..6c1ca81e7c49a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternAction.java @@ -22,12 +22,11 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; +import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; public class TransportDeleteAutoFollowPatternAction extends TransportMasterNodeAction { @@ -54,7 +53,7 @@ protected AcknowledgedResponse newResponse() { protected void masterOperation(DeleteAutoFollowPatternAction.Request request, ClusterState state, ActionListener listener) throws Exception { - clusterService.submitStateUpdateTask("put_auto_follow_pattern-" + request.getRemoteClusterAlias(), + clusterService.submitStateUpdateTask("put-auto-follow-pattern-" + request.getLeaderClusterAlias(), new AckedClusterStateUpdateTask(request, listener) { @Override @@ -72,30 +71,23 @@ public ClusterState execute(ClusterState currentState) throws Exception { static ClusterState innerDelete(DeleteAutoFollowPatternAction.Request request, ClusterState currentState) { AutoFollowMetadata currentAutoFollowMetadata = currentState.metaData().custom(AutoFollowMetadata.TYPE); if (currentAutoFollowMetadata == null) { - throw new ResourceNotFoundException("auto follow patterns for [{}] cluster alias are missing", - request.getRemoteClusterAlias()); + throw new ResourceNotFoundException("no auto-follow patterns for cluster alias [{}] found", + request.getLeaderClusterAlias()); } - Map configurations = currentAutoFollowMetadata.getPatterns(); - Set toRemove = new HashSet<>(); - for (String configurationKey : configurations.keySet()) { - if (request.getRemoteClusterAlias().equals(configurationKey)) { - toRemove.add(configurationKey); - } - } - if (toRemove.isEmpty()) { - throw new ResourceNotFoundException("auto follow patterns for [{}] cluster alias are missing", - request.getRemoteClusterAlias()); + Map patterns = currentAutoFollowMetadata.getPatterns(); + AutoFollowPattern autoFollowPatternToRemove = patterns.get(request.getLeaderClusterAlias()); + if (autoFollowPatternToRemove == null) { + throw new ResourceNotFoundException("no auto-follow patterns for cluster alias [{}] found", + request.getLeaderClusterAlias()); } - final Map configurationsCopy = new HashMap<>(configurations); + final Map patternsCopy = new HashMap<>(patterns); final Map> followedLeaderIndexUUIDSCopy = new HashMap<>(currentAutoFollowMetadata.getFollowedLeaderIndexUUIDs()); - for (String key : toRemove) { - configurationsCopy.remove(key); - followedLeaderIndexUUIDSCopy.remove(key); - } + patternsCopy.remove(request.getLeaderClusterAlias()); + followedLeaderIndexUUIDSCopy.remove(request.getLeaderClusterAlias()); - AutoFollowMetadata newAutoFollowMetadata = new AutoFollowMetadata(configurationsCopy, followedLeaderIndexUUIDSCopy); + AutoFollowMetadata newAutoFollowMetadata = new AutoFollowMetadata(patternsCopy, followedLeaderIndexUUIDSCopy); ClusterState.Builder newState = ClusterState.builder(currentState); newState.metaData(MetaData.builder(currentState.getMetaData()) .putCustom(AutoFollowMetadata.TYPE, newAutoFollowMetadata) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java index 01bb33cdabae1..3d3e342c0cd3e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java @@ -20,7 +20,6 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java index 9899e88f35aab..bd3585c7982c1 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestDeleteAutoFollowPatternAction.java @@ -21,7 +21,7 @@ public class RestDeleteAutoFollowPatternAction extends BaseRestHandler { public RestDeleteAutoFollowPatternAction(Settings settings, RestController controller) { super(settings); - controller.registerHandler(RestRequest.Method.DELETE, "/_ccr/_auto_follow/{remote_cluster_alias}", this); + controller.registerHandler(RestRequest.Method.DELETE, "/_ccr/_auto_follow/{leader_cluster_alias}", this); } @Override @@ -32,7 +32,7 @@ public String getName() { @Override protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { Request request = new Request(); - request.setRemoteClusterAlias(restRequest.param("remote_cluster_alias")); + request.setLeaderClusterAlias(restRequest.param("leader_cluster_alias")); return channel -> client.execute(INSTANCE, request, new RestToXContentListener<>(channel)); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java index 93a455d8e4b4b..d92ebb7b0bbe5 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestPutAutoFollowPatternAction.java @@ -22,7 +22,7 @@ public class RestPutAutoFollowPatternAction extends BaseRestHandler { public RestPutAutoFollowPatternAction(Settings settings, RestController controller) { super(settings); - controller.registerHandler(RestRequest.Method.PUT, "/_ccr/_auto_follow/{remote_cluster_alias}", this); + controller.registerHandler(RestRequest.Method.PUT, "/_ccr/_auto_follow/{leader_cluster_alias}", this); } @Override @@ -38,7 +38,7 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient static Request createRequest(RestRequest restRequest) throws IOException { try (XContentParser parser = restRequest.contentOrSourceParamParser()) { - return Request.fromXContent(parser, restRequest.param("remote_cluster_alias")); + return Request.fromXContent(parser, restRequest.param("leader_cluster_alias")); } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java index 771d2ea5c32dd..a4808e428feca 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowTests.java @@ -182,7 +182,7 @@ private void putAutoFollowPatterns(String... patterns) { private void deleteAutoFollowPatternSetting() { DeleteAutoFollowPatternAction.Request request = new DeleteAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias("_local_"); + request.setLeaderClusterAlias("_local_"); assertTrue(client().execute(DeleteAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java index 1b5192f253fab..0ca1b3d127827 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/DeleteAutoFollowPatternRequestTests.java @@ -17,7 +17,7 @@ protected DeleteAutoFollowPatternAction.Request createBlankInstance() { @Override protected DeleteAutoFollowPatternAction.Request createTestInstance() { DeleteAutoFollowPatternAction.Request request = new DeleteAutoFollowPatternAction.Request(); - request.setRemoteClusterAlias(randomAlphaOfLength(4)); + request.setLeaderClusterAlias(randomAlphaOfLength(4)); return request; } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java index 8876d4b24c9b0..03065ea8d38f2 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java @@ -52,7 +52,7 @@ public void testInnerDelete() { .build(); Request request = new Request(); - request.setRemoteClusterAlias("eu_cluster"); + request.setLeaderClusterAlias("eu_cluster"); AutoFollowMetadata result = TransportDeleteAutoFollowPatternAction.innerDelete(request, clusterState) .getMetaData() .custom(AutoFollowMetadata.TYPE); @@ -77,10 +77,10 @@ public void testInnerDeleteDoesNotExist() { .build(); Request request = new Request(); - request.setRemoteClusterAlias("asia_cluster"); + request.setLeaderClusterAlias("asia_cluster"); Exception e = expectThrows(ResourceNotFoundException.class, () -> TransportDeleteAutoFollowPatternAction.innerDelete(request, clusterState)); - assertThat(e.getMessage(), equalTo("auto follow patterns for [asia_cluster] cluster alias are missing")); + assertThat(e.getMessage(), equalTo("no auto-follow patterns for cluster alias [asia_cluster] found")); } public void testInnerDeleteNoAutoFollowMetadata() { @@ -89,10 +89,10 @@ public void testInnerDeleteNoAutoFollowMetadata() { .build(); Request request = new Request(); - request.setRemoteClusterAlias("asia_cluster"); + request.setLeaderClusterAlias("asia_cluster"); Exception e = expectThrows(ResourceNotFoundException.class, () -> TransportDeleteAutoFollowPatternAction.innerDelete(request, clusterState)); - assertThat(e.getMessage(), equalTo("auto follow patterns for [asia_cluster] cluster alias are missing")); + assertThat(e.getMessage(), equalTo("no auto-follow patterns for cluster alias [asia_cluster] found")); } } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json index d883e05153ec5..b14effd5f3f73 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.delete_auto_follow_pattern.json @@ -3,13 +3,13 @@ "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current", "methods": [ "DELETE" ], "url": { - "path": "/_ccr/_auto_follow/{remote_cluster_alias}", - "paths": [ "/_ccr/_auto_follow/{remote_cluster_alias}" ], + "path": "/_ccr/_auto_follow/{leader_cluster_alias}", + "paths": [ "/_ccr/_auto_follow/{leader_cluster_alias}" ], "parts": { - "remote_cluster_alias": { + "leader_cluster_alias": { "type": "string", "required": true, - "description": "The name of the remote cluster alias." + "description": "The name of the leader cluster alias." } } } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json index 9ff8acafe1466..28e7299713da5 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.put_auto_follow_pattern.json @@ -3,13 +3,13 @@ "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current", "methods": [ "PUT" ], "url": { - "path": "/_ccr/_auto_follow/{remote_cluster_alias}", - "paths": [ "/_ccr/_auto_follow/{remote_cluster_alias}" ], + "path": "/_ccr/_auto_follow/{leader_cluster_alias}", + "paths": [ "/_ccr/_auto_follow/{leader_cluster_alias}" ], "parts": { - "remote_cluster_alias": { + "leader_cluster_alias": { "type": "string", "required": true, - "description": "The name of the remote cluster alias." + "description": "The name of the leader cluster alias." } } }, From cf984b32d2488aa7a63cbbdf74245a687b8843ed Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 5 Sep 2018 20:51:40 +0200 Subject: [PATCH 17/19] fixed yaml tests --- .../src/test/resources/rest-api-spec/test/ccr/auto_follow.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml index c6974e9f37bb6..f4cf79fb5589a 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml @@ -2,12 +2,12 @@ "Test put and delete auto follow pattern": - do: ccr.put_auto_follow_pattern: - remote_cluster_alias: _local_ + leader_cluster_alias: _local_ body: leader_index_patterns: ['logs-*'] - is_true: acknowledged - do: ccr.delete_auto_follow_pattern: - remote_cluster_alias: _local_ + leader_cluster_alias: _local_ - is_true: acknowledged From f8e9ffe493714e3617d79551a76f8e7d6b35a893 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 5 Sep 2018 21:30:00 -0400 Subject: [PATCH 18/19] Delegate to simpleMatch, it will probably be inlined anyway --- .../org/elasticsearch/common/regex/Regex.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/regex/Regex.java b/server/src/main/java/org/elasticsearch/common/regex/Regex.java index a102b869a6238..1f4e465141222 100644 --- a/server/src/main/java/org/elasticsearch/common/regex/Regex.java +++ b/server/src/main/java/org/elasticsearch/common/regex/Regex.java @@ -139,18 +139,12 @@ public static boolean simpleMatch(String[] patterns, String str) { } /** - * Similar to {@link #simpleMatch(String[], String)}, but accepts a list of strings instead of an array of strings - * for the patterns to match. + * Similar to {@link #simpleMatch(String[], String)}, but accepts a list of strings instead of an array of strings for the patterns to + * match. */ - public static boolean simpleMatch(List patterns, String str) { - if (patterns != null) { - for (String pattern : patterns) { - if (simpleMatch(pattern, str)) { - return true; - } - } - } - return false; + public static boolean simpleMatch(final List patterns, final String str) { + // #simpleMatch(String[], String) is likely to be inlined into this method + return patterns != null && simpleMatch(patterns.toArray(Strings.EMPTY_ARRAY), str); } public static boolean simpleMatch(String[] patterns, String[] types) { From 7dea40c3841f1f9230910de2634ae451b62c9476 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 5 Sep 2018 21:31:53 -0400 Subject: [PATCH 19/19] Organize endpoints --- .../java/org/elasticsearch/xpack/ccr/Ccr.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java index 1b783f8ee5564..cd0561b1c0c60 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java @@ -142,14 +142,18 @@ public List> getPersistentTasksExecutor(ClusterServic } return Arrays.asList( + // internal actions new ActionHandler<>(BulkShardOperationsAction.INSTANCE, TransportBulkShardOperationsAction.class), + new ActionHandler<>(ShardChangesAction.INSTANCE, ShardChangesAction.TransportAction.class), + // stats action new ActionHandler<>(CcrStatsAction.INSTANCE, TransportCcrStatsAction.class), + // follow actions new ActionHandler<>(CreateAndFollowIndexAction.INSTANCE, CreateAndFollowIndexAction.TransportAction.class), new ActionHandler<>(FollowIndexAction.INSTANCE, FollowIndexAction.TransportAction.class), - new ActionHandler<>(ShardChangesAction.INSTANCE, ShardChangesAction.TransportAction.class), new ActionHandler<>(UnfollowIndexAction.INSTANCE, UnfollowIndexAction.TransportAction.class), - new ActionHandler<>(PutAutoFollowPatternAction.INSTANCE, TransportPutAutoFollowPatternAction.class), - new ActionHandler<>(DeleteAutoFollowPatternAction.INSTANCE, TransportDeleteAutoFollowPatternAction.class)); + // auto-follow actions + new ActionHandler<>(DeleteAutoFollowPatternAction.INSTANCE, TransportDeleteAutoFollowPatternAction.class), + new ActionHandler<>(PutAutoFollowPatternAction.INSTANCE, TransportPutAutoFollowPatternAction.class)); } public List getRestHandlers(Settings settings, RestController restController, ClusterSettings clusterSettings, @@ -157,13 +161,15 @@ public List getRestHandlers(Settings settings, RestController restC IndexNameExpressionResolver indexNameExpressionResolver, Supplier nodesInCluster) { return Arrays.asList( + // stats API new RestCcrStatsAction(settings, restController), + // follow APIs new RestCreateAndFollowIndexAction(settings, restController), new RestFollowIndexAction(settings, restController), new RestUnfollowIndexAction(settings, restController), - new RestPutAutoFollowPatternAction(settings, restController), - new RestDeleteAutoFollowPatternAction(settings, restController) - ); + // auto-follow APIs + new RestDeleteAutoFollowPatternAction(settings, restController), + new RestPutAutoFollowPatternAction(settings, restController)); } public List getNamedWriteables() {