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 353a66db26339..66b8a5c9590db 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 @@ -40,19 +40,21 @@ 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.TransportUnfollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.CcrStatsAction; +import org.elasticsearch.xpack.ccr.action.TransportCreateAndFollowIndexAction; +import org.elasticsearch.xpack.ccr.action.TransportFollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; import org.elasticsearch.xpack.ccr.action.DeleteAutoFollowPatternAction; -import org.elasticsearch.xpack.ccr.action.FollowIndexAction; +import org.elasticsearch.xpack.core.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.core.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; @@ -63,6 +65,7 @@ import org.elasticsearch.xpack.ccr.rest.RestPutAutoFollowPatternAction; import org.elasticsearch.xpack.ccr.rest.RestUnfollowIndexAction; import org.elasticsearch.xpack.core.XPackPlugin; +import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; import java.util.Arrays; import java.util.Collection; @@ -148,9 +151,9 @@ public List> getPersistentTasksExecutor(ClusterServic // 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<>(UnfollowIndexAction.INSTANCE, UnfollowIndexAction.TransportAction.class), + new ActionHandler<>(CreateAndFollowIndexAction.INSTANCE, TransportCreateAndFollowIndexAction.class), + new ActionHandler<>(FollowIndexAction.INSTANCE, TransportFollowIndexAction.class), + new ActionHandler<>(UnfollowIndexAction.INSTANCE, TransportUnfollowIndexAction.class), // auto-follow actions new ActionHandler<>(DeleteAutoFollowPatternAction.INSTANCE, TransportDeleteAutoFollowPatternAction.class), new ActionHandler<>(PutAutoFollowPatternAction.INSTANCE, TransportPutAutoFollowPatternAction.class)); @@ -179,8 +182,8 @@ public List getNamedWriteables() { ShardFollowTask::new), // Task statuses - new NamedWriteableRegistry.Entry(Task.Status.class, ShardFollowNodeTask.Status.STATUS_PARSER_NAME, - ShardFollowNodeTask.Status::new) + new NamedWriteableRegistry.Entry(Task.Status.class, ShardFollowNodeTaskStatus.STATUS_PARSER_NAME, + ShardFollowNodeTaskStatus::new) ); } @@ -192,9 +195,9 @@ public List getNamedXContent() { // Task statuses new NamedXContentRegistry.Entry( - ShardFollowNodeTask.Status.class, - new ParseField(ShardFollowNodeTask.Status.STATUS_PARSER_NAME), - ShardFollowNodeTask.Status::fromXContent)); + ShardFollowNodeTaskStatus.class, + new ParseField(ShardFollowNodeTaskStatus.STATUS_PARSER_NAME), + ShardFollowNodeTaskStatus::fromXContent)); } /** 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 e28214341a927..bc62e439538c8 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 @@ -27,6 +27,8 @@ import org.elasticsearch.xpack.ccr.CcrSettings; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; import java.util.ArrayList; import java.util.HashMap; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexAction.java deleted file mode 100644 index 223f6ed8e6d25..0000000000000 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexAction.java +++ /dev/null @@ -1,367 +0,0 @@ -/* - * 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 com.carrotsearch.hppc.cursors.ObjectObjectCursor; -import org.elasticsearch.ResourceAlreadyExistsException; -import org.elasticsearch.action.Action; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.ActiveShardCount; -import org.elasticsearch.action.support.ActiveShardsObserver; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.AcknowledgedRequest; -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.MappingMetaData; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.license.LicenseUtils; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.RemoteClusterAware; -import org.elasticsearch.transport.RemoteClusterService; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.ccr.CcrLicenseChecker; -import org.elasticsearch.xpack.ccr.CcrSettings; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -public class CreateAndFollowIndexAction extends Action { - - public static final CreateAndFollowIndexAction INSTANCE = new CreateAndFollowIndexAction(); - public static final String NAME = "indices:admin/xpack/ccr/create_and_follow_index"; - - private CreateAndFollowIndexAction() { - super(NAME); - } - - @Override - public Response newResponse() { - return new Response(); - } - - public static class Request extends AcknowledgedRequest implements IndicesRequest { - - private FollowIndexAction.Request followRequest; - - public Request(FollowIndexAction.Request followRequest) { - this.followRequest = Objects.requireNonNull(followRequest); - } - - Request() { - } - - public FollowIndexAction.Request getFollowRequest() { - return followRequest; - } - - @Override - public ActionRequestValidationException validate() { - return followRequest.validate(); - } - - @Override - public String[] indices() { - return new String[]{followRequest.getFollowerIndex()}; - } - - @Override - public IndicesOptions indicesOptions() { - return IndicesOptions.strictSingleIndexNoExpandForbidClosed(); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - followRequest = new FollowIndexAction.Request(); - followRequest.readFrom(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - followRequest.writeTo(out); - } - - @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(followRequest, request.followRequest); - } - - @Override - public int hashCode() { - return Objects.hash(followRequest); - } - } - - public static class Response extends ActionResponse implements ToXContentObject { - - private boolean followIndexCreated; - private boolean followIndexShardsAcked; - private boolean indexFollowingStarted; - - Response() { - } - - Response(boolean followIndexCreated, boolean followIndexShardsAcked, boolean indexFollowingStarted) { - this.followIndexCreated = followIndexCreated; - this.followIndexShardsAcked = followIndexShardsAcked; - this.indexFollowingStarted = indexFollowingStarted; - } - - public boolean isFollowIndexCreated() { - return followIndexCreated; - } - - public boolean isFollowIndexShardsAcked() { - return followIndexShardsAcked; - } - - public boolean isIndexFollowingStarted() { - return indexFollowingStarted; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - followIndexCreated = in.readBoolean(); - followIndexShardsAcked = in.readBoolean(); - indexFollowingStarted = in.readBoolean(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeBoolean(followIndexCreated); - out.writeBoolean(followIndexShardsAcked); - out.writeBoolean(indexFollowingStarted); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - { - builder.field("follow_index_created", followIndexCreated); - builder.field("follow_index_shards_acked", followIndexShardsAcked); - builder.field("index_following_started", indexFollowingStarted); - } - builder.endObject(); - return builder; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Response response = (Response) o; - return followIndexCreated == response.followIndexCreated && - followIndexShardsAcked == response.followIndexShardsAcked && - indexFollowingStarted == response.indexFollowingStarted; - } - - @Override - public int hashCode() { - return Objects.hash(followIndexCreated, followIndexShardsAcked, indexFollowingStarted); - } - } - - public static class TransportAction extends TransportMasterNodeAction { - - private final Client client; - private final AllocationService allocationService; - private final RemoteClusterService remoteClusterService; - private final ActiveShardsObserver activeShardsObserver; - private final CcrLicenseChecker ccrLicenseChecker; - - @Inject - public TransportAction( - final Settings settings, - final ThreadPool threadPool, - final TransportService transportService, - final ClusterService clusterService, - final ActionFilters actionFilters, - final IndexNameExpressionResolver indexNameExpressionResolver, - final Client client, - final AllocationService allocationService, - final CcrLicenseChecker ccrLicenseChecker) { - super(settings, NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, Request::new); - this.client = client; - this.allocationService = allocationService; - this.remoteClusterService = transportService.getRemoteClusterService(); - this.activeShardsObserver = new ActiveShardsObserver(settings, clusterService, threadPool); - this.ccrLicenseChecker = Objects.requireNonNull(ccrLicenseChecker); - } - - @Override - protected String executor() { - return ThreadPool.Names.SAME; - } - - @Override - protected Response newResponse() { - return new Response(); - } - - @Override - protected void masterOperation( - final Request request, final ClusterState state, final ActionListener listener) throws Exception { - if (ccrLicenseChecker.isCcrAllowed() == false) { - listener.onFailure(LicenseUtils.newComplianceException("ccr")); - return; - } - final String[] indices = new String[]{request.getFollowRequest().getLeaderIndex()}; - final Map> remoteClusterIndices = remoteClusterService.groupClusterIndices(indices, s -> false); - if (remoteClusterIndices.containsKey(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)) { - createFollowerIndexAndFollowLocalIndex(request, state, listener); - } else { - assert remoteClusterIndices.size() == 1; - final Map.Entry> entry = remoteClusterIndices.entrySet().iterator().next(); - assert entry.getValue().size() == 1; - final String clusterAlias = entry.getKey(); - final String leaderIndex = entry.getValue().get(0); - createFollowerIndexAndFollowRemoteIndex(request, clusterAlias, leaderIndex, listener); - } - } - - private void createFollowerIndexAndFollowLocalIndex( - final Request request, final ClusterState state, final ActionListener listener) { - // following an index in local cluster, so use local cluster state to fetch leader index metadata - final IndexMetaData leaderIndexMetadata = state.getMetaData().index(request.getFollowRequest().getLeaderIndex()); - createFollowerIndex(leaderIndexMetadata, request, listener); - } - - private void createFollowerIndexAndFollowRemoteIndex( - final Request request, - final String clusterAlias, - final String leaderIndex, - final ActionListener listener) { - ccrLicenseChecker.checkRemoteClusterLicenseAndFetchLeaderIndexMetadata( - client, - clusterAlias, - leaderIndex, - listener::onFailure, - leaderIndexMetaData -> createFollowerIndex(leaderIndexMetaData, request, listener)); - } - - private void createFollowerIndex( - final IndexMetaData leaderIndexMetaData, final Request request, final ActionListener listener) { - if (leaderIndexMetaData == null) { - listener.onFailure(new IllegalArgumentException("leader index [" + request.getFollowRequest().getLeaderIndex() + - "] does not exist")); - return; - } - - ActionListener handler = ActionListener.wrap( - result -> { - if (result) { - initiateFollowing(request, listener); - } else { - listener.onResponse(new Response(true, false, false)); - } - }, - listener::onFailure); - // Can't use create index api here, because then index templates can alter the mappings / settings. - // And index templates could introduce settings / mappings that are incompatible with the leader index. - clusterService.submitStateUpdateTask("follow_index_action", new AckedClusterStateUpdateTask(request, handler) { - - @Override - protected Boolean newResponse(boolean acknowledged) { - return acknowledged; - } - - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - String followIndex = request.getFollowRequest().getFollowerIndex(); - IndexMetaData currentIndex = currentState.metaData().index(followIndex); - if (currentIndex != null) { - throw new ResourceAlreadyExistsException(currentIndex.getIndex()); - } - - MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); - IndexMetaData.Builder imdBuilder = IndexMetaData.builder(followIndex); - - // Copy all settings, but overwrite a few settings. - Settings.Builder settingsBuilder = Settings.builder(); - settingsBuilder.put(leaderIndexMetaData.getSettings()); - // Overwriting UUID here, because otherwise we can't follow indices in the same cluster - settingsBuilder.put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()); - settingsBuilder.put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followIndex); - settingsBuilder.put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); - imdBuilder.settings(settingsBuilder); - - // Copy mappings from leader IMD to follow IMD - for (ObjectObjectCursor cursor : leaderIndexMetaData.getMappings()) { - imdBuilder.putMapping(cursor.value); - } - imdBuilder.setRoutingNumShards(leaderIndexMetaData.getRoutingNumShards()); - IndexMetaData followIMD = imdBuilder.build(); - mdBuilder.put(followIMD, false); - - ClusterState.Builder builder = ClusterState.builder(currentState); - builder.metaData(mdBuilder.build()); - ClusterState updatedState = builder.build(); - - RoutingTable.Builder routingTableBuilder = RoutingTable.builder(updatedState.routingTable()) - .addAsNew(updatedState.metaData().index(request.getFollowRequest().getFollowerIndex())); - updatedState = allocationService.reroute( - ClusterState.builder(updatedState).routingTable(routingTableBuilder.build()).build(), - "follow index [" + request.getFollowRequest().getFollowerIndex() + "] created"); - - logger.info("[{}] creating index, cause [ccr_create_and_follow], shards [{}]/[{}]", - followIndex, followIMD.getNumberOfShards(), followIMD.getNumberOfReplicas()); - - return updatedState; - } - }); - } - - private void initiateFollowing(Request request, ActionListener listener) { - activeShardsObserver.waitForActiveShards(new String[]{request.followRequest.getFollowerIndex()}, - ActiveShardCount.DEFAULT, request.timeout(), result -> { - if (result) { - client.execute(FollowIndexAction.INSTANCE, request.getFollowRequest(), ActionListener.wrap( - r -> listener.onResponse(new Response(true, true, r.isAcknowledged())), - listener::onFailure - )); - } else { - listener.onResponse(new Response(true, false, false)); - } - }, listener::onFailure); - } - - @Override - protected ClusterBlockException checkBlock(Request request, ClusterState state) { - return state.blocks().indexBlockedException(ClusterBlockLevel.METADATA_WRITE, request.getFollowRequest().getFollowerIndex()); - } - - } - -} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowIndexAction.java deleted file mode 100644 index 498224551106d..0000000000000 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowIndexAction.java +++ /dev/null @@ -1,571 +0,0 @@ -/* - * 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.ActionListener; -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.HandledTransportAction; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; -import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Settings; -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.index.IndexSettings; -import org.elasticsearch.index.IndexingSlowLog; -import org.elasticsearch.index.SearchSlowLog; -import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.IndicesRequestCache; -import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.license.LicenseUtils; -import org.elasticsearch.persistent.PersistentTasksCustomMetaData; -import org.elasticsearch.persistent.PersistentTasksService; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.RemoteClusterAware; -import org.elasticsearch.transport.RemoteClusterService; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.ccr.CcrLicenseChecker; -import org.elasticsearch.xpack.ccr.CcrSettings; - -import java.io.IOException; -import java.util.Collections; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReferenceArray; -import java.util.stream.Collectors; - -public class FollowIndexAction extends Action { - - public static final FollowIndexAction INSTANCE = new FollowIndexAction(); - public static final String NAME = "cluster:admin/xpack/ccr/follow_index"; - - private FollowIndexAction() { - super(NAME); - } - - @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); - } - - public static class Request extends ActionRequest implements ToXContentObject { - - private static final ParseField LEADER_INDEX_FIELD = new ParseField("leader_index"); - private static final ParseField FOLLOWER_INDEX_FIELD = new ParseField("follower_index"); - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, true, - (args, followerIndex) -> { - if (args[1] != null) { - followerIndex = (String) args[1]; - } - return new Request((String) args[0], followerIndex, (Integer) args[2], (Integer) args[3], (Long) args[4], - (Integer) args[5], (Integer) args[6], (TimeValue) args[7], (TimeValue) args[8]); - }); - - static { - PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), LEADER_INDEX_FIELD); - PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), FOLLOWER_INDEX_FIELD); - PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), ShardFollowTask.MAX_BATCH_OPERATION_COUNT); - PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), ShardFollowTask.MAX_CONCURRENT_READ_BATCHES); - PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES); - PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), ShardFollowTask.MAX_CONCURRENT_WRITE_BATCHES); - PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), ShardFollowTask.MAX_WRITE_BUFFER_SIZE); - PARSER.declareField(ConstructingObjectParser.optionalConstructorArg(), - (p, c) -> TimeValue.parseTimeValue(p.text(), ShardFollowTask.RETRY_TIMEOUT.getPreferredName()), - ShardFollowTask.RETRY_TIMEOUT, ObjectParser.ValueType.STRING); - PARSER.declareField(ConstructingObjectParser.optionalConstructorArg(), - (p, c) -> TimeValue.parseTimeValue(p.text(), ShardFollowTask.IDLE_SHARD_RETRY_DELAY.getPreferredName()), - ShardFollowTask.IDLE_SHARD_RETRY_DELAY, ObjectParser.ValueType.STRING); - } - - public static Request fromXContent(XContentParser parser, String followerIndex) throws IOException { - Request request = PARSER.parse(parser, followerIndex); - if (followerIndex != null) { - if (request.followerIndex == null) { - request.followerIndex = followerIndex; - } else { - if (request.followerIndex.equals(followerIndex) == false) { - throw new IllegalArgumentException("provided follower_index is not equal"); - } - } - } - return request; - } - - private String leaderIndex; - private String followerIndex; - private int maxBatchOperationCount; - private int maxConcurrentReadBatches; - private long maxOperationSizeInBytes; - private int maxConcurrentWriteBatches; - private int maxWriteBufferSize; - private TimeValue retryTimeout; - private TimeValue idleShardRetryDelay; - - public Request( - String leaderIndex, - String followerIndex, - Integer maxBatchOperationCount, - Integer maxConcurrentReadBatches, - Long maxOperationSizeInBytes, - Integer maxConcurrentWriteBatches, - Integer maxWriteBufferSize, - TimeValue retryTimeout, - TimeValue idleShardRetryDelay) { - - if (leaderIndex == null) { - throw new IllegalArgumentException("leader_index is missing"); - } - if (followerIndex == null) { - throw new IllegalArgumentException("follower_index is missing"); - } - if (maxBatchOperationCount == null) { - maxBatchOperationCount = ShardFollowNodeTask.DEFAULT_MAX_BATCH_OPERATION_COUNT; - } - if (maxConcurrentReadBatches == null) { - maxConcurrentReadBatches = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READ_BATCHES; - } - if (maxOperationSizeInBytes == null) { - maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_BATCH_SIZE_IN_BYTES; - } - if (maxConcurrentWriteBatches == null) { - maxConcurrentWriteBatches = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITE_BATCHES; - } - if (maxWriteBufferSize == null) { - maxWriteBufferSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_BUFFER_SIZE; - } - if (retryTimeout == null) { - retryTimeout = ShardFollowNodeTask.DEFAULT_RETRY_TIMEOUT; - } - if (idleShardRetryDelay == null) { - idleShardRetryDelay = ShardFollowNodeTask.DEFAULT_IDLE_SHARD_RETRY_DELAY; - } - - if (maxBatchOperationCount < 1) { - throw new IllegalArgumentException("maxBatchOperationCount must be larger than 0"); - } - if (maxConcurrentReadBatches < 1) { - throw new IllegalArgumentException("concurrent_processors must be larger than 0"); - } - if (maxOperationSizeInBytes <= 0) { - throw new IllegalArgumentException("processor_max_translog_bytes must be larger than 0"); - } - if (maxConcurrentWriteBatches < 1) { - throw new IllegalArgumentException("maxConcurrentWriteBatches must be larger than 0"); - } - if (maxWriteBufferSize < 1) { - throw new IllegalArgumentException("maxWriteBufferSize must be larger than 0"); - } - - this.leaderIndex = leaderIndex; - this.followerIndex = followerIndex; - this.maxBatchOperationCount = maxBatchOperationCount; - this.maxConcurrentReadBatches = maxConcurrentReadBatches; - this.maxOperationSizeInBytes = maxOperationSizeInBytes; - this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; - this.maxWriteBufferSize = maxWriteBufferSize; - this.retryTimeout = retryTimeout; - this.idleShardRetryDelay = idleShardRetryDelay; - } - - Request() { - } - - public String getLeaderIndex() { - return leaderIndex; - } - - public String getFollowerIndex() { - return followerIndex; - } - - public int getMaxBatchOperationCount() { - return maxBatchOperationCount; - } - - @Override - public ActionRequestValidationException validate() { - return null; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - leaderIndex = in.readString(); - followerIndex = in.readString(); - maxBatchOperationCount = in.readVInt(); - maxConcurrentReadBatches = in.readVInt(); - maxOperationSizeInBytes = in.readVLong(); - maxConcurrentWriteBatches = in.readVInt(); - maxWriteBufferSize = in.readVInt(); - retryTimeout = in.readOptionalTimeValue(); - idleShardRetryDelay = in.readOptionalTimeValue(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeString(leaderIndex); - out.writeString(followerIndex); - out.writeVInt(maxBatchOperationCount); - out.writeVInt(maxConcurrentReadBatches); - out.writeVLong(maxOperationSizeInBytes); - out.writeVInt(maxConcurrentWriteBatches); - out.writeVInt(maxWriteBufferSize); - out.writeOptionalTimeValue(retryTimeout); - out.writeOptionalTimeValue(idleShardRetryDelay); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - { - builder.field(LEADER_INDEX_FIELD.getPreferredName(), leaderIndex); - builder.field(FOLLOWER_INDEX_FIELD.getPreferredName(), followerIndex); - builder.field(ShardFollowTask.MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); - builder.field(ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); - builder.field(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize); - builder.field(ShardFollowTask.MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); - builder.field(ShardFollowTask.MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); - builder.field(ShardFollowTask.RETRY_TIMEOUT.getPreferredName(), retryTimeout.getStringRep()); - 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 maxBatchOperationCount == request.maxBatchOperationCount && - maxConcurrentReadBatches == request.maxConcurrentReadBatches && - maxOperationSizeInBytes == request.maxOperationSizeInBytes && - maxConcurrentWriteBatches == request.maxConcurrentWriteBatches && - maxWriteBufferSize == request.maxWriteBufferSize && - Objects.equals(retryTimeout, request.retryTimeout) && - Objects.equals(idleShardRetryDelay, request.idleShardRetryDelay) && - Objects.equals(leaderIndex, request.leaderIndex) && - Objects.equals(followerIndex, request.followerIndex); - } - - @Override - public int hashCode() { - return Objects.hash( - leaderIndex, - followerIndex, - maxBatchOperationCount, - maxConcurrentReadBatches, - maxOperationSizeInBytes, - maxConcurrentWriteBatches, - maxWriteBufferSize, - retryTimeout, - idleShardRetryDelay - ); - } - } - - public static class TransportAction extends HandledTransportAction { - - private final Client client; - private final ThreadPool threadPool; - private final ClusterService clusterService; - private final RemoteClusterService remoteClusterService; - private final PersistentTasksService persistentTasksService; - private final IndicesService indicesService; - private final CcrLicenseChecker ccrLicenseChecker; - - @Inject - public TransportAction( - final Settings settings, - final ThreadPool threadPool, - final TransportService transportService, - final ActionFilters actionFilters, - final Client client, - final ClusterService clusterService, - final PersistentTasksService persistentTasksService, - final IndicesService indicesService, - final CcrLicenseChecker ccrLicenseChecker) { - super(settings, NAME, transportService, actionFilters, Request::new); - this.client = client; - this.threadPool = threadPool; - this.clusterService = clusterService; - this.remoteClusterService = transportService.getRemoteClusterService(); - this.persistentTasksService = persistentTasksService; - this.indicesService = indicesService; - this.ccrLicenseChecker = Objects.requireNonNull(ccrLicenseChecker); - } - - @Override - protected void doExecute(final Task task, - final Request request, - final ActionListener listener) { - if (ccrLicenseChecker.isCcrAllowed() == false) { - listener.onFailure(LicenseUtils.newComplianceException("ccr")); - return; - } - final String[] indices = new String[]{request.leaderIndex}; - final Map> remoteClusterIndices = remoteClusterService.groupClusterIndices(indices, s -> false); - if (remoteClusterIndices.containsKey(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)) { - followLocalIndex(request, listener); - } else { - assert remoteClusterIndices.size() == 1; - final Map.Entry> entry = remoteClusterIndices.entrySet().iterator().next(); - assert entry.getValue().size() == 1; - final String clusterAlias = entry.getKey(); - final String leaderIndex = entry.getValue().get(0); - followRemoteIndex(request, clusterAlias, leaderIndex, listener); - } - } - - private void followLocalIndex(final Request request, - final ActionListener listener) { - final ClusterState state = clusterService.state(); - final IndexMetaData followerIndexMetadata = state.getMetaData().index(request.getFollowerIndex()); - // following an index in local cluster, so use local cluster state to fetch leader index metadata - final IndexMetaData leaderIndexMetadata = state.getMetaData().index(request.getLeaderIndex()); - try { - start(request, null, leaderIndexMetadata, followerIndexMetadata, listener); - } catch (final IOException e) { - listener.onFailure(e); - } - } - - private void followRemoteIndex( - final Request request, - final String clusterAlias, - final String leaderIndex, - final ActionListener listener) { - final ClusterState state = clusterService.state(); - final IndexMetaData followerIndexMetadata = state.getMetaData().index(request.getFollowerIndex()); - ccrLicenseChecker.checkRemoteClusterLicenseAndFetchLeaderIndexMetadata( - client, - clusterAlias, - leaderIndex, - listener::onFailure, - leaderIndexMetadata -> { - try { - start(request, clusterAlias, leaderIndexMetadata, followerIndexMetadata, listener); - } catch (final IOException e) { - listener.onFailure(e); - } - }); - } - - /** - * Performs validation on the provided leader and follow {@link IndexMetaData} instances and then - * creates a persistent task for each leader primary shard. This persistent tasks track changes in the leader - * shard and replicate these changes to a follower shard. - * - * Currently the following validation is performed: - *
    - *
  • The leader index and follow index need to have the same number of primary shards
  • - *
- */ - void start( - Request request, - String clusterNameAlias, - IndexMetaData leaderIndexMetadata, - IndexMetaData followIndexMetadata, - ActionListener handler) throws IOException { - - MapperService mapperService = followIndexMetadata != null ? indicesService.createIndexMapperService(followIndexMetadata) : null; - validate(request, leaderIndexMetadata, followIndexMetadata, mapperService); - final int numShards = followIndexMetadata.getNumberOfShards(); - final AtomicInteger counter = new AtomicInteger(numShards); - final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); - Map filteredHeaders = threadPool.getThreadContext().getHeaders().entrySet().stream() - .filter(e -> ShardFollowTask.HEADER_FILTERS.contains(e.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));for (int i = 0; i < numShards; i++) { - final int shardId = i; - String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; - - ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, - new ShardId(followIndexMetadata.getIndex(), shardId), - new ShardId(leaderIndexMetadata.getIndex(), shardId), - request.maxBatchOperationCount, request.maxConcurrentReadBatches, request.maxOperationSizeInBytes, - request.maxConcurrentWriteBatches, request.maxWriteBufferSize, request.retryTimeout, - request.idleShardRetryDelay, filteredHeaders); - persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, - new ActionListener>() { - @Override - public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { - responses.set(shardId, task); - finalizeResponse(); - } - - @Override - public void onFailure(Exception e) { - responses.set(shardId, e); - finalizeResponse(); - } - - void finalizeResponse() { - Exception error = null; - if (counter.decrementAndGet() == 0) { - for (int j = 0; j < responses.length(); j++) { - Object response = responses.get(j); - if (response instanceof Exception) { - if (error == null) { - error = (Exception) response; - } else { - error.addSuppressed((Throwable) response); - } - } - } - - if (error == null) { - // include task ids? - handler.onResponse(new AcknowledgedResponse(true)); - } else { - // TODO: cancel all started tasks - handler.onFailure(error); - } - } - } - } - ); - } - } - } - - private static final Set> WHITELISTED_SETTINGS; - - static { - Set> whiteListedSettings = new HashSet<>(); - whiteListedSettings.add(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING); - whiteListedSettings.add(IndexMetaData.INDEX_AUTO_EXPAND_REPLICAS_SETTING); - - whiteListedSettings.add(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING); - whiteListedSettings.add(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING); - whiteListedSettings.add(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING); - whiteListedSettings.add(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING); - whiteListedSettings.add(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING); - whiteListedSettings.add(ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING); - - whiteListedSettings.add(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING); - whiteListedSettings.add(IndexSettings.MAX_RESULT_WINDOW_SETTING); - whiteListedSettings.add(IndexSettings.INDEX_WARMER_ENABLED_SETTING); - whiteListedSettings.add(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING); - whiteListedSettings.add(IndexSettings.MAX_RESCORE_WINDOW_SETTING); - whiteListedSettings.add(IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING); - whiteListedSettings.add(IndexSettings.DEFAULT_FIELD_SETTING); - whiteListedSettings.add(IndexSettings.QUERY_STRING_LENIENT_SETTING); - whiteListedSettings.add(IndexSettings.QUERY_STRING_ANALYZE_WILDCARD); - whiteListedSettings.add(IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD); - whiteListedSettings.add(IndexSettings.ALLOW_UNMAPPED); - whiteListedSettings.add(IndexSettings.INDEX_SEARCH_IDLE_AFTER); - whiteListedSettings.add(BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING); - - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE_SETTING); - whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL); - whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN_SETTING); - whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG_SETTING); - whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO_SETTING); - whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE_SETTING); - whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL_SETTING); - whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING); - whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG_SETTING); - - whiteListedSettings.add(IndexSettings.INDEX_SOFT_DELETES_SETTING); - whiteListedSettings.add(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); - - WHITELISTED_SETTINGS = Collections.unmodifiableSet(whiteListedSettings); - } - - static void validate(Request request, - IndexMetaData leaderIndex, - IndexMetaData followIndex, MapperService followerMapperService) { - if (leaderIndex == null) { - throw new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not exist"); - } - if (followIndex == null) { - throw new IllegalArgumentException("follow index [" + request.followerIndex + "] does not exist"); - } - if (leaderIndex.getSettings().getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false) == false) { - throw new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not have soft deletes enabled"); - } - if (leaderIndex.getNumberOfShards() != followIndex.getNumberOfShards()) { - throw new IllegalArgumentException("leader index primary shards [" + leaderIndex.getNumberOfShards() + - "] does not match with the number of shards of the follow index [" + followIndex.getNumberOfShards() + "]"); - } - if (leaderIndex.getRoutingNumShards() != followIndex.getRoutingNumShards()) { - throw new IllegalArgumentException("leader index number_of_routing_shards [" + leaderIndex.getRoutingNumShards() + - "] does not match with the number_of_routing_shards of the follow index [" + followIndex.getRoutingNumShards() + "]"); - } - if (leaderIndex.getState() != IndexMetaData.State.OPEN || followIndex.getState() != IndexMetaData.State.OPEN) { - throw new IllegalArgumentException("leader and follow index must be open"); - } - if (CcrSettings.CCR_FOLLOWING_INDEX_SETTING.get(followIndex.getSettings()) == false) { - throw new IllegalArgumentException("the following index [" + request.followerIndex + "] is not ready " + - "to follow; the setting [" + CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey() + "] must be enabled."); - } - // Make a copy, remove settings that are allowed to be different and then compare if the settings are equal. - Settings leaderSettings = filter(leaderIndex.getSettings()); - Settings followerSettings = filter(followIndex.getSettings()); - if (leaderSettings.equals(followerSettings) == false) { - throw new IllegalArgumentException("the leader and follower index settings must be identical"); - } - - // Validates if the current follower mapping is mergable with the leader mapping. - // This also validates for example whether specific mapper plugins have been installed - followerMapperService.merge(leaderIndex, MapperService.MergeReason.MAPPING_RECOVERY); - } - - private static Settings filter(Settings originalSettings) { - Settings.Builder settings = Settings.builder().put(originalSettings); - // Remove settings that are always going to be different between leader and follow index: - settings.remove(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey()); - settings.remove(IndexMetaData.SETTING_INDEX_UUID); - settings.remove(IndexMetaData.SETTING_INDEX_PROVIDED_NAME); - settings.remove(IndexMetaData.SETTING_CREATION_DATE); - - Iterator iterator = settings.keys().iterator(); - while (iterator.hasNext()) { - String key = iterator.next(); - for (Setting whitelistedSetting : WHITELISTED_SETTINGS) { - if (whitelistedSetting.match(key)) { - iterator.remove(); - break; - } - } - } - return settings.build(); - } - -} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java index d102c6b5b7af8..b6f82783a56ab 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; import java.io.IOException; import java.util.ArrayList; @@ -57,7 +58,7 @@ public static class Request extends SingleShardRequest { private long fromSeqNo; private int maxOperationCount; private ShardId shardId; - private long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_BATCH_SIZE_IN_BYTES; + private long maxOperationSizeInBytes = FollowIndexAction.DEFAULT_MAX_BATCH_SIZE_IN_BYTES; public Request(ShardId shardId) { super(shardId.getIndexName()); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java index 00e3aaaae2a8e..0a0a6877dc92a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java @@ -10,35 +10,23 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.support.TransportActions; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.transport.NetworkExceptionHelper; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.persistent.AllocatedPersistentTask; -import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; +import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; -import java.io.IOException; -import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.NavigableMap; -import java.util.Objects; import java.util.PriorityQueue; import java.util.Queue; import java.util.TreeMap; @@ -48,7 +36,6 @@ import java.util.function.Consumer; import java.util.function.LongConsumer; import java.util.function.LongSupplier; -import java.util.stream.Collectors; /** * The node task that fetch the write operations from a leader shard and @@ -56,15 +43,6 @@ */ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { - public static final int DEFAULT_MAX_BATCH_OPERATION_COUNT = 1024; - public static final int DEFAULT_MAX_CONCURRENT_READ_BATCHES = 1; - public static final int DEFAULT_MAX_CONCURRENT_WRITE_BATCHES = 1; - public static final int DEFAULT_MAX_WRITE_BUFFER_SIZE = 10240; - public static final long DEFAULT_MAX_BATCH_SIZE_IN_BYTES = Long.MAX_VALUE; - private static final int RETRY_LIMIT = 10; - public static final TimeValue DEFAULT_RETRY_TIMEOUT = new TimeValue(500); - public static final TimeValue DEFAULT_IDLE_SHARD_RETRY_DELAY = TimeValue.timeValueSeconds(10); - private static final Logger LOGGER = Loggers.getLogger(ShardFollowNodeTask.class); private final String leaderIndex; @@ -380,7 +358,7 @@ private void updateMapping(LongConsumer handler, AtomicInteger retryCounter) { private void handleFailure(Exception e, AtomicInteger retryCounter, Runnable task) { assert e != null; if (shouldRetry(e)) { - if (isStopped() == false && retryCounter.incrementAndGet() <= RETRY_LIMIT) { + if (isStopped() == false && retryCounter.incrementAndGet() <= FollowIndexAction.RETRY_LIMIT) { LOGGER.debug(new ParameterizedMessage("{} error during follow shard task, retrying...", params.getFollowShardId()), e); scheduler.accept(retryTimeout, task); } else { @@ -421,7 +399,7 @@ public ShardId getFollowShardId() { } @Override - public synchronized Status getStatus() { + public synchronized ShardFollowNodeTaskStatus getStatus() { final long timeSinceLastFetchMillis; if (lastFetchTime != -1) { timeSinceLastFetchMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - lastFetchTime); @@ -429,7 +407,7 @@ public synchronized Status getStatus() { // To avoid confusion when ccr didn't yet execute a fetch: timeSinceLastFetchMillis = -1; } - return new Status( + return new ShardFollowNodeTaskStatus( leaderIndex, getFollowShardId().getId(), leaderGlobalCheckpoint, @@ -454,476 +432,4 @@ public synchronized Status getStatus() { timeSinceLastFetchMillis); } - public static class Status implements Task.Status { - - public static final String STATUS_PARSER_NAME = "shard-follow-node-task-status"; - - static final ParseField LEADER_INDEX = new ParseField("leader_index"); - static final ParseField SHARD_ID = new ParseField("shard_id"); - static final ParseField LEADER_GLOBAL_CHECKPOINT_FIELD = new ParseField("leader_global_checkpoint"); - static final ParseField LEADER_MAX_SEQ_NO_FIELD = new ParseField("leader_max_seq_no"); - static final ParseField FOLLOWER_GLOBAL_CHECKPOINT_FIELD = new ParseField("follower_global_checkpoint"); - static final ParseField FOLLOWER_MAX_SEQ_NO_FIELD = new ParseField("follower_max_seq_no"); - static final ParseField LAST_REQUESTED_SEQ_NO_FIELD = new ParseField("last_requested_seq_no"); - static final ParseField NUMBER_OF_CONCURRENT_READS_FIELD = new ParseField("number_of_concurrent_reads"); - static final ParseField NUMBER_OF_CONCURRENT_WRITES_FIELD = new ParseField("number_of_concurrent_writes"); - static final ParseField NUMBER_OF_QUEUED_WRITES_FIELD = new ParseField("number_of_queued_writes"); - static final ParseField MAPPING_VERSION_FIELD = new ParseField("mapping_version"); - static final ParseField TOTAL_FETCH_TIME_MILLIS_FIELD = new ParseField("total_fetch_time_millis"); - static final ParseField NUMBER_OF_SUCCESSFUL_FETCHES_FIELD = new ParseField("number_of_successful_fetches"); - static final ParseField NUMBER_OF_FAILED_FETCHES_FIELD = new ParseField("number_of_failed_fetches"); - static final ParseField OPERATIONS_RECEIVED_FIELD = new ParseField("operations_received"); - static final ParseField TOTAL_TRANSFERRED_BYTES = new ParseField("total_transferred_bytes"); - static final ParseField TOTAL_INDEX_TIME_MILLIS_FIELD = new ParseField("total_index_time_millis"); - static final ParseField NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD = new ParseField("number_of_successful_bulk_operations"); - static final ParseField NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD = new ParseField("number_of_failed_bulk_operations"); - static final ParseField NUMBER_OF_OPERATIONS_INDEXED_FIELD = new ParseField("number_of_operations_indexed"); - static final ParseField FETCH_EXCEPTIONS = new ParseField("fetch_exceptions"); - static final ParseField TIME_SINCE_LAST_FETCH_MILLIS_FIELD = new ParseField("time_since_last_fetch_millis"); - - @SuppressWarnings("unchecked") - static final ConstructingObjectParser STATUS_PARSER = new ConstructingObjectParser<>(STATUS_PARSER_NAME, - args -> new Status( - (String) args[0], - (int) args[1], - (long) args[2], - (long) args[3], - (long) args[4], - (long) args[5], - (long) args[6], - (int) args[7], - (int) args[8], - (int) args[9], - (long) args[10], - (long) args[11], - (long) args[12], - (long) args[13], - (long) args[14], - (long) args[15], - (long) args[16], - (long) args[17], - (long) args[18], - (long) args[19], - new TreeMap<>( - ((List>) args[20]) - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))), - (long) args[21])); - - public static final String FETCH_EXCEPTIONS_ENTRY_PARSER_NAME = "shard-follow-node-task-status-fetch-exceptions-entry"; - - static final ConstructingObjectParser, Void> FETCH_EXCEPTIONS_ENTRY_PARSER = - new ConstructingObjectParser<>( - FETCH_EXCEPTIONS_ENTRY_PARSER_NAME, - args -> new AbstractMap.SimpleEntry<>((long) args[0], (ElasticsearchException) args[1])); - - static { - STATUS_PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_INDEX); - STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), SHARD_ID); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), LEADER_GLOBAL_CHECKPOINT_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), LEADER_MAX_SEQ_NO_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_GLOBAL_CHECKPOINT_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_MAX_SEQ_NO_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), LAST_REQUESTED_SEQ_NO_FIELD); - STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_READS_FIELD); - STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_WRITES_FIELD); - STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_QUEUED_WRITES_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAPPING_VERSION_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_FETCH_TIME_MILLIS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_SUCCESSFUL_FETCHES_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_FAILED_FETCHES_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), OPERATIONS_RECEIVED_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_TRANSFERRED_BYTES); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_INDEX_TIME_MILLIS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_OPERATIONS_INDEXED_FIELD); - STATUS_PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), FETCH_EXCEPTIONS_ENTRY_PARSER, FETCH_EXCEPTIONS); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TIME_SINCE_LAST_FETCH_MILLIS_FIELD); - } - - static final ParseField FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO = new ParseField("from_seq_no"); - static final ParseField FETCH_EXCEPTIONS_ENTRY_EXCEPTION = new ParseField("exception"); - - static { - FETCH_EXCEPTIONS_ENTRY_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO); - FETCH_EXCEPTIONS_ENTRY_PARSER.declareObject( - ConstructingObjectParser.constructorArg(), - (p, c) -> ElasticsearchException.fromXContent(p), - FETCH_EXCEPTIONS_ENTRY_EXCEPTION); - } - - private final String leaderIndex; - - public String leaderIndex() { - return leaderIndex; - } - - private final int shardId; - - public int getShardId() { - return shardId; - } - - private final long leaderGlobalCheckpoint; - - public long leaderGlobalCheckpoint() { - return leaderGlobalCheckpoint; - } - - private final long leaderMaxSeqNo; - - public long leaderMaxSeqNo() { - return leaderMaxSeqNo; - } - - private final long followerGlobalCheckpoint; - - public long followerGlobalCheckpoint() { - return followerGlobalCheckpoint; - } - - private final long followerMaxSeqNo; - - public long followerMaxSeqNo() { - return followerMaxSeqNo; - } - - private final long lastRequestedSeqNo; - - public long lastRequestedSeqNo() { - return lastRequestedSeqNo; - } - - private final int numberOfConcurrentReads; - - public int numberOfConcurrentReads() { - return numberOfConcurrentReads; - } - - private final int numberOfConcurrentWrites; - - public int numberOfConcurrentWrites() { - return numberOfConcurrentWrites; - } - - private final int numberOfQueuedWrites; - - public int numberOfQueuedWrites() { - return numberOfQueuedWrites; - } - - private final long mappingVersion; - - public long mappingVersion() { - return mappingVersion; - } - - private final long totalFetchTimeMillis; - - public long totalFetchTimeMillis() { - return totalFetchTimeMillis; - } - - private final long numberOfSuccessfulFetches; - - public long numberOfSuccessfulFetches() { - return numberOfSuccessfulFetches; - } - - private final long numberOfFailedFetches; - - public long numberOfFailedFetches() { - return numberOfFailedFetches; - } - - private final long operationsReceived; - - public long operationsReceived() { - return operationsReceived; - } - - private final long totalTransferredBytes; - - public long totalTransferredBytes() { - return totalTransferredBytes; - } - - private final long totalIndexTimeMillis; - - public long totalIndexTimeMillis() { - return totalIndexTimeMillis; - } - - private final long numberOfSuccessfulBulkOperations; - - public long numberOfSuccessfulBulkOperations() { - return numberOfSuccessfulBulkOperations; - } - - private final long numberOfFailedBulkOperations; - - public long numberOfFailedBulkOperations() { - return numberOfFailedBulkOperations; - } - - private final long numberOfOperationsIndexed; - - public long numberOfOperationsIndexed() { - return numberOfOperationsIndexed; - } - - private final NavigableMap fetchExceptions; - - public NavigableMap fetchExceptions() { - return fetchExceptions; - } - - private final long timeSinceLastFetchMillis; - - public long timeSinceLastFetchMillis() { - return timeSinceLastFetchMillis; - } - - Status( - final String leaderIndex, - final int shardId, - final long leaderGlobalCheckpoint, - final long leaderMaxSeqNo, - final long followerGlobalCheckpoint, - final long followerMaxSeqNo, - final long lastRequestedSeqNo, - final int numberOfConcurrentReads, - final int numberOfConcurrentWrites, - final int numberOfQueuedWrites, - final long mappingVersion, - final long totalFetchTimeMillis, - final long numberOfSuccessfulFetches, - final long numberOfFailedFetches, - final long operationsReceived, - final long totalTransferredBytes, - final long totalIndexTimeMillis, - final long numberOfSuccessfulBulkOperations, - final long numberOfFailedBulkOperations, - final long numberOfOperationsIndexed, - final NavigableMap fetchExceptions, - final long timeSinceLastFetchMillis) { - this.leaderIndex = leaderIndex; - this.shardId = shardId; - this.leaderGlobalCheckpoint = leaderGlobalCheckpoint; - this.leaderMaxSeqNo = leaderMaxSeqNo; - this.followerGlobalCheckpoint = followerGlobalCheckpoint; - this.followerMaxSeqNo = followerMaxSeqNo; - this.lastRequestedSeqNo = lastRequestedSeqNo; - this.numberOfConcurrentReads = numberOfConcurrentReads; - this.numberOfConcurrentWrites = numberOfConcurrentWrites; - this.numberOfQueuedWrites = numberOfQueuedWrites; - this.mappingVersion = mappingVersion; - this.totalFetchTimeMillis = totalFetchTimeMillis; - this.numberOfSuccessfulFetches = numberOfSuccessfulFetches; - this.numberOfFailedFetches = numberOfFailedFetches; - this.operationsReceived = operationsReceived; - this.totalTransferredBytes = totalTransferredBytes; - this.totalIndexTimeMillis = totalIndexTimeMillis; - this.numberOfSuccessfulBulkOperations = numberOfSuccessfulBulkOperations; - this.numberOfFailedBulkOperations = numberOfFailedBulkOperations; - this.numberOfOperationsIndexed = numberOfOperationsIndexed; - this.fetchExceptions = Objects.requireNonNull(fetchExceptions); - this.timeSinceLastFetchMillis = timeSinceLastFetchMillis; - } - - public Status(final StreamInput in) throws IOException { - this.leaderIndex = in.readString(); - this.shardId = in.readVInt(); - this.leaderGlobalCheckpoint = in.readZLong(); - this.leaderMaxSeqNo = in.readZLong(); - this.followerGlobalCheckpoint = in.readZLong(); - this.followerMaxSeqNo = in.readZLong(); - this.lastRequestedSeqNo = in.readZLong(); - this.numberOfConcurrentReads = in.readVInt(); - this.numberOfConcurrentWrites = in.readVInt(); - this.numberOfQueuedWrites = in.readVInt(); - this.mappingVersion = in.readVLong(); - this.totalFetchTimeMillis = in.readVLong(); - this.numberOfSuccessfulFetches = in.readVLong(); - this.numberOfFailedFetches = in.readVLong(); - this.operationsReceived = in.readVLong(); - this.totalTransferredBytes = in.readVLong(); - this.totalIndexTimeMillis = in.readVLong(); - this.numberOfSuccessfulBulkOperations = in.readVLong(); - this.numberOfFailedBulkOperations = in.readVLong(); - this.numberOfOperationsIndexed = in.readVLong(); - this.fetchExceptions = new TreeMap<>(in.readMap(StreamInput::readVLong, StreamInput::readException)); - this.timeSinceLastFetchMillis = in.readZLong(); - } - - @Override - public String getWriteableName() { - return STATUS_PARSER_NAME; - } - - @Override - public void writeTo(final StreamOutput out) throws IOException { - out.writeString(leaderIndex); - out.writeVInt(shardId); - out.writeZLong(leaderGlobalCheckpoint); - out.writeZLong(leaderMaxSeqNo); - out.writeZLong(followerGlobalCheckpoint); - out.writeZLong(followerMaxSeqNo); - out.writeZLong(lastRequestedSeqNo); - out.writeVInt(numberOfConcurrentReads); - out.writeVInt(numberOfConcurrentWrites); - out.writeVInt(numberOfQueuedWrites); - out.writeVLong(mappingVersion); - out.writeVLong(totalFetchTimeMillis); - out.writeVLong(numberOfSuccessfulFetches); - out.writeVLong(numberOfFailedFetches); - out.writeVLong(operationsReceived); - out.writeVLong(totalTransferredBytes); - out.writeVLong(totalIndexTimeMillis); - out.writeVLong(numberOfSuccessfulBulkOperations); - out.writeVLong(numberOfFailedBulkOperations); - out.writeVLong(numberOfOperationsIndexed); - out.writeMap(fetchExceptions, StreamOutput::writeVLong, StreamOutput::writeException); - out.writeZLong(timeSinceLastFetchMillis); - } - - @Override - public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException { - builder.startObject(); - { - builder.field(LEADER_INDEX.getPreferredName(), leaderIndex); - builder.field(SHARD_ID.getPreferredName(), shardId); - builder.field(LEADER_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), leaderGlobalCheckpoint); - builder.field(LEADER_MAX_SEQ_NO_FIELD.getPreferredName(), leaderMaxSeqNo); - builder.field(FOLLOWER_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), followerGlobalCheckpoint); - builder.field(FOLLOWER_MAX_SEQ_NO_FIELD.getPreferredName(), followerMaxSeqNo); - builder.field(LAST_REQUESTED_SEQ_NO_FIELD.getPreferredName(), lastRequestedSeqNo); - builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads); - builder.field(NUMBER_OF_CONCURRENT_WRITES_FIELD.getPreferredName(), numberOfConcurrentWrites); - builder.field(NUMBER_OF_QUEUED_WRITES_FIELD.getPreferredName(), numberOfQueuedWrites); - builder.field(MAPPING_VERSION_FIELD.getPreferredName(), mappingVersion); - builder.humanReadableField( - TOTAL_FETCH_TIME_MILLIS_FIELD.getPreferredName(), - "total_fetch_time", - new TimeValue(totalFetchTimeMillis, TimeUnit.MILLISECONDS)); - builder.field(NUMBER_OF_SUCCESSFUL_FETCHES_FIELD.getPreferredName(), numberOfSuccessfulFetches); - builder.field(NUMBER_OF_FAILED_FETCHES_FIELD.getPreferredName(), numberOfFailedFetches); - builder.field(OPERATIONS_RECEIVED_FIELD.getPreferredName(), operationsReceived); - builder.humanReadableField( - TOTAL_TRANSFERRED_BYTES.getPreferredName(), - "total_transferred", - new ByteSizeValue(totalTransferredBytes, ByteSizeUnit.BYTES)); - builder.humanReadableField( - TOTAL_INDEX_TIME_MILLIS_FIELD.getPreferredName(), - "total_index_time", - new TimeValue(totalIndexTimeMillis, TimeUnit.MILLISECONDS)); - builder.field(NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD.getPreferredName(), numberOfSuccessfulBulkOperations); - builder.field(NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD.getPreferredName(), numberOfFailedBulkOperations); - builder.field(NUMBER_OF_OPERATIONS_INDEXED_FIELD.getPreferredName(), numberOfOperationsIndexed); - builder.startArray(FETCH_EXCEPTIONS.getPreferredName()); - { - for (final Map.Entry entry : fetchExceptions.entrySet()) { - builder.startObject(); - { - builder.field(FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO.getPreferredName(), entry.getKey()); - builder.field(FETCH_EXCEPTIONS_ENTRY_EXCEPTION.getPreferredName()); - builder.startObject(); - { - ElasticsearchException.generateThrowableXContent(builder, params, entry.getValue()); - } - builder.endObject(); - } - builder.endObject(); - } - } - builder.endArray(); - builder.humanReadableField( - TIME_SINCE_LAST_FETCH_MILLIS_FIELD.getPreferredName(), - "time_since_last_fetch", - new TimeValue(timeSinceLastFetchMillis, TimeUnit.MILLISECONDS)); - } - builder.endObject(); - return builder; - } - - public static Status fromXContent(final XContentParser parser) { - return STATUS_PARSER.apply(parser, null); - } - - @Override - public boolean equals(final Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - final Status that = (Status) o; - return leaderIndex.equals(that.leaderIndex) && - shardId == that.shardId && - leaderGlobalCheckpoint == that.leaderGlobalCheckpoint && - leaderMaxSeqNo == that.leaderMaxSeqNo && - followerGlobalCheckpoint == that.followerGlobalCheckpoint && - followerMaxSeqNo == that.followerMaxSeqNo && - lastRequestedSeqNo == that.lastRequestedSeqNo && - numberOfConcurrentReads == that.numberOfConcurrentReads && - numberOfConcurrentWrites == that.numberOfConcurrentWrites && - numberOfQueuedWrites == that.numberOfQueuedWrites && - mappingVersion == that.mappingVersion && - totalFetchTimeMillis == that.totalFetchTimeMillis && - numberOfSuccessfulFetches == that.numberOfSuccessfulFetches && - numberOfFailedFetches == that.numberOfFailedFetches && - operationsReceived == that.operationsReceived && - totalTransferredBytes == that.totalTransferredBytes && - numberOfSuccessfulBulkOperations == that.numberOfSuccessfulBulkOperations && - numberOfFailedBulkOperations == that.numberOfFailedBulkOperations && - numberOfOperationsIndexed == that.numberOfOperationsIndexed && - /* - * ElasticsearchException does not implement equals so we will assume the fetch exceptions are equal if they are equal - * up to the key set and their messages. Note that we are relying on the fact that the fetch exceptions are ordered by - * keys. - */ - fetchExceptions.keySet().equals(that.fetchExceptions.keySet()) && - getFetchExceptionMessages(this).equals(getFetchExceptionMessages(that)) && - timeSinceLastFetchMillis == that.timeSinceLastFetchMillis; - } - - @Override - public int hashCode() { - return Objects.hash( - leaderIndex, - shardId, - leaderGlobalCheckpoint, - leaderMaxSeqNo, - followerGlobalCheckpoint, - followerMaxSeqNo, - lastRequestedSeqNo, - numberOfConcurrentReads, - numberOfConcurrentWrites, - numberOfQueuedWrites, - mappingVersion, - totalFetchTimeMillis, - numberOfSuccessfulFetches, - numberOfFailedFetches, - operationsReceived, - totalTransferredBytes, - numberOfSuccessfulBulkOperations, - numberOfFailedBulkOperations, - numberOfOperationsIndexed, - /* - * ElasticsearchException does not implement hash code so we will compute the hash code based on the key set and the - * messages. Note that we are relying on the fact that the fetch exceptions are ordered by keys. - */ - fetchExceptions.keySet(), - getFetchExceptionMessages(this), - timeSinceLastFetchMillis); - } - - private static List getFetchExceptionMessages(final Status status) { - return status.fetchExceptions().values().stream().map(ElasticsearchException::getMessage).collect(Collectors.toList()); - } - - public String toString() { - return Strings.toString(this); - } - - } - } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportCcrStatsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportCcrStatsAction.java index 3b5d0ac53cf81..d4425773fa1c8 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportCcrStatsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportCcrStatsAction.java @@ -22,6 +22,7 @@ import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.ccr.Ccr; import org.elasticsearch.xpack.ccr.CcrLicenseChecker; +import org.elasticsearch.xpack.core.ccr.action.CcrStatsAction; import java.io.IOException; import java.util.Arrays; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportCreateAndFollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportCreateAndFollowIndexAction.java new file mode 100644 index 0000000000000..b99b569a525ca --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportCreateAndFollowIndexAction.java @@ -0,0 +1,231 @@ +/* + * 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 com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.ActiveShardCount; +import org.elasticsearch.action.support.ActiveShardsObserver; +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.MappingMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.routing.RoutingTable; +import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.license.LicenseUtils; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.RemoteClusterAware; +import org.elasticsearch.transport.RemoteClusterService; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.ccr.CcrLicenseChecker; +import org.elasticsearch.xpack.ccr.CcrSettings; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public final class TransportCreateAndFollowIndexAction + extends TransportMasterNodeAction { + + private final Client client; + private final AllocationService allocationService; + private final RemoteClusterService remoteClusterService; + private final ActiveShardsObserver activeShardsObserver; + private final CcrLicenseChecker ccrLicenseChecker; + + @Inject + public TransportCreateAndFollowIndexAction( + final Settings settings, + final ThreadPool threadPool, + final TransportService transportService, + final ClusterService clusterService, + final ActionFilters actionFilters, + final IndexNameExpressionResolver indexNameExpressionResolver, + final Client client, + final AllocationService allocationService, + final CcrLicenseChecker ccrLicenseChecker) { + super( + settings, + CreateAndFollowIndexAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + indexNameExpressionResolver, + CreateAndFollowIndexAction.Request::new); + this.client = client; + this.allocationService = allocationService; + this.remoteClusterService = transportService.getRemoteClusterService(); + this.activeShardsObserver = new ActiveShardsObserver(settings, clusterService, threadPool); + this.ccrLicenseChecker = Objects.requireNonNull(ccrLicenseChecker); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected CreateAndFollowIndexAction.Response newResponse() { + return new CreateAndFollowIndexAction.Response(); + } + + @Override + protected void masterOperation( + final CreateAndFollowIndexAction.Request request, + final ClusterState state, + final ActionListener listener) throws Exception { + if (ccrLicenseChecker.isCcrAllowed() == false) { + listener.onFailure(LicenseUtils.newComplianceException("ccr")); + return; + } + final String[] indices = new String[]{request.getFollowRequest().getLeaderIndex()}; + final Map> remoteClusterIndices = remoteClusterService.groupClusterIndices(indices, s -> false); + if (remoteClusterIndices.containsKey(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)) { + createFollowerIndexAndFollowLocalIndex(request, state, listener); + } else { + assert remoteClusterIndices.size() == 1; + final Map.Entry> entry = remoteClusterIndices.entrySet().iterator().next(); + assert entry.getValue().size() == 1; + final String clusterAlias = entry.getKey(); + final String leaderIndex = entry.getValue().get(0); + createFollowerIndexAndFollowRemoteIndex(request, clusterAlias, leaderIndex, listener); + } + } + + private void createFollowerIndexAndFollowLocalIndex( + final CreateAndFollowIndexAction.Request request, + final ClusterState state, + final ActionListener listener) { + // following an index in local cluster, so use local cluster state to fetch leader index metadata + final IndexMetaData leaderIndexMetadata = state.getMetaData().index(request.getFollowRequest().getLeaderIndex()); + createFollowerIndex(leaderIndexMetadata, request, listener); + } + + private void createFollowerIndexAndFollowRemoteIndex( + final CreateAndFollowIndexAction.Request request, + final String clusterAlias, + final String leaderIndex, + final ActionListener listener) { + ccrLicenseChecker.checkRemoteClusterLicenseAndFetchLeaderIndexMetadata( + client, + clusterAlias, + leaderIndex, + listener::onFailure, + leaderIndexMetaData -> createFollowerIndex(leaderIndexMetaData, request, listener)); + } + + private void createFollowerIndex( + final IndexMetaData leaderIndexMetaData, + final CreateAndFollowIndexAction.Request request, + final ActionListener listener) { + if (leaderIndexMetaData == null) { + listener.onFailure(new IllegalArgumentException("leader index [" + request.getFollowRequest().getLeaderIndex() + + "] does not exist")); + return; + } + + ActionListener handler = ActionListener.wrap( + result -> { + if (result) { + initiateFollowing(request, listener); + } else { + listener.onResponse(new CreateAndFollowIndexAction.Response(true, false, false)); + } + }, + listener::onFailure); + // Can't use create index api here, because then index templates can alter the mappings / settings. + // And index templates could introduce settings / mappings that are incompatible with the leader index. + clusterService.submitStateUpdateTask("follow_index_action", new AckedClusterStateUpdateTask(request, handler) { + + @Override + protected Boolean newResponse(final boolean acknowledged) { + return acknowledged; + } + + @Override + public ClusterState execute(final ClusterState currentState) throws Exception { + String followIndex = request.getFollowRequest().getFollowerIndex(); + IndexMetaData currentIndex = currentState.metaData().index(followIndex); + if (currentIndex != null) { + throw new ResourceAlreadyExistsException(currentIndex.getIndex()); + } + + MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); + IndexMetaData.Builder imdBuilder = IndexMetaData.builder(followIndex); + + // Copy all settings, but overwrite a few settings. + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put(leaderIndexMetaData.getSettings()); + // Overwriting UUID here, because otherwise we can't follow indices in the same cluster + settingsBuilder.put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()); + settingsBuilder.put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followIndex); + settingsBuilder.put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); + imdBuilder.settings(settingsBuilder); + + // Copy mappings from leader IMD to follow IMD + for (ObjectObjectCursor cursor : leaderIndexMetaData.getMappings()) { + imdBuilder.putMapping(cursor.value); + } + imdBuilder.setRoutingNumShards(leaderIndexMetaData.getRoutingNumShards()); + IndexMetaData followIMD = imdBuilder.build(); + mdBuilder.put(followIMD, false); + + ClusterState.Builder builder = ClusterState.builder(currentState); + builder.metaData(mdBuilder.build()); + ClusterState updatedState = builder.build(); + + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(updatedState.routingTable()) + .addAsNew(updatedState.metaData().index(request.getFollowRequest().getFollowerIndex())); + updatedState = allocationService.reroute( + ClusterState.builder(updatedState).routingTable(routingTableBuilder.build()).build(), + "follow index [" + request.getFollowRequest().getFollowerIndex() + "] created"); + + logger.info("[{}] creating index, cause [ccr_create_and_follow], shards [{}]/[{}]", + followIndex, followIMD.getNumberOfShards(), followIMD.getNumberOfReplicas()); + + return updatedState; + } + }); + } + + private void initiateFollowing( + final CreateAndFollowIndexAction.Request request, + final ActionListener listener) { + activeShardsObserver.waitForActiveShards(new String[]{request.getFollowRequest().getFollowerIndex()}, + ActiveShardCount.DEFAULT, request.timeout(), result -> { + if (result) { + client.execute(FollowIndexAction.INSTANCE, request.getFollowRequest(), ActionListener.wrap( + r -> listener.onResponse(new CreateAndFollowIndexAction.Response(true, true, r.isAcknowledged())), + listener::onFailure + )); + } else { + listener.onResponse(new CreateAndFollowIndexAction.Response(true, false, false)); + } + }, listener::onFailure); + } + + @Override + protected ClusterBlockException checkBlock(final CreateAndFollowIndexAction.Request request, final ClusterState state) { + return state.blocks().indexBlockedException(ClusterBlockLevel.METADATA_WRITE, request.getFollowRequest().getFollowerIndex()); + } + +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportFollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportFollowIndexAction.java new file mode 100644 index 0000000000000..33447ef420800 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportFollowIndexAction.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.ccr.action; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; +import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.IndexingSlowLog; +import org.elasticsearch.index.SearchSlowLog; +import org.elasticsearch.index.cache.bitset.BitsetFilterCache; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesRequestCache; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.license.LicenseUtils; +import org.elasticsearch.persistent.PersistentTasksCustomMetaData; +import org.elasticsearch.persistent.PersistentTasksService; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.RemoteClusterAware; +import org.elasticsearch.transport.RemoteClusterService; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.ccr.CcrLicenseChecker; +import org.elasticsearch.xpack.ccr.CcrSettings; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.stream.Collectors; + +public class TransportFollowIndexAction extends HandledTransportAction { + + private final Client client; + private final ThreadPool threadPool; + private final ClusterService clusterService; + private final RemoteClusterService remoteClusterService; + private final PersistentTasksService persistentTasksService; + private final IndicesService indicesService; + private final CcrLicenseChecker ccrLicenseChecker; + + @Inject + public TransportFollowIndexAction( + final Settings settings, + final ThreadPool threadPool, + final TransportService transportService, + final ActionFilters actionFilters, + final Client client, + final ClusterService clusterService, + final PersistentTasksService persistentTasksService, + final IndicesService indicesService, + final CcrLicenseChecker ccrLicenseChecker) { + super(settings, FollowIndexAction.NAME, transportService, actionFilters, FollowIndexAction.Request::new); + this.client = client; + this.threadPool = threadPool; + this.clusterService = clusterService; + this.remoteClusterService = transportService.getRemoteClusterService(); + this.persistentTasksService = persistentTasksService; + this.indicesService = indicesService; + this.ccrLicenseChecker = Objects.requireNonNull(ccrLicenseChecker); + } + + @Override + protected void doExecute(final Task task, + final FollowIndexAction.Request request, + final ActionListener listener) { + if (ccrLicenseChecker.isCcrAllowed() == false) { + listener.onFailure(LicenseUtils.newComplianceException("ccr")); + return; + } + final String[] indices = new String[]{request.getLeaderIndex()}; + final Map> remoteClusterIndices = remoteClusterService.groupClusterIndices(indices, s -> false); + if (remoteClusterIndices.containsKey(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)) { + followLocalIndex(request, listener); + } else { + assert remoteClusterIndices.size() == 1; + final Map.Entry> entry = remoteClusterIndices.entrySet().iterator().next(); + assert entry.getValue().size() == 1; + final String clusterAlias = entry.getKey(); + final String leaderIndex = entry.getValue().get(0); + followRemoteIndex(request, clusterAlias, leaderIndex, listener); + } + } + + private void followLocalIndex(final FollowIndexAction.Request request, + final ActionListener listener) { + final ClusterState state = clusterService.state(); + final IndexMetaData followerIndexMetadata = state.getMetaData().index(request.getFollowerIndex()); + // following an index in local cluster, so use local cluster state to fetch leader index metadata + final IndexMetaData leaderIndexMetadata = state.getMetaData().index(request.getLeaderIndex()); + try { + start(request, null, leaderIndexMetadata, followerIndexMetadata, listener); + } catch (final IOException e) { + listener.onFailure(e); + } + } + + private void followRemoteIndex( + final FollowIndexAction.Request request, + final String clusterAlias, + final String leaderIndex, + final ActionListener listener) { + final ClusterState state = clusterService.state(); + final IndexMetaData followerIndexMetadata = state.getMetaData().index(request.getFollowerIndex()); + ccrLicenseChecker.checkRemoteClusterLicenseAndFetchLeaderIndexMetadata( + client, + clusterAlias, + leaderIndex, + listener::onFailure, + leaderIndexMetadata -> { + try { + start(request, clusterAlias, leaderIndexMetadata, followerIndexMetadata, listener); + } catch (final IOException e) { + listener.onFailure(e); + } + }); + } + + /** + * Performs validation on the provided leader and follow {@link IndexMetaData} instances and then + * creates a persistent task for each leader primary shard. This persistent tasks track changes in the leader + * shard and replicate these changes to a follower shard. + * + * Currently the following validation is performed: + *
    + *
  • The leader index and follow index need to have the same number of primary shards
  • + *
+ */ + void start( + FollowIndexAction.Request request, + String clusterNameAlias, + IndexMetaData leaderIndexMetadata, + IndexMetaData followIndexMetadata, + ActionListener handler) throws IOException { + + MapperService mapperService = followIndexMetadata != null ? indicesService.createIndexMapperService(followIndexMetadata) : null; + validate(request, leaderIndexMetadata, followIndexMetadata, mapperService); + final int numShards = followIndexMetadata.getNumberOfShards(); + final AtomicInteger counter = new AtomicInteger(numShards); + final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); + Map filteredHeaders = threadPool.getThreadContext().getHeaders().entrySet().stream() + .filter(e -> ShardFollowTask.HEADER_FILTERS.contains(e.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));for (int i = 0; i < numShards; i++) { + final int shardId = i; + String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; + + ShardFollowTask shardFollowTask = new ShardFollowTask( + clusterNameAlias, + new ShardId(followIndexMetadata.getIndex(), shardId), + new ShardId(leaderIndexMetadata.getIndex(), shardId), + request.getMaxBatchOperationCount(), + request.getMaxConcurrentReadBatches(), + request.getMaxOperationSizeInBytes(), + request.getMaxConcurrentWriteBatches(), + request.getMaxWriteBufferSize(), + request.getRetryTimeout(), + request.getIdleShardRetryDelay(), + filteredHeaders); + persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, + new ActionListener>() { + @Override + public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { + responses.set(shardId, task); + finalizeResponse(); + } + + @Override + public void onFailure(Exception e) { + responses.set(shardId, e); + finalizeResponse(); + } + + void finalizeResponse() { + Exception error = null; + if (counter.decrementAndGet() == 0) { + for (int j = 0; j < responses.length(); j++) { + Object response = responses.get(j); + if (response instanceof Exception) { + if (error == null) { + error = (Exception) response; + } else { + error.addSuppressed((Throwable) response); + } + } + } + + if (error == null) { + // include task ids? + handler.onResponse(new AcknowledgedResponse(true)); + } else { + // TODO: cancel all started tasks + handler.onFailure(error); + } + } + } + } + ); + } + } + + static void validate( + final FollowIndexAction.Request request, + final IndexMetaData leaderIndex, + final IndexMetaData followIndex, + final MapperService followerMapperService) { + if (leaderIndex == null) { + throw new IllegalArgumentException("leader index [" + request.getLeaderIndex() + "] does not exist"); + } + if (followIndex == null) { + throw new IllegalArgumentException("follow index [" + request.getFollowerIndex() + "] does not exist"); + } + if (leaderIndex.getSettings().getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false) == false) { + throw new IllegalArgumentException("leader index [" + request.getLeaderIndex() + "] does not have soft deletes enabled"); + } + if (leaderIndex.getNumberOfShards() != followIndex.getNumberOfShards()) { + throw new IllegalArgumentException("leader index primary shards [" + leaderIndex.getNumberOfShards() + + "] does not match with the number of shards of the follow index [" + followIndex.getNumberOfShards() + "]"); + } + if (leaderIndex.getRoutingNumShards() != followIndex.getRoutingNumShards()) { + throw new IllegalArgumentException("leader index number_of_routing_shards [" + leaderIndex.getRoutingNumShards() + + "] does not match with the number_of_routing_shards of the follow index [" + followIndex.getRoutingNumShards() + "]"); + } + if (leaderIndex.getState() != IndexMetaData.State.OPEN || followIndex.getState() != IndexMetaData.State.OPEN) { + throw new IllegalArgumentException("leader and follow index must be open"); + } + if (CcrSettings.CCR_FOLLOWING_INDEX_SETTING.get(followIndex.getSettings()) == false) { + throw new IllegalArgumentException("the following index [" + request.getFollowerIndex() + "] is not ready " + + "to follow; the setting [" + CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey() + "] must be enabled."); + } + // Make a copy, remove settings that are allowed to be different and then compare if the settings are equal. + Settings leaderSettings = filter(leaderIndex.getSettings()); + Settings followerSettings = filter(followIndex.getSettings()); + if (leaderSettings.equals(followerSettings) == false) { + throw new IllegalArgumentException("the leader and follower index settings must be identical"); + } + + // Validates if the current follower mapping is mergable with the leader mapping. + // This also validates for example whether specific mapper plugins have been installed + followerMapperService.merge(leaderIndex, MapperService.MergeReason.MAPPING_RECOVERY); + } + + private static final Set> WHITE_LISTED_SETTINGS; + + static { + final Set> whiteListedSettings = new HashSet<>(); + whiteListedSettings.add(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING); + whiteListedSettings.add(IndexMetaData.INDEX_AUTO_EXPAND_REPLICAS_SETTING); + + whiteListedSettings.add(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING); + whiteListedSettings.add(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING); + whiteListedSettings.add(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING); + whiteListedSettings.add(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING); + whiteListedSettings.add(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING); + whiteListedSettings.add(ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING); + + whiteListedSettings.add(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING); + whiteListedSettings.add(IndexSettings.MAX_RESULT_WINDOW_SETTING); + whiteListedSettings.add(IndexSettings.INDEX_WARMER_ENABLED_SETTING); + whiteListedSettings.add(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING); + whiteListedSettings.add(IndexSettings.MAX_RESCORE_WINDOW_SETTING); + whiteListedSettings.add(IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING); + whiteListedSettings.add(IndexSettings.DEFAULT_FIELD_SETTING); + whiteListedSettings.add(IndexSettings.QUERY_STRING_LENIENT_SETTING); + whiteListedSettings.add(IndexSettings.QUERY_STRING_ANALYZE_WILDCARD); + whiteListedSettings.add(IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD); + whiteListedSettings.add(IndexSettings.ALLOW_UNMAPPED); + whiteListedSettings.add(IndexSettings.INDEX_SEARCH_IDLE_AFTER); + whiteListedSettings.add(BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING); + + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE_SETTING); + whiteListedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL); + whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN_SETTING); + whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG_SETTING); + whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO_SETTING); + whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE_SETTING); + whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL_SETTING); + whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING); + whiteListedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG_SETTING); + + whiteListedSettings.add(IndexSettings.INDEX_SOFT_DELETES_SETTING); + whiteListedSettings.add(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); + + WHITE_LISTED_SETTINGS = Collections.unmodifiableSet(whiteListedSettings); + } + + private static Settings filter(Settings originalSettings) { + Settings.Builder settings = Settings.builder().put(originalSettings); + // Remove settings that are always going to be different between leader and follow index: + settings.remove(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey()); + settings.remove(IndexMetaData.SETTING_INDEX_UUID); + settings.remove(IndexMetaData.SETTING_INDEX_PROVIDED_NAME); + settings.remove(IndexMetaData.SETTING_CREATION_DATE); + + Iterator iterator = settings.keys().iterator(); + while (iterator.hasNext()) { + String key = iterator.next(); + for (Setting whitelistedSetting : WHITE_LISTED_SETTINGS) { + if (whitelistedSetting.match(key)) { + iterator.remove(); + break; + } + } + } + return settings.build(); + } + +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowIndexAction.java new file mode 100644 index 0000000000000..05cde0eab8523 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowIndexAction.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.action.ActionListener; +import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.persistent.PersistentTasksCustomMetaData; +import org.elasticsearch.persistent.PersistentTasksService; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.ccr.action.UnfollowIndexAction; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceArray; + +public class TransportUnfollowIndexAction extends HandledTransportAction { + + private final Client client; + private final PersistentTasksService persistentTasksService; + + @Inject + public TransportUnfollowIndexAction( + final Settings settings, + final TransportService transportService, + final ActionFilters actionFilters, + final Client client, + final PersistentTasksService persistentTasksService) { + super(settings, UnfollowIndexAction.NAME, transportService, actionFilters, UnfollowIndexAction.Request::new); + this.client = client; + this.persistentTasksService = persistentTasksService; + } + + @Override + protected void doExecute( + final Task task, + final UnfollowIndexAction.Request request, + final ActionListener listener) { + + client.admin().cluster().state(new ClusterStateRequest(), ActionListener.wrap(r -> { + IndexMetaData followIndexMetadata = r.getState().getMetaData().index(request.getFollowIndex()); + if (followIndexMetadata == null) { + listener.onFailure(new IllegalArgumentException("follow index [" + request.getFollowIndex() + "] does not exist")); + return; + } + + final int numShards = followIndexMetadata.getNumberOfShards(); + final AtomicInteger counter = new AtomicInteger(numShards); + final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); + for (int i = 0; i < numShards; i++) { + final int shardId = i; + String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; + persistentTasksService.sendRemoveRequest(taskId, + new ActionListener>() { + @Override + public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { + responses.set(shardId, task); + finalizeResponse(); + } + + @Override + public void onFailure(Exception e) { + responses.set(shardId, e); + finalizeResponse(); + } + + void finalizeResponse() { + Exception error = null; + if (counter.decrementAndGet() == 0) { + for (int j = 0; j < responses.length(); j++) { + Object response = responses.get(j); + if (response instanceof Exception) { + if (error == null) { + error = (Exception) response; + } else { + error.addSuppressed((Throwable) response); + } + } + } + + if (error == null) { + // include task ids? + listener.onResponse(new AcknowledgedResponse(true)); + } else { + // TODO: cancel all started tasks + listener.onFailure(error); + } + } + } + }); + } + }, listener::onFailure)); + } + +} \ No newline at end of file diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/UnfollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/UnfollowIndexAction.java deleted file mode 100644 index 93b2bcc3e4096..0000000000000 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/UnfollowIndexAction.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * 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.ActionListener; -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.HandledTransportAction; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.persistent.PersistentTasksCustomMetaData; -import org.elasticsearch.persistent.PersistentTasksService; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.transport.TransportService; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReferenceArray; - -public class UnfollowIndexAction extends Action { - - public static final UnfollowIndexAction INSTANCE = new UnfollowIndexAction(); - public static final String NAME = "cluster:admin/xpack/ccr/unfollow_index"; - - private UnfollowIndexAction() { - super(NAME); - } - - @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); - } - - public static class Request extends ActionRequest { - - private String followIndex; - - public String getFollowIndex() { - return followIndex; - } - - public void setFollowIndex(String followIndex) { - this.followIndex = followIndex; - } - - @Override - public ActionRequestValidationException validate() { - return null; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - followIndex = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeString(followIndex); - } - } - - public static class TransportAction extends HandledTransportAction { - - private final Client client; - private final PersistentTasksService persistentTasksService; - - @Inject - public TransportAction(Settings settings, - TransportService transportService, - ActionFilters actionFilters, - Client client, - PersistentTasksService persistentTasksService) { - super(settings, NAME, transportService, actionFilters, Request::new); - this.client = client; - this.persistentTasksService = persistentTasksService; - } - - @Override - protected void doExecute(Task task, - Request request, - ActionListener listener) { - - client.admin().cluster().state(new ClusterStateRequest(), ActionListener.wrap(r -> { - IndexMetaData followIndexMetadata = r.getState().getMetaData().index(request.followIndex); - if (followIndexMetadata == null) { - listener.onFailure(new IllegalArgumentException("follow index [" + request.followIndex + "] does not exist")); - return; - } - - final int numShards = followIndexMetadata.getNumberOfShards(); - final AtomicInteger counter = new AtomicInteger(numShards); - final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); - for (int i = 0; i < numShards; i++) { - final int shardId = i; - String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; - persistentTasksService.sendRemoveRequest(taskId, - new ActionListener>() { - @Override - public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { - responses.set(shardId, task); - finalizeResponse(); - } - - @Override - public void onFailure(Exception e) { - responses.set(shardId, e); - finalizeResponse(); - } - - void finalizeResponse() { - Exception error = null; - if (counter.decrementAndGet() == 0) { - for (int j = 0; j < responses.length(); j++) { - Object response = responses.get(j); - if (response instanceof Exception) { - if (error == null) { - error = (Exception) response; - } else { - error.addSuppressed((Throwable) response); - } - } - } - - if (error == null) { - // include task ids? - listener.onResponse(new AcknowledgedResponse(true)); - } else { - // TODO: cancel all started tasks - listener.onFailure(error); - } - } - } - }); - } - }, listener::onFailure)); - } - } - -} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCcrStatsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCcrStatsAction.java index df34fd6cd45b2..0cf0aaf2e49b1 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCcrStatsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCcrStatsAction.java @@ -14,7 +14,7 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; -import org.elasticsearch.xpack.ccr.action.CcrStatsAction; +import org.elasticsearch.xpack.core.ccr.action.CcrStatsAction; import java.io.IOException; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCreateAndFollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCreateAndFollowIndexAction.java index 4d9079b36c943..8816760f52617 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCreateAndFollowIndexAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestCreateAndFollowIndexAction.java @@ -14,8 +14,8 @@ import java.io.IOException; -import static org.elasticsearch.xpack.ccr.action.CreateAndFollowIndexAction.INSTANCE; -import static org.elasticsearch.xpack.ccr.action.CreateAndFollowIndexAction.Request; +import static org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction.INSTANCE; +import static org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction.Request; public class RestCreateAndFollowIndexAction extends BaseRestHandler { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestFollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestFollowIndexAction.java index 88f5b74f4b141..8a1d7d778bd86 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestFollowIndexAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestFollowIndexAction.java @@ -15,8 +15,8 @@ import java.io.IOException; -import static org.elasticsearch.xpack.ccr.action.FollowIndexAction.INSTANCE; -import static org.elasticsearch.xpack.ccr.action.FollowIndexAction.Request; +import static org.elasticsearch.xpack.core.ccr.action.FollowIndexAction.INSTANCE; +import static org.elasticsearch.xpack.core.ccr.action.FollowIndexAction.Request; public class RestFollowIndexAction extends BaseRestHandler { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestUnfollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestUnfollowIndexAction.java index 2df6c77379b24..9a82717b621bd 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestUnfollowIndexAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestUnfollowIndexAction.java @@ -14,8 +14,8 @@ import java.io.IOException; -import static org.elasticsearch.xpack.ccr.action.UnfollowIndexAction.INSTANCE; -import static org.elasticsearch.xpack.ccr.action.UnfollowIndexAction.Request; +import static org.elasticsearch.xpack.core.ccr.action.UnfollowIndexAction.INSTANCE; +import static org.elasticsearch.xpack.core.ccr.action.UnfollowIndexAction.Request; public class RestUnfollowIndexAction extends BaseRestHandler { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java index 2d58358d11f88..ecf2bd47fc7d9 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java @@ -22,11 +22,10 @@ import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.MockLogAppender; 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.FollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.CcrStatsAction; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; import org.elasticsearch.xpack.ccr.action.PutAutoFollowPatternAction; -import org.elasticsearch.xpack.ccr.action.ShardFollowNodeTask; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; @@ -196,11 +195,11 @@ private FollowIndexAction.Request getFollowRequest() { return new FollowIndexAction.Request( "leader", "follower", - ShardFollowNodeTask.DEFAULT_MAX_BATCH_OPERATION_COUNT, - ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READ_BATCHES, - ShardFollowNodeTask.DEFAULT_MAX_BATCH_SIZE_IN_BYTES, - ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITE_BATCHES, - ShardFollowNodeTask.DEFAULT_MAX_WRITE_BUFFER_SIZE, + FollowIndexAction.DEFAULT_MAX_BATCH_OPERATION_COUNT, + FollowIndexAction.DEFAULT_MAX_CONCURRENT_READ_BATCHES, + FollowIndexAction.DEFAULT_MAX_BATCH_SIZE_IN_BYTES, + FollowIndexAction.DEFAULT_MAX_CONCURRENT_WRITE_BATCHES, + FollowIndexAction.DEFAULT_MAX_WRITE_BUFFER_SIZE, TimeValue.timeValueMillis(10), TimeValue.timeValueMillis(10)); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java index 7980e1281406a..c0919f25fe3ff 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java @@ -38,13 +38,13 @@ import org.elasticsearch.test.MockHttpTransport; import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.junit.annotations.TestLogging; -import org.elasticsearch.xpack.ccr.action.CreateAndFollowIndexAction; -import org.elasticsearch.xpack.ccr.action.FollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; 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.UnfollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.UnfollowIndexAction; import org.elasticsearch.xpack.core.XPackSettings; +import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; import java.io.IOException; import java.util.Arrays; @@ -335,7 +335,7 @@ public void testFollowIndexAndCloseNode() throws Exception { final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", randomIntBetween(32, 2048), randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(2, 10), - ShardFollowNodeTask.DEFAULT_MAX_WRITE_BUFFER_SIZE, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); + FollowIndexAction.DEFAULT_MAX_WRITE_BUFFER_SIZE, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); long maxNumDocsReplicated = Math.min(1000, randomLongBetween(followRequest.getMaxBatchOperationCount(), @@ -507,7 +507,7 @@ private CheckedRunnable assertTask(final int numberOfPrimaryShards, f } } assertThat(taskInfo, notNullValue()); - ShardFollowNodeTask.Status status = (ShardFollowNodeTask.Status) taskInfo.getStatus(); + ShardFollowNodeTaskStatus status = (ShardFollowNodeTaskStatus) taskInfo.getStatus(); assertThat(status, notNullValue()); assertThat("incorrect global checkpoint " + shardFollowTaskParams, status.followerGlobalCheckpoint(), @@ -665,9 +665,9 @@ private void assertSameDocCount(String index1, String index2) throws Exception { } public static FollowIndexAction.Request createFollowRequest(String leaderIndex, String followIndex) { - return new FollowIndexAction.Request(leaderIndex, followIndex, ShardFollowNodeTask.DEFAULT_MAX_BATCH_OPERATION_COUNT, - ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READ_BATCHES, ShardFollowNodeTask.DEFAULT_MAX_BATCH_SIZE_IN_BYTES, - ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITE_BATCHES, ShardFollowNodeTask.DEFAULT_MAX_WRITE_BUFFER_SIZE, + return new FollowIndexAction.Request(leaderIndex, followIndex, FollowIndexAction.DEFAULT_MAX_BATCH_OPERATION_COUNT, + FollowIndexAction.DEFAULT_MAX_CONCURRENT_READ_BATCHES, FollowIndexAction.DEFAULT_MAX_BATCH_SIZE_IN_BYTES, + FollowIndexAction.DEFAULT_MAX_CONCURRENT_WRITE_BATCHES, FollowIndexAction.DEFAULT_MAX_WRITE_BUFFER_SIZE, TimeValue.timeValueMillis(10), TimeValue.timeValueMillis(10)); } } 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 2ef841292322a..5ab11cf5b0c81 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 @@ -17,6 +17,7 @@ import org.elasticsearch.xpack.ccr.action.AutoFollowCoordinator.AutoFollower; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata; import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata.AutoFollowPattern; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; import java.util.ArrayList; import java.util.Collections; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexRequestTests.java index c68d18499658c..c751ca5f00082 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexRequestTests.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.ccr.action; import org.elasticsearch.test.AbstractStreamableTestCase; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; public class CreateAndFollowIndexRequestTests extends AbstractStreamableTestCase { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexResponseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexResponseTests.java index 11a518ef06757..44ac21055a778 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexResponseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/CreateAndFollowIndexResponseTests.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.ccr.action; import org.elasticsearch.test.AbstractStreamableTestCase; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; public class CreateAndFollowIndexResponseTests extends AbstractStreamableTestCase { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowIndexRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowIndexRequestTests.java index 7202f7202c643..2017fa2fdb989 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowIndexRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowIndexRequestTests.java @@ -8,6 +8,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractStreamableXContentTestCase; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; import java.io.IOException; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java index 9bfd6b9d6ef42..dacb60372e634 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java @@ -15,6 +15,8 @@ import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; +import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -52,7 +54,7 @@ public void testMultipleReaderWriter() throws Exception { private void startAndAssertAndStopTask(ShardFollowNodeTask task, TestRun testRun) throws Exception { task.start(testRun.startSeqNo - 1, testRun.startSeqNo - 1, testRun.startSeqNo - 1, testRun.startSeqNo - 1); assertBusy(() -> { - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.leaderGlobalCheckpoint(), equalTo(testRun.finalExpectedGlobalCheckpoint)); assertThat(status.followerGlobalCheckpoint(), equalTo(testRun.finalExpectedGlobalCheckpoint)); final long numberOfFailedFetches = @@ -65,7 +67,7 @@ private void startAndAssertAndStopTask(ShardFollowNodeTask task, TestRun testRun task.markAsCompleted(); assertBusy(() -> { - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(0)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); }); @@ -75,7 +77,7 @@ private ShardFollowNodeTask createShardFollowTask(int concurrency, TestRun testR AtomicBoolean stopped = new AtomicBoolean(false); ShardFollowTask params = new ShardFollowTask(null, new ShardId("follow_index", "", 0), new ShardId("leader_index", "", 0), testRun.maxOperationCount, concurrency, - ShardFollowNodeTask.DEFAULT_MAX_BATCH_SIZE_IN_BYTES, concurrency, 10240, + FollowIndexAction.DEFAULT_MAX_BATCH_SIZE_IN_BYTES, concurrency, 10240, TimeValue.timeValueMillis(10), TimeValue.timeValueMillis(10), Collections.emptyMap()); ThreadPool threadPool = new TestThreadPool(getClass().getSimpleName()); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskStatusTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskStatusTests.java index 8368a818e006e..2f145e7a98c9f 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskStatusTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskStatusTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; +import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; import java.io.IOException; import java.util.Map; @@ -21,17 +22,17 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -public class ShardFollowNodeTaskStatusTests extends AbstractSerializingTestCase { +public class ShardFollowNodeTaskStatusTests extends AbstractSerializingTestCase { @Override - protected ShardFollowNodeTask.Status doParseInstance(XContentParser parser) throws IOException { - return ShardFollowNodeTask.Status.fromXContent(parser); + protected ShardFollowNodeTaskStatus doParseInstance(XContentParser parser) throws IOException { + return ShardFollowNodeTaskStatus.fromXContent(parser); } @Override - protected ShardFollowNodeTask.Status createTestInstance() { + protected ShardFollowNodeTaskStatus createTestInstance() { // if you change this constructor, reflect the changes in the hand-written assertions below - return new ShardFollowNodeTask.Status( + return new ShardFollowNodeTaskStatus( randomAlphaOfLength(4), randomInt(), randomNonNegativeLong(), @@ -57,7 +58,7 @@ protected ShardFollowNodeTask.Status createTestInstance() { } @Override - protected void assertEqualInstances(final ShardFollowNodeTask.Status expectedInstance, final ShardFollowNodeTask.Status newInstance) { + protected void assertEqualInstances(final ShardFollowNodeTaskStatus expectedInstance, final ShardFollowNodeTaskStatus newInstance) { assertNotSame(expectedInstance, newInstance); assertThat(newInstance.leaderIndex(), equalTo(expectedInstance.leaderIndex())); assertThat(newInstance.getShardId(), equalTo(expectedInstance.getShardId())); @@ -108,8 +109,8 @@ private NavigableMap randomReadExceptions() { } @Override - protected Writeable.Reader instanceReader() { - return ShardFollowNodeTask.Status::new; + protected Writeable.Reader instanceReader() { + return ShardFollowNodeTaskStatus::new; } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskTests.java index 4f7c0bf16645c..e177f77e61377 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; +import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; import java.net.ConnectException; import java.nio.charset.StandardCharsets; @@ -44,7 +45,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase { private List> bulkShardOperationRequests; private BiConsumer scheduler = (delay, task) -> task.run(); - private Consumer beforeSendShardChangesRequest = status -> {}; + private Consumer beforeSendShardChangesRequest = status -> {}; private AtomicBoolean simulateResponse = new AtomicBoolean(); @@ -66,7 +67,7 @@ public void testCoordinateReads() { assertThat(shardChangesRequests, contains(new long[][]{ {6L, 8L}, {14L, 8L}, {22L, 8L}, {30L, 8L}, {38L, 8L}, {46L, 8L}, {54L, 7L}} )); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(7)); assertThat(status.lastRequestedSeqNo(), equalTo(60L)); } @@ -86,7 +87,7 @@ public void testWriteBuffer() { task.innerHandleReadResponse(0L, 63L, generateShardChangesResponse(0, 63, 0L, 128L)); assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because write buffer is full - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(0)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); @@ -102,7 +103,7 @@ public void testMaxConcurrentReads() { assertThat(shardChangesRequests.get(0)[0], equalTo(0L)); assertThat(shardChangesRequests.get(0)[1], equalTo(8L)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(7L)); } @@ -140,7 +141,7 @@ public void testTaskCancelledAfterReadLimitHasBeenReached() { assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because task has been cancelled assertThat(bulkShardOperationRequests.size(), equalTo(0)); // no more writes, because task has been cancelled - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(0)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(15L)); @@ -164,7 +165,7 @@ public void testTaskCancelledAfterWriteBufferLimitHasBeenReached() { assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because task has been cancelled assertThat(bulkShardOperationRequests.size(), equalTo(0)); // no more writes, because task has been cancelled - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(0)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); @@ -211,7 +212,7 @@ public void testReceiveRetryableError() { } assertFalse("task is not stopped", task.isStopped()); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.numberOfFailedFetches(), equalTo((long)max)); @@ -258,7 +259,7 @@ public void testReceiveRetryableErrorRetriedTooManyTimes() { assertTrue("task is stopped", task.isStopped()); assertThat(fatalError, notNullValue()); assertThat(fatalError.getMessage(), containsString("retrying failed [")); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.numberOfFailedFetches(), equalTo(11L)); @@ -299,7 +300,7 @@ public void testReceiveNonRetryableError() { assertTrue("task is stopped", task.isStopped()); assertThat(fatalError, sameInstance(failure)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.numberOfFailedFetches(), equalTo(1L)); @@ -326,7 +327,7 @@ public void testHandleReadResponse() { assertThat(bulkShardOperationRequests.size(), equalTo(1)); assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.mappingVersion(), equalTo(0L)); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentReads(), equalTo(1)); @@ -353,7 +354,7 @@ public void testReceiveLessThanRequested() { assertThat(shardChangesRequests.get(0)[0], equalTo(21L)); assertThat(shardChangesRequests.get(0)[1], equalTo(43L)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); @@ -376,7 +377,7 @@ public void testCancelAndReceiveLessThanRequested() { assertThat(shardChangesRequests.size(), equalTo(0)); assertThat(bulkShardOperationRequests.size(), equalTo(0)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(0)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); @@ -399,7 +400,7 @@ public void testReceiveNothingExpectedSomething() { assertThat(shardChangesRequests.get(0)[0], equalTo(0L)); assertThat(shardChangesRequests.get(0)[1], equalTo(64L)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); @@ -441,7 +442,7 @@ public void testMappingUpdate() { assertThat(bulkShardOperationRequests.size(), equalTo(1)); assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.mappingVersion(), equalTo(1L)); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(1)); @@ -466,7 +467,7 @@ public void testMappingUpdateRetryableError() { assertThat(mappingUpdateFailures.size(), equalTo(0)); assertThat(bulkShardOperationRequests.size(), equalTo(1)); assertThat(task.isStopped(), equalTo(false)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.mappingVersion(), equalTo(1L)); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(1)); @@ -492,7 +493,7 @@ public void testMappingUpdateRetryableErrorRetriedTooManyTimes() { assertThat(mappingVersions.size(), equalTo(1)); assertThat(bulkShardOperationRequests.size(), equalTo(0)); assertThat(task.isStopped(), equalTo(true)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.mappingVersion(), equalTo(0L)); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); @@ -511,7 +512,7 @@ public void testMappingUpdateNonRetryableError() { assertThat(bulkShardOperationRequests.size(), equalTo(0)); assertThat(task.isStopped(), equalTo(true)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.mappingVersion(), equalTo(0L)); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(0)); @@ -535,7 +536,7 @@ public void testCoordinateWrites() { assertThat(bulkShardOperationRequests.size(), equalTo(1)); assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.numberOfConcurrentWrites(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); @@ -553,7 +554,7 @@ public void testMaxConcurrentWrites() { assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()).subList(0, 64))); assertThat(bulkShardOperationRequests.get(1), equalTo(Arrays.asList(response.getOperations()).subList(64, 128))); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentWrites(), equalTo(2)); task = createShardFollowTask(64, 1, 4, Integer.MAX_VALUE, Long.MAX_VALUE); @@ -583,7 +584,7 @@ public void testMaxBatchOperationCount() { assertThat(bulkShardOperationRequests.get(i), equalTo(Arrays.asList(response.getOperations()).subList(offset, offset + 8))); } - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentWrites(), equalTo(32)); } @@ -610,7 +611,7 @@ public void testRetryableError() { assertThat(operations, equalTo(Arrays.asList(response.getOperations()))); } assertThat(task.isStopped(), equalTo(false)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentWrites(), equalTo(1)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } @@ -638,7 +639,7 @@ public void testRetryableErrorRetriedTooManyTimes() { assertThat(operations, equalTo(Arrays.asList(response.getOperations()))); } assertThat(task.isStopped(), equalTo(true)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentWrites(), equalTo(1)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } @@ -660,7 +661,7 @@ public void testNonRetryableError() { assertThat(bulkShardOperationRequests.size(), equalTo(1)); assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); assertThat(task.isStopped(), equalTo(true)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentWrites(), equalTo(1)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } @@ -704,7 +705,7 @@ public void testHandleWriteResponse() { assertThat(shardChangesRequests.get(0)[0], equalTo(64L)); assertThat(shardChangesRequests.get(0)[1], equalTo(64L)); - ShardFollowNodeTask.Status status = task.getStatus(); + ShardFollowNodeTaskStatus status = task.getStatus(); assertThat(status.numberOfConcurrentReads(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowIndexActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportFollowIndexActionTests.java similarity index 89% rename from x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowIndexActionTests.java rename to x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportFollowIndexActionTests.java index 5b52700f5579b..7691945643d57 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowIndexActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportFollowIndexActionTests.java @@ -3,6 +3,7 @@ * 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; @@ -15,32 +16,33 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.ccr.CcrSettings; import org.elasticsearch.xpack.ccr.ShardChangesIT; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; import java.io.IOException; +import static org.elasticsearch.xpack.ccr.action.TransportFollowIndexAction.validate; import static org.hamcrest.Matchers.equalTo; -public class FollowIndexActionTests extends ESTestCase { +public class TransportFollowIndexActionTests extends ESTestCase { public void testValidation() throws IOException { FollowIndexAction.Request request = ShardChangesIT.createFollowRequest("index1", "index2"); { // should fail, because leader index does not exist - Exception e = expectThrows(IllegalArgumentException.class, () -> FollowIndexAction.validate(request, null, null, null)); + Exception e = expectThrows(IllegalArgumentException.class, () -> validate(request, null, null, null)); assertThat(e.getMessage(), equalTo("leader index [index1] does not exist")); } { // should fail, because follow index does not exist IndexMetaData leaderIMD = createIMD("index1", 5, Settings.EMPTY); - Exception e = expectThrows(IllegalArgumentException.class, () -> FollowIndexAction.validate(request, leaderIMD, null, null)); + Exception e = expectThrows(IllegalArgumentException.class, () -> validate(request, leaderIMD, null, null)); assertThat(e.getMessage(), equalTo("follow index [index2] does not exist")); } { // should fail because leader index does not have soft deletes enabled IndexMetaData leaderIMD = createIMD("index1", 5, Settings.EMPTY); IndexMetaData followIMD = createIMD("index2", 5, Settings.EMPTY); - Exception e = expectThrows(IllegalArgumentException.class, - () -> FollowIndexAction.validate(request, leaderIMD, followIMD, null)); + Exception e = expectThrows(IllegalArgumentException.class, () -> validate(request, leaderIMD, followIMD, null)); assertThat(e.getMessage(), equalTo("leader index [index1] does not have soft deletes enabled")); } { @@ -48,8 +50,7 @@ public void testValidation() throws IOException { IndexMetaData leaderIMD = createIMD("index1", 5, Settings.builder() .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true").build()); IndexMetaData followIMD = createIMD("index2", 4, Settings.EMPTY); - Exception e = expectThrows(IllegalArgumentException.class, - () -> FollowIndexAction.validate(request, leaderIMD, followIMD, null)); + Exception e = expectThrows(IllegalArgumentException.class, () -> validate(request, leaderIMD, followIMD, null)); assertThat(e.getMessage(), equalTo("leader index primary shards [5] does not match with the number of shards of the follow index [4]")); } @@ -59,8 +60,7 @@ public void testValidation() throws IOException { .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true").build()); IndexMetaData followIMD = createIMD("index2", State.OPEN, "{}", 5, Settings.builder() .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true").build()); - Exception e = expectThrows(IllegalArgumentException.class, - () -> FollowIndexAction.validate(request, leaderIMD, followIMD, null)); + Exception e = expectThrows(IllegalArgumentException.class, () -> validate(request, leaderIMD, followIMD, null)); assertThat(e.getMessage(), equalTo("leader and follow index must be open")); } { @@ -71,8 +71,7 @@ public void testValidation() throws IOException { Settings.builder().put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true).build()); MapperService mapperService = MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), Settings.EMPTY, "index2"); mapperService.updateMapping(null, followIMD); - Exception e = expectThrows(IllegalArgumentException.class, - () -> FollowIndexAction.validate(request, leaderIMD, followIMD, mapperService)); + Exception e = expectThrows(IllegalArgumentException.class, () -> validate(request, leaderIMD, followIMD, mapperService)); assertThat(e.getMessage(), equalTo("mapper [field] of different type, current_type [text], merged_type [keyword]")); } { @@ -86,8 +85,7 @@ public void testValidation() throws IOException { .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true) .put("index.analysis.analyzer.my_analyzer.type", "custom") .put("index.analysis.analyzer.my_analyzer.tokenizer", "standard").build()); - Exception e = expectThrows(IllegalArgumentException.class, - () -> FollowIndexAction.validate(request, leaderIMD, followIMD, null)); + Exception e = expectThrows(IllegalArgumentException.class, () -> validate(request, leaderIMD, followIMD, null)); assertThat(e.getMessage(), equalTo("the leader and follower index settings must be identical")); } { @@ -100,8 +98,8 @@ public void testValidation() throws IOException { MapperService mapperService = MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), followingIndexSettings, "index2"); mapperService.updateMapping(null, followIMD); - IllegalArgumentException error = expectThrows(IllegalArgumentException.class, - () -> FollowIndexAction.validate(request, leaderIMD, followIMD, mapperService)); + IllegalArgumentException error = + expectThrows(IllegalArgumentException.class, () -> validate(request, leaderIMD, followIMD, mapperService)); assertThat(error.getMessage(), equalTo("the following index [index2] is not ready to follow; " + "the setting [index.xpack.ccr.following_index] must be enabled.")); } @@ -113,7 +111,7 @@ public void testValidation() throws IOException { .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true).build()); MapperService mapperService = MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), Settings.EMPTY, "index2"); mapperService.updateMapping(null, followIMD); - FollowIndexAction.validate(request, leaderIMD, followIMD, mapperService); + validate(request, leaderIMD, followIMD, mapperService); } { // should succeed, index settings are identical @@ -129,7 +127,7 @@ public void testValidation() throws IOException { MapperService mapperService = MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), followIMD.getSettings(), "index2"); mapperService.updateMapping(null, followIMD); - FollowIndexAction.validate(request, leaderIMD, followIMD, mapperService); + validate(request, leaderIMD, followIMD, mapperService); } { // should succeed despite whitelisted settings being different @@ -147,7 +145,7 @@ public void testValidation() throws IOException { MapperService mapperService = MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), followIMD.getSettings(), "index2"); mapperService.updateMapping(null, followIMD); - FollowIndexAction.validate(request, leaderIMD, followIMD, mapperService); + validate(request, leaderIMD, followIMD, mapperService); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClient.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClient.java index 77f511ba4d0ef..3f27f66b27b77 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClient.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClient.java @@ -13,6 +13,7 @@ import org.elasticsearch.protocol.xpack.XPackInfoResponse; import org.elasticsearch.xpack.core.action.XPackInfoAction; import org.elasticsearch.xpack.core.action.XPackInfoRequestBuilder; +import org.elasticsearch.xpack.core.ccr.client.CcrClient; import org.elasticsearch.xpack.core.ml.client.MachineLearningClient; import org.elasticsearch.xpack.core.monitoring.client.MonitoringClient; import org.elasticsearch.xpack.core.security.client.SecurityClient; @@ -20,6 +21,7 @@ import java.util.Collections; import java.util.Map; +import java.util.Objects; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.BASIC_AUTH_HEADER; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; @@ -28,6 +30,7 @@ public class XPackClient { private final Client client; + private final CcrClient ccrClient; private final LicensingClient licensingClient; private final MonitoringClient monitoringClient; private final SecurityClient securityClient; @@ -35,7 +38,8 @@ public class XPackClient { private final MachineLearningClient machineLearning; public XPackClient(Client client) { - this.client = client; + this.client = Objects.requireNonNull(client, "client"); + this.ccrClient = new CcrClient(client); this.licensingClient = new LicensingClient(client); this.monitoringClient = new MonitoringClient(client); this.securityClient = new SecurityClient(client); @@ -47,6 +51,10 @@ public Client es() { return client; } + public CcrClient ccr() { + return ccrClient; + } + public LicensingClient licensing() { return licensingClient; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/ShardFollowNodeTaskStatus.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/ShardFollowNodeTaskStatus.java new file mode 100644 index 0000000000000..783999cf1836e --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/ShardFollowNodeTaskStatus.java @@ -0,0 +1,504 @@ +/* + * 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.ElasticsearchException; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.tasks.Task; + +import java.io.IOException; +import java.util.AbstractMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class ShardFollowNodeTaskStatus implements Task.Status { + + public static final String STATUS_PARSER_NAME = "shard-follow-node-task-status"; + + private static final ParseField LEADER_INDEX = new ParseField("leader_index"); + private static final ParseField SHARD_ID = new ParseField("shard_id"); + private static final ParseField LEADER_GLOBAL_CHECKPOINT_FIELD = new ParseField("leader_global_checkpoint"); + private static final ParseField LEADER_MAX_SEQ_NO_FIELD = new ParseField("leader_max_seq_no"); + private static final ParseField FOLLOWER_GLOBAL_CHECKPOINT_FIELD = new ParseField("follower_global_checkpoint"); + private static final ParseField FOLLOWER_MAX_SEQ_NO_FIELD = new ParseField("follower_max_seq_no"); + private static final ParseField LAST_REQUESTED_SEQ_NO_FIELD = new ParseField("last_requested_seq_no"); + private static final ParseField NUMBER_OF_CONCURRENT_READS_FIELD = new ParseField("number_of_concurrent_reads"); + private static final ParseField NUMBER_OF_CONCURRENT_WRITES_FIELD = new ParseField("number_of_concurrent_writes"); + private static final ParseField NUMBER_OF_QUEUED_WRITES_FIELD = new ParseField("number_of_queued_writes"); + private static final ParseField MAPPING_VERSION_FIELD = new ParseField("mapping_version"); + private static final ParseField TOTAL_FETCH_TIME_MILLIS_FIELD = new ParseField("total_fetch_time_millis"); + private static final ParseField NUMBER_OF_SUCCESSFUL_FETCHES_FIELD = new ParseField("number_of_successful_fetches"); + private static final ParseField NUMBER_OF_FAILED_FETCHES_FIELD = new ParseField("number_of_failed_fetches"); + private static final ParseField OPERATIONS_RECEIVED_FIELD = new ParseField("operations_received"); + private static final ParseField TOTAL_TRANSFERRED_BYTES = new ParseField("total_transferred_bytes"); + private static final ParseField TOTAL_INDEX_TIME_MILLIS_FIELD = new ParseField("total_index_time_millis"); + private static final ParseField NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD = new ParseField("number_of_successful_bulk_operations"); + private static final ParseField NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD = new ParseField("number_of_failed_bulk_operations"); + private static final ParseField NUMBER_OF_OPERATIONS_INDEXED_FIELD = new ParseField("number_of_operations_indexed"); + private static final ParseField FETCH_EXCEPTIONS = new ParseField("fetch_exceptions"); + private static final ParseField TIME_SINCE_LAST_FETCH_MILLIS_FIELD = new ParseField("time_since_last_fetch_millis"); + + @SuppressWarnings("unchecked") + static final ConstructingObjectParser STATUS_PARSER = + new ConstructingObjectParser<>( + STATUS_PARSER_NAME, + args -> new ShardFollowNodeTaskStatus( + (String) args[0], + (int) args[1], + (long) args[2], + (long) args[3], + (long) args[4], + (long) args[5], + (long) args[6], + (int) args[7], + (int) args[8], + (int) args[9], + (long) args[10], + (long) args[11], + (long) args[12], + (long) args[13], + (long) args[14], + (long) args[15], + (long) args[16], + (long) args[17], + (long) args[18], + (long) args[19], + new TreeMap<>( + ((List>) args[20]) + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))), + (long) args[21])); + + public static final String FETCH_EXCEPTIONS_ENTRY_PARSER_NAME = "shard-follow-node-task-status-fetch-exceptions-entry"; + + static final ConstructingObjectParser, Void> FETCH_EXCEPTIONS_ENTRY_PARSER = + new ConstructingObjectParser<>( + FETCH_EXCEPTIONS_ENTRY_PARSER_NAME, + args -> new AbstractMap.SimpleEntry<>((long) args[0], (ElasticsearchException) args[1])); + + static { + STATUS_PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_INDEX); + STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), SHARD_ID); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), LEADER_GLOBAL_CHECKPOINT_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), LEADER_MAX_SEQ_NO_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_GLOBAL_CHECKPOINT_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_MAX_SEQ_NO_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), LAST_REQUESTED_SEQ_NO_FIELD); + STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_READS_FIELD); + STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_WRITES_FIELD); + STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_QUEUED_WRITES_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAPPING_VERSION_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_FETCH_TIME_MILLIS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_SUCCESSFUL_FETCHES_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_FAILED_FETCHES_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), OPERATIONS_RECEIVED_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_TRANSFERRED_BYTES); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_INDEX_TIME_MILLIS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_OPERATIONS_INDEXED_FIELD); + STATUS_PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), FETCH_EXCEPTIONS_ENTRY_PARSER, FETCH_EXCEPTIONS); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TIME_SINCE_LAST_FETCH_MILLIS_FIELD); + } + + static final ParseField FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO = new ParseField("from_seq_no"); + static final ParseField FETCH_EXCEPTIONS_ENTRY_EXCEPTION = new ParseField("exception"); + + static { + FETCH_EXCEPTIONS_ENTRY_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO); + FETCH_EXCEPTIONS_ENTRY_PARSER.declareObject( + ConstructingObjectParser.constructorArg(), + (p, c) -> ElasticsearchException.fromXContent(p), + FETCH_EXCEPTIONS_ENTRY_EXCEPTION); + } + + private final String leaderIndex; + + public String leaderIndex() { + return leaderIndex; + } + + private final int shardId; + + public int getShardId() { + return shardId; + } + + private final long leaderGlobalCheckpoint; + + public long leaderGlobalCheckpoint() { + return leaderGlobalCheckpoint; + } + + private final long leaderMaxSeqNo; + + public long leaderMaxSeqNo() { + return leaderMaxSeqNo; + } + + private final long followerGlobalCheckpoint; + + public long followerGlobalCheckpoint() { + return followerGlobalCheckpoint; + } + + private final long followerMaxSeqNo; + + public long followerMaxSeqNo() { + return followerMaxSeqNo; + } + + private final long lastRequestedSeqNo; + + public long lastRequestedSeqNo() { + return lastRequestedSeqNo; + } + + private final int numberOfConcurrentReads; + + public int numberOfConcurrentReads() { + return numberOfConcurrentReads; + } + + private final int numberOfConcurrentWrites; + + public int numberOfConcurrentWrites() { + return numberOfConcurrentWrites; + } + + private final int numberOfQueuedWrites; + + public int numberOfQueuedWrites() { + return numberOfQueuedWrites; + } + + private final long mappingVersion; + + public long mappingVersion() { + return mappingVersion; + } + + private final long totalFetchTimeMillis; + + public long totalFetchTimeMillis() { + return totalFetchTimeMillis; + } + + private final long numberOfSuccessfulFetches; + + public long numberOfSuccessfulFetches() { + return numberOfSuccessfulFetches; + } + + private final long numberOfFailedFetches; + + public long numberOfFailedFetches() { + return numberOfFailedFetches; + } + + private final long operationsReceived; + + public long operationsReceived() { + return operationsReceived; + } + + private final long totalTransferredBytes; + + public long totalTransferredBytes() { + return totalTransferredBytes; + } + + private final long totalIndexTimeMillis; + + public long totalIndexTimeMillis() { + return totalIndexTimeMillis; + } + + private final long numberOfSuccessfulBulkOperations; + + public long numberOfSuccessfulBulkOperations() { + return numberOfSuccessfulBulkOperations; + } + + private final long numberOfFailedBulkOperations; + + public long numberOfFailedBulkOperations() { + return numberOfFailedBulkOperations; + } + + private final long numberOfOperationsIndexed; + + public long numberOfOperationsIndexed() { + return numberOfOperationsIndexed; + } + + private final NavigableMap fetchExceptions; + + public NavigableMap fetchExceptions() { + return fetchExceptions; + } + + private final long timeSinceLastFetchMillis; + + public long timeSinceLastFetchMillis() { + return timeSinceLastFetchMillis; + } + + public ShardFollowNodeTaskStatus( + final String leaderIndex, + final int shardId, + final long leaderGlobalCheckpoint, + final long leaderMaxSeqNo, + final long followerGlobalCheckpoint, + final long followerMaxSeqNo, + final long lastRequestedSeqNo, + final int numberOfConcurrentReads, + final int numberOfConcurrentWrites, + final int numberOfQueuedWrites, + final long mappingVersion, + final long totalFetchTimeMillis, + final long numberOfSuccessfulFetches, + final long numberOfFailedFetches, + final long operationsReceived, + final long totalTransferredBytes, + final long totalIndexTimeMillis, + final long numberOfSuccessfulBulkOperations, + final long numberOfFailedBulkOperations, + final long numberOfOperationsIndexed, + final NavigableMap fetchExceptions, + final long timeSinceLastFetchMillis) { + this.leaderIndex = leaderIndex; + this.shardId = shardId; + this.leaderGlobalCheckpoint = leaderGlobalCheckpoint; + this.leaderMaxSeqNo = leaderMaxSeqNo; + this.followerGlobalCheckpoint = followerGlobalCheckpoint; + this.followerMaxSeqNo = followerMaxSeqNo; + this.lastRequestedSeqNo = lastRequestedSeqNo; + this.numberOfConcurrentReads = numberOfConcurrentReads; + this.numberOfConcurrentWrites = numberOfConcurrentWrites; + this.numberOfQueuedWrites = numberOfQueuedWrites; + this.mappingVersion = mappingVersion; + this.totalFetchTimeMillis = totalFetchTimeMillis; + this.numberOfSuccessfulFetches = numberOfSuccessfulFetches; + this.numberOfFailedFetches = numberOfFailedFetches; + this.operationsReceived = operationsReceived; + this.totalTransferredBytes = totalTransferredBytes; + this.totalIndexTimeMillis = totalIndexTimeMillis; + this.numberOfSuccessfulBulkOperations = numberOfSuccessfulBulkOperations; + this.numberOfFailedBulkOperations = numberOfFailedBulkOperations; + this.numberOfOperationsIndexed = numberOfOperationsIndexed; + this.fetchExceptions = Objects.requireNonNull(fetchExceptions); + this.timeSinceLastFetchMillis = timeSinceLastFetchMillis; + } + + public ShardFollowNodeTaskStatus(final StreamInput in) throws IOException { + this.leaderIndex = in.readString(); + this.shardId = in.readVInt(); + this.leaderGlobalCheckpoint = in.readZLong(); + this.leaderMaxSeqNo = in.readZLong(); + this.followerGlobalCheckpoint = in.readZLong(); + this.followerMaxSeqNo = in.readZLong(); + this.lastRequestedSeqNo = in.readZLong(); + this.numberOfConcurrentReads = in.readVInt(); + this.numberOfConcurrentWrites = in.readVInt(); + this.numberOfQueuedWrites = in.readVInt(); + this.mappingVersion = in.readVLong(); + this.totalFetchTimeMillis = in.readVLong(); + this.numberOfSuccessfulFetches = in.readVLong(); + this.numberOfFailedFetches = in.readVLong(); + this.operationsReceived = in.readVLong(); + this.totalTransferredBytes = in.readVLong(); + this.totalIndexTimeMillis = in.readVLong(); + this.numberOfSuccessfulBulkOperations = in.readVLong(); + this.numberOfFailedBulkOperations = in.readVLong(); + this.numberOfOperationsIndexed = in.readVLong(); + this.fetchExceptions = new TreeMap<>(in.readMap(StreamInput::readVLong, StreamInput::readException)); + this.timeSinceLastFetchMillis = in.readZLong(); + } + + @Override + public String getWriteableName() { + return STATUS_PARSER_NAME; + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + out.writeString(leaderIndex); + out.writeVInt(shardId); + out.writeZLong(leaderGlobalCheckpoint); + out.writeZLong(leaderMaxSeqNo); + out.writeZLong(followerGlobalCheckpoint); + out.writeZLong(followerMaxSeqNo); + out.writeZLong(lastRequestedSeqNo); + out.writeVInt(numberOfConcurrentReads); + out.writeVInt(numberOfConcurrentWrites); + out.writeVInt(numberOfQueuedWrites); + out.writeVLong(mappingVersion); + out.writeVLong(totalFetchTimeMillis); + out.writeVLong(numberOfSuccessfulFetches); + out.writeVLong(numberOfFailedFetches); + out.writeVLong(operationsReceived); + out.writeVLong(totalTransferredBytes); + out.writeVLong(totalIndexTimeMillis); + out.writeVLong(numberOfSuccessfulBulkOperations); + out.writeVLong(numberOfFailedBulkOperations); + out.writeVLong(numberOfOperationsIndexed); + out.writeMap(fetchExceptions, StreamOutput::writeVLong, StreamOutput::writeException); + out.writeZLong(timeSinceLastFetchMillis); + } + + @Override + public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException { + builder.startObject(); + { + builder.field(LEADER_INDEX.getPreferredName(), leaderIndex); + builder.field(SHARD_ID.getPreferredName(), shardId); + builder.field(LEADER_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), leaderGlobalCheckpoint); + builder.field(LEADER_MAX_SEQ_NO_FIELD.getPreferredName(), leaderMaxSeqNo); + builder.field(FOLLOWER_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), followerGlobalCheckpoint); + builder.field(FOLLOWER_MAX_SEQ_NO_FIELD.getPreferredName(), followerMaxSeqNo); + builder.field(LAST_REQUESTED_SEQ_NO_FIELD.getPreferredName(), lastRequestedSeqNo); + builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads); + builder.field(NUMBER_OF_CONCURRENT_WRITES_FIELD.getPreferredName(), numberOfConcurrentWrites); + builder.field(NUMBER_OF_QUEUED_WRITES_FIELD.getPreferredName(), numberOfQueuedWrites); + builder.field(MAPPING_VERSION_FIELD.getPreferredName(), mappingVersion); + builder.humanReadableField( + TOTAL_FETCH_TIME_MILLIS_FIELD.getPreferredName(), + "total_fetch_time", + new TimeValue(totalFetchTimeMillis, TimeUnit.MILLISECONDS)); + builder.field(NUMBER_OF_SUCCESSFUL_FETCHES_FIELD.getPreferredName(), numberOfSuccessfulFetches); + builder.field(NUMBER_OF_FAILED_FETCHES_FIELD.getPreferredName(), numberOfFailedFetches); + builder.field(OPERATIONS_RECEIVED_FIELD.getPreferredName(), operationsReceived); + builder.humanReadableField( + TOTAL_TRANSFERRED_BYTES.getPreferredName(), + "total_transferred", + new ByteSizeValue(totalTransferredBytes, ByteSizeUnit.BYTES)); + builder.humanReadableField( + TOTAL_INDEX_TIME_MILLIS_FIELD.getPreferredName(), + "total_index_time", + new TimeValue(totalIndexTimeMillis, TimeUnit.MILLISECONDS)); + builder.field(NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD.getPreferredName(), numberOfSuccessfulBulkOperations); + builder.field(NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD.getPreferredName(), numberOfFailedBulkOperations); + builder.field(NUMBER_OF_OPERATIONS_INDEXED_FIELD.getPreferredName(), numberOfOperationsIndexed); + builder.startArray(FETCH_EXCEPTIONS.getPreferredName()); + { + for (final Map.Entry entry : fetchExceptions.entrySet()) { + builder.startObject(); + { + builder.field(FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO.getPreferredName(), entry.getKey()); + builder.field(FETCH_EXCEPTIONS_ENTRY_EXCEPTION.getPreferredName()); + builder.startObject(); + { + ElasticsearchException.generateThrowableXContent(builder, params, entry.getValue()); + } + builder.endObject(); + } + builder.endObject(); + } + } + builder.endArray(); + builder.humanReadableField( + TIME_SINCE_LAST_FETCH_MILLIS_FIELD.getPreferredName(), + "time_since_last_fetch", + new TimeValue(timeSinceLastFetchMillis, TimeUnit.MILLISECONDS)); + } + builder.endObject(); + return builder; + } + + public static ShardFollowNodeTaskStatus fromXContent(final XContentParser parser) { + return STATUS_PARSER.apply(parser, null); + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + final ShardFollowNodeTaskStatus that = (ShardFollowNodeTaskStatus) o; + return leaderIndex.equals(that.leaderIndex) && + shardId == that.shardId && + leaderGlobalCheckpoint == that.leaderGlobalCheckpoint && + leaderMaxSeqNo == that.leaderMaxSeqNo && + followerGlobalCheckpoint == that.followerGlobalCheckpoint && + followerMaxSeqNo == that.followerMaxSeqNo && + lastRequestedSeqNo == that.lastRequestedSeqNo && + numberOfConcurrentReads == that.numberOfConcurrentReads && + numberOfConcurrentWrites == that.numberOfConcurrentWrites && + numberOfQueuedWrites == that.numberOfQueuedWrites && + mappingVersion == that.mappingVersion && + totalFetchTimeMillis == that.totalFetchTimeMillis && + numberOfSuccessfulFetches == that.numberOfSuccessfulFetches && + numberOfFailedFetches == that.numberOfFailedFetches && + operationsReceived == that.operationsReceived && + totalTransferredBytes == that.totalTransferredBytes && + numberOfSuccessfulBulkOperations == that.numberOfSuccessfulBulkOperations && + numberOfFailedBulkOperations == that.numberOfFailedBulkOperations && + numberOfOperationsIndexed == that.numberOfOperationsIndexed && + /* + * ElasticsearchException does not implement equals so we will assume the fetch exceptions are equal if they are equal + * up to the key set and their messages. Note that we are relying on the fact that the fetch exceptions are ordered by + * keys. + */ + fetchExceptions.keySet().equals(that.fetchExceptions.keySet()) && + getFetchExceptionMessages(this).equals(getFetchExceptionMessages(that)) && + timeSinceLastFetchMillis == that.timeSinceLastFetchMillis; + } + + @Override + public int hashCode() { + return Objects.hash( + leaderIndex, + shardId, + leaderGlobalCheckpoint, + leaderMaxSeqNo, + followerGlobalCheckpoint, + followerMaxSeqNo, + lastRequestedSeqNo, + numberOfConcurrentReads, + numberOfConcurrentWrites, + numberOfQueuedWrites, + mappingVersion, + totalFetchTimeMillis, + numberOfSuccessfulFetches, + numberOfFailedFetches, + operationsReceived, + totalTransferredBytes, + numberOfSuccessfulBulkOperations, + numberOfFailedBulkOperations, + numberOfOperationsIndexed, + /* + * ElasticsearchException does not implement hash code so we will compute the hash code based on the key set and the + * messages. Note that we are relying on the fact that the fetch exceptions are ordered by keys. + */ + fetchExceptions.keySet(), + getFetchExceptionMessages(this), + timeSinceLastFetchMillis); + } + + private static List getFetchExceptionMessages(final ShardFollowNodeTaskStatus status) { + return status.fetchExceptions().values().stream().map(ElasticsearchException::getMessage).collect(Collectors.toList()); + } + + public String toString() { + return Strings.toString(this); + } + +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/CcrStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/CcrStatsAction.java similarity index 92% rename from x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/CcrStatsAction.java rename to x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/CcrStatsAction.java index b5d6697fc73c2..ace3d6bb194e2 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/CcrStatsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/CcrStatsAction.java @@ -4,7 +4,7 @@ * you may not use this file except in compliance with the Elastic License. */ -package org.elasticsearch.xpack.ccr.action; +package org.elasticsearch.xpack.core.ccr.action; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; @@ -21,6 +21,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.tasks.Task; +import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; import java.io.IOException; import java.util.Collections; @@ -51,7 +52,7 @@ public TasksResponse() { this(Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); } - TasksResponse( + public TasksResponse( final List taskFailures, final List nodeFailures, final List taskResponses) { @@ -151,20 +152,20 @@ ShardId followerShardId() { return followerShardId; } - private final ShardFollowNodeTask.Status status; + private final ShardFollowNodeTaskStatus status; - ShardFollowNodeTask.Status status() { + ShardFollowNodeTaskStatus status() { return status; } - TaskResponse(final ShardId followerShardId, final ShardFollowNodeTask.Status status) { + public TaskResponse(final ShardId followerShardId, final ShardFollowNodeTaskStatus status) { this.followerShardId = followerShardId; this.status = status; } - TaskResponse(final StreamInput in) throws IOException { + public TaskResponse(final StreamInput in) throws IOException { this.followerShardId = ShardId.readShardId(in); - this.status = new ShardFollowNodeTask.Status(in); + this.status = new ShardFollowNodeTaskStatus(in); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/CreateAndFollowIndexAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/CreateAndFollowIndexAction.java new file mode 100644 index 0000000000000..ea63815c2b933 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/CreateAndFollowIndexAction.java @@ -0,0 +1,167 @@ +/* + * 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.action; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +public final class CreateAndFollowIndexAction extends Action { + + public static final CreateAndFollowIndexAction INSTANCE = new CreateAndFollowIndexAction(); + public static final String NAME = "indices:admin/xpack/ccr/create_and_follow_index"; + + private CreateAndFollowIndexAction() { + super(NAME); + } + + @Override + public Response newResponse() { + return new Response(); + } + + public static class Request extends AcknowledgedRequest implements IndicesRequest { + + private FollowIndexAction.Request followRequest; + + public Request(FollowIndexAction.Request followRequest) { + this.followRequest = Objects.requireNonNull(followRequest); + } + + public Request() { + + } + + public FollowIndexAction.Request getFollowRequest() { + return followRequest; + } + + @Override + public ActionRequestValidationException validate() { + return followRequest.validate(); + } + + @Override + public String[] indices() { + return new String[]{followRequest.getFollowerIndex()}; + } + + @Override + public IndicesOptions indicesOptions() { + return IndicesOptions.strictSingleIndexNoExpandForbidClosed(); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + followRequest = new FollowIndexAction.Request(); + followRequest.readFrom(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + followRequest.writeTo(out); + } + + @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(followRequest, request.followRequest); + } + + @Override + public int hashCode() { + return Objects.hash(followRequest); + } + } + + public static class Response extends ActionResponse implements ToXContentObject { + + private boolean followIndexCreated; + private boolean followIndexShardsAcked; + private boolean indexFollowingStarted; + + public Response() { + + } + + public Response(boolean followIndexCreated, boolean followIndexShardsAcked, boolean indexFollowingStarted) { + this.followIndexCreated = followIndexCreated; + this.followIndexShardsAcked = followIndexShardsAcked; + this.indexFollowingStarted = indexFollowingStarted; + } + + public boolean isFollowIndexCreated() { + return followIndexCreated; + } + + public boolean isFollowIndexShardsAcked() { + return followIndexShardsAcked; + } + + public boolean isIndexFollowingStarted() { + return indexFollowingStarted; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + followIndexCreated = in.readBoolean(); + followIndexShardsAcked = in.readBoolean(); + indexFollowingStarted = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(followIndexCreated); + out.writeBoolean(followIndexShardsAcked); + out.writeBoolean(indexFollowingStarted); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + { + builder.field("follow_index_created", followIndexCreated); + builder.field("follow_index_shards_acked", followIndexShardsAcked); + builder.field("index_following_started", indexFollowingStarted); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Response response = (Response) o; + return followIndexCreated == response.followIndexCreated && + followIndexShardsAcked == response.followIndexShardsAcked && + indexFollowingStarted == response.indexFollowingStarted; + } + + @Override + public int hashCode() { + return Objects.hash(followIndexCreated, followIndexShardsAcked, indexFollowingStarted); + } + } + +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowIndexAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowIndexAction.java new file mode 100644 index 0000000000000..c42ef8db9c1fe --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/FollowIndexAction.java @@ -0,0 +1,307 @@ +/* + * 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.action; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +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.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 java.io.IOException; +import java.util.Objects; + +public final class FollowIndexAction extends Action { + + public static final FollowIndexAction INSTANCE = new FollowIndexAction(); + public static final String NAME = "cluster:admin/xpack/ccr/follow_index"; + + public static final int DEFAULT_MAX_WRITE_BUFFER_SIZE = 10240; + public static final int DEFAULT_MAX_BATCH_OPERATION_COUNT = 1024; + public static final int DEFAULT_MAX_CONCURRENT_READ_BATCHES = 1; + public static final int DEFAULT_MAX_CONCURRENT_WRITE_BATCHES = 1; + public static final long DEFAULT_MAX_BATCH_SIZE_IN_BYTES = Long.MAX_VALUE; + public static final int RETRY_LIMIT = 10; + public static final TimeValue DEFAULT_RETRY_TIMEOUT = new TimeValue(500); + public static final TimeValue DEFAULT_IDLE_SHARD_RETRY_DELAY = TimeValue.timeValueSeconds(10); + + private FollowIndexAction() { + super(NAME); + } + + @Override + public AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + public static class Request extends ActionRequest implements ToXContentObject { + + private static final ParseField LEADER_INDEX_FIELD = new ParseField("leader_index"); + private static final ParseField FOLLOWER_INDEX_FIELD = new ParseField("follower_index"); + private static final ParseField MAX_BATCH_OPERATION_COUNT = new ParseField("max_batch_operation_count"); + private static final ParseField MAX_CONCURRENT_READ_BATCHES = new ParseField("max_concurrent_read_batches"); + private static final ParseField MAX_BATCH_SIZE_IN_BYTES = new ParseField("max_batch_size_in_bytes"); + private static final ParseField MAX_CONCURRENT_WRITE_BATCHES = new ParseField("max_concurrent_write_batches"); + private static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); + private static final ParseField RETRY_TIMEOUT = new ParseField("retry_timeout"); + private static final ParseField IDLE_SHARD_RETRY_DELAY = new ParseField("idle_shard_retry_delay"); + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, true, + (args, followerIndex) -> { + if (args[1] != null) { + followerIndex = (String) args[1]; + } + return new Request((String) args[0], followerIndex, (Integer) args[2], (Integer) args[3], (Long) args[4], + (Integer) args[5], (Integer) args[6], (TimeValue) args[7], (TimeValue) args[8]); + }); + + static { + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), LEADER_INDEX_FIELD); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), FOLLOWER_INDEX_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); + } + + public static Request fromXContent(final XContentParser parser, final String followerIndex) throws IOException { + Request request = PARSER.parse(parser, followerIndex); + if (followerIndex != null) { + if (request.followerIndex == null) { + request.followerIndex = followerIndex; + } else { + if (request.followerIndex.equals(followerIndex) == false) { + throw new IllegalArgumentException("provided follower_index is not equal"); + } + } + } + return request; + } + + private String leaderIndex; + + public String getLeaderIndex() { + return leaderIndex; + } + + + private String followerIndex; + + public String getFollowerIndex() { + return followerIndex; + } + + private int maxBatchOperationCount; + + public int getMaxBatchOperationCount() { + return maxBatchOperationCount; + } + + private int maxConcurrentReadBatches; + + public int getMaxConcurrentReadBatches() { + return maxConcurrentReadBatches; + } + + private long maxOperationSizeInBytes; + + public long getMaxOperationSizeInBytes() { + return maxOperationSizeInBytes; + } + + private int maxConcurrentWriteBatches; + + public int getMaxConcurrentWriteBatches() { + return maxConcurrentWriteBatches; + } + + private int maxWriteBufferSize; + + public int getMaxWriteBufferSize() { + return maxWriteBufferSize; + } + + private TimeValue retryTimeout; + + public TimeValue getRetryTimeout() { + return retryTimeout; + } + + private TimeValue idleShardRetryDelay; + + public TimeValue getIdleShardRetryDelay() { + return idleShardRetryDelay; + } + + public Request( + final String leaderIndex, + final String followerIndex, + final Integer maxBatchOperationCount, + final Integer maxConcurrentReadBatches, + final Long maxOperationSizeInBytes, + final Integer maxConcurrentWriteBatches, + final Integer maxWriteBufferSize, + final TimeValue retryTimeout, + final TimeValue idleShardRetryDelay) { + + if (leaderIndex == null) { + throw new IllegalArgumentException(LEADER_INDEX_FIELD.getPreferredName() + " is missing"); + } + + if (followerIndex == null) { + throw new IllegalArgumentException(FOLLOWER_INDEX_FIELD.getPreferredName() + " is missing"); + } + + final int actualMaxBatchOperationCount = + maxBatchOperationCount == null ? DEFAULT_MAX_BATCH_OPERATION_COUNT : maxBatchOperationCount; + if (actualMaxBatchOperationCount < 1) { + throw new IllegalArgumentException(MAX_BATCH_OPERATION_COUNT.getPreferredName() + " must be larger than 0"); + } + + final int actualMaxConcurrentReadBatches = + maxConcurrentReadBatches == null ? DEFAULT_MAX_CONCURRENT_READ_BATCHES : maxConcurrentReadBatches; + if (actualMaxConcurrentReadBatches < 1) { + throw new IllegalArgumentException(MAX_CONCURRENT_READ_BATCHES.getPreferredName() + " must be larger than 0"); + } + + final long actualMaxOperationSizeInBytes = + maxOperationSizeInBytes == null ? DEFAULT_MAX_BATCH_SIZE_IN_BYTES : maxOperationSizeInBytes; + if (actualMaxOperationSizeInBytes <= 0) { + throw new IllegalArgumentException(MAX_BATCH_SIZE_IN_BYTES.getPreferredName() + " must be larger than 0"); + } + + final int actualMaxConcurrentWriteBatches = + maxConcurrentWriteBatches == null ? DEFAULT_MAX_CONCURRENT_WRITE_BATCHES : maxConcurrentWriteBatches; + if (actualMaxConcurrentWriteBatches < 1) { + throw new IllegalArgumentException(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName() + " must be larger than 0"); + } + + final int actualMaxWriteBufferSize = maxWriteBufferSize == null ? DEFAULT_MAX_WRITE_BUFFER_SIZE : maxWriteBufferSize; + if (actualMaxWriteBufferSize < 1) { + throw new IllegalArgumentException(MAX_WRITE_BUFFER_SIZE.getPreferredName() + " must be larger than 0"); + } + + final TimeValue actualRetryTimeout = retryTimeout == null ? DEFAULT_RETRY_TIMEOUT : retryTimeout; + final TimeValue actualIdleShardRetryDelay = idleShardRetryDelay == null ? DEFAULT_IDLE_SHARD_RETRY_DELAY : idleShardRetryDelay; + + this.leaderIndex = leaderIndex; + this.followerIndex = followerIndex; + this.maxBatchOperationCount = actualMaxBatchOperationCount; + this.maxConcurrentReadBatches = actualMaxConcurrentReadBatches; + this.maxOperationSizeInBytes = actualMaxOperationSizeInBytes; + this.maxConcurrentWriteBatches = actualMaxConcurrentWriteBatches; + this.maxWriteBufferSize = actualMaxWriteBufferSize; + this.retryTimeout = actualRetryTimeout; + this.idleShardRetryDelay = actualIdleShardRetryDelay; + } + + public Request() { + + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void readFrom(final StreamInput in) throws IOException { + super.readFrom(in); + leaderIndex = in.readString(); + followerIndex = in.readString(); + maxBatchOperationCount = in.readVInt(); + maxConcurrentReadBatches = in.readVInt(); + maxOperationSizeInBytes = in.readVLong(); + maxConcurrentWriteBatches = in.readVInt(); + maxWriteBufferSize = in.readVInt(); + retryTimeout = in.readOptionalTimeValue(); + idleShardRetryDelay = in.readOptionalTimeValue(); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(leaderIndex); + out.writeString(followerIndex); + out.writeVInt(maxBatchOperationCount); + out.writeVInt(maxConcurrentReadBatches); + out.writeVLong(maxOperationSizeInBytes); + out.writeVInt(maxConcurrentWriteBatches); + out.writeVInt(maxWriteBufferSize); + out.writeOptionalTimeValue(retryTimeout); + out.writeOptionalTimeValue(idleShardRetryDelay); + } + + @Override + public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException { + builder.startObject(); + { + builder.field(LEADER_INDEX_FIELD.getPreferredName(), leaderIndex); + builder.field(FOLLOWER_INDEX_FIELD.getPreferredName(), followerIndex); + builder.field(MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); + builder.field(MAX_BATCH_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); + builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize); + builder.field(MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); + builder.field(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + builder.field(RETRY_TIMEOUT.getPreferredName(), retryTimeout.getStringRep()); + builder.field(IDLE_SHARD_RETRY_DELAY.getPreferredName(), idleShardRetryDelay.getStringRep()); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return maxBatchOperationCount == request.maxBatchOperationCount && + maxConcurrentReadBatches == request.maxConcurrentReadBatches && + maxOperationSizeInBytes == request.maxOperationSizeInBytes && + maxConcurrentWriteBatches == request.maxConcurrentWriteBatches && + maxWriteBufferSize == request.maxWriteBufferSize && + Objects.equals(retryTimeout, request.retryTimeout) && + Objects.equals(idleShardRetryDelay, request.idleShardRetryDelay) && + Objects.equals(leaderIndex, request.leaderIndex) && + Objects.equals(followerIndex, request.followerIndex); + } + + @Override + public int hashCode() { + return Objects.hash( + leaderIndex, + followerIndex, + maxBatchOperationCount, + maxConcurrentReadBatches, + maxOperationSizeInBytes, + maxConcurrentWriteBatches, + maxWriteBufferSize, + retryTimeout, + idleShardRetryDelay + ); + } + } + +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/UnfollowIndexAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/UnfollowIndexAction.java new file mode 100644 index 0000000000000..65ecd3dad2f8f --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/UnfollowIndexAction.java @@ -0,0 +1,62 @@ +/* + * 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.action; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +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; + +public class UnfollowIndexAction extends Action { + + public static final UnfollowIndexAction INSTANCE = new UnfollowIndexAction(); + public static final String NAME = "cluster:admin/xpack/ccr/unfollow_index"; + + private UnfollowIndexAction() { + super(NAME); + } + + @Override + public AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + public static class Request extends ActionRequest { + + private String followIndex; + + public String getFollowIndex() { + return followIndex; + } + + public void setFollowIndex(final String followIndex) { + this.followIndex = followIndex; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void readFrom(final StreamInput in) throws IOException { + super.readFrom(in); + followIndex = in.readString(); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(followIndex); + } + } + +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/client/CcrClient.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/client/CcrClient.java new file mode 100644 index 0000000000000..e880b38482460 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/client/CcrClient.java @@ -0,0 +1,73 @@ +/* + * 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.client; + +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.ElasticsearchClient; +import org.elasticsearch.xpack.core.ccr.action.CcrStatsAction; +import org.elasticsearch.xpack.core.ccr.action.CreateAndFollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.FollowIndexAction; +import org.elasticsearch.xpack.core.ccr.action.UnfollowIndexAction; + +import java.util.Objects; + +public class CcrClient { + + private final ElasticsearchClient client; + + public CcrClient(final ElasticsearchClient client) { + this.client = Objects.requireNonNull(client, "client"); + } + + public void createAndFollow( + final CreateAndFollowIndexAction.Request request, + final ActionListener listener) { + client.execute(CreateAndFollowIndexAction.INSTANCE, request, listener); + } + + public ActionFuture createAndFollow(final CreateAndFollowIndexAction.Request request) { + final PlainActionFuture listener = PlainActionFuture.newFuture(); + client.execute(CreateAndFollowIndexAction.INSTANCE, request, listener); + return listener; + } + + public void follow(final FollowIndexAction.Request request, final ActionListener listener) { + client.execute(FollowIndexAction.INSTANCE, request, listener); + } + + public ActionFuture follow(final FollowIndexAction.Request request) { + final PlainActionFuture listener = PlainActionFuture.newFuture(); + client.execute(FollowIndexAction.INSTANCE, request, listener); + return listener; + } + + public void stats( + final CcrStatsAction.TasksRequest request, + final ActionListener listener) { + client.execute(CcrStatsAction.INSTANCE, request, listener); + } + + public ActionFuture stats(final CcrStatsAction.TasksRequest request) { + final PlainActionFuture listener = PlainActionFuture.newFuture(); + client.execute(CcrStatsAction.INSTANCE, request, listener); + return listener; + } + + public void unfollow(final UnfollowIndexAction.Request request, final ActionListener listener) { + client.execute(UnfollowIndexAction.INSTANCE, request, listener); + } + + public ActionFuture unfollow(final UnfollowIndexAction.Request request) { + final PlainActionFuture listener = PlainActionFuture.newFuture(); + client.execute(UnfollowIndexAction.INSTANCE, request, listener); + return listener; + } + +}