From 516dcb7a9455beec699c7f9c29e15058a6efb660 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 22 Jun 2018 10:55:29 +0200 Subject: [PATCH 01/41] Rewrite shard follow node task logic The current shard follow mechanism is complex and does not give us easy ways the have visibility into the system (e.g. why we are falling behind). The main reason why it is complex is because the current design is highly asynchronous. Also in the current model it is hard to apply backpressure other than reducing the concurrent reads from the leader shard. This PR has the following changes: * Rewrote the shard follow task to coordinate the shard follow mechanism between a leader and follow shard in a single threaded manner. This allows for better unit testing and makes it easier to add stats. * All write operations read from the shard changes api should be added to a buffer instead of directly sending it to the bulk shard operations api. This allows to apply backpressure. In this PR there is a limit that controls how many write ops are allowed in the buffer after which no new reads will be performed until the number of ops is below that limit. * The shard changes api includes the current global checkpoint on the leader shard copy. This allows reading to be a more self sufficient process; instead of relying on a background thread to fetch the leader shard's global checkpoint. * Reading write operations from the leader shard (via shard changes api) is a seperate step then writing the write operations (via bulk shards operations api). Whereas before a read would immediately result into a write. * The bulk shard operations api returns the local checkpoint on the follow primary shard, to keep the shard follow task up to date with what has been written. * Moved the shard follow logic that was previously in ShardFollowTasksExecutor to ShardFollowNodeTask. * Moved over the changes from #31242 to make shard follow mechanism resilient from node and shard failures. Relates to #30086 --- .../xpack/ccr/action/FollowIndexAction.java | 87 +++- .../xpack/ccr/action/ShardChangesAction.java | 52 ++- .../xpack/ccr/action/ShardFollowNodeTask.java | 386 +++++++++++++++- .../xpack/ccr/action/ShardFollowTask.java | 157 +++++-- .../ccr/action/ShardFollowTasksExecutor.java | 418 +----------------- .../bulk/BulkShardOperationsResponse.java | 27 ++ .../TransportBulkShardOperationsAction.java | 3 +- .../xpack/ccr/rest/RestFollowIndexAction.java | 18 +- .../xpack/ccr/ShardChangesIT.java | 242 ++++++++-- .../ccr/action/ChunksCoordinatorTests.java | 407 ----------------- .../ccr/action/FollowIndexRequestTests.java | 8 +- .../ccr/action/ShardChangesActionTests.java | 8 +- .../ccr/action/ShardChangesRequestTests.java | 3 +- .../ccr/action/ShardChangesResponseTests.java | 3 +- .../ShardFollowNodeTaskStatusTests.java | 3 +- .../ccr/action/ShardFollowNodeTaskTests.java | 232 ++++++++++ .../ccr/action/ShardFollowTaskTests.java | 15 +- 17 files changed, 1108 insertions(+), 961 deletions(-) delete mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ChunksCoordinatorTests.java create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskTests.java 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 index 21c56b53f2e62..499d93023e459 100644 --- 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 @@ -71,9 +71,12 @@ public static class Request extends ActionRequest { private String leaderIndex; private String followIndex; - private long batchSize = ShardFollowTasksExecutor.DEFAULT_BATCH_SIZE; - private int concurrentProcessors = ShardFollowTasksExecutor.DEFAULT_CONCURRENT_PROCESSORS; - private long processorMaxTranslogBytes = ShardFollowTasksExecutor.DEFAULT_MAX_TRANSLOG_BYTES; + private int maxReadSize = ShardFollowNodeTask.DEFAULT_MAX_READ_SIZE; + private int maxConcurrentReads = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS; + private long processorMaxTranslogBytes = ShardFollowNodeTask.DEFAULT_MAX_TRANSLOG_BYTES; + private int maxWriteSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE; + private int maxConcurrentWrites = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES; + private int maxBufferSize = ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE; public String getLeaderIndex() { return leaderIndex; @@ -91,23 +94,23 @@ public void setFollowIndex(String followIndex) { this.followIndex = followIndex; } - public long getBatchSize() { - return batchSize; + public int getMaxReadSize() { + return maxReadSize; } - public void setBatchSize(long batchSize) { - if (batchSize < 1) { - throw new IllegalArgumentException("Illegal batch_size [" + batchSize + "]"); + public void setMaxReadSize(int maxReadSize) { + if (maxReadSize < 1) { + throw new IllegalArgumentException("Illegal batch_size [" + maxReadSize + "]"); } - this.batchSize = batchSize; + this.maxReadSize = maxReadSize; } - public void setConcurrentProcessors(int concurrentProcessors) { - if (concurrentProcessors < 1) { + public void setMaxConcurrentReads(int maxConcurrentReads) { + if (maxConcurrentReads < 1) { throw new IllegalArgumentException("concurrent_processors must be larger than 0"); } - this.concurrentProcessors = concurrentProcessors; + this.maxConcurrentReads = maxConcurrentReads; } public void setProcessorMaxTranslogBytes(long processorMaxTranslogBytes) { @@ -117,6 +120,39 @@ public void setProcessorMaxTranslogBytes(long processorMaxTranslogBytes) { this.processorMaxTranslogBytes = processorMaxTranslogBytes; } + public int getMaxWriteSize() { + return maxWriteSize; + } + + public void setMaxWriteSize(int maxWriteSize) { + if (maxWriteSize < 1) { + throw new IllegalArgumentException("maxWriteSize must be larger than 0"); + } + this.maxWriteSize = maxWriteSize; + } + + public int getMaxConcurrentWrites() { + return maxConcurrentWrites; + } + + public void setMaxConcurrentWrites(int maxConcurrentWrites) { + if (maxConcurrentWrites < 1) { + throw new IllegalArgumentException("maxConcurrentWrites must be larger than 0"); + } + this.maxConcurrentWrites = maxConcurrentWrites; + } + + public int getMaxBufferSize() { + return maxBufferSize; + } + + public void setMaxBufferSize(int maxBufferSize) { + if (maxBufferSize < 1) { + throw new IllegalArgumentException("maxBufferSize must be larger than 0"); + } + this.maxBufferSize = maxBufferSize; + } + @Override public ActionRequestValidationException validate() { return null; @@ -127,9 +163,12 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); leaderIndex = in.readString(); followIndex = in.readString(); - batchSize = in.readVLong(); - concurrentProcessors = in.readVInt(); + maxReadSize = in.readVInt(); + maxConcurrentReads = in.readVInt(); processorMaxTranslogBytes = in.readVLong(); + maxWriteSize = in.readVInt(); + maxConcurrentWrites = in.readVInt(); + maxBufferSize = in.readVInt(); } @Override @@ -137,9 +176,12 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(leaderIndex); out.writeString(followIndex); - out.writeVLong(batchSize); - out.writeVInt(concurrentProcessors); + out.writeVInt(maxReadSize); + out.writeVInt(maxConcurrentReads); out.writeVLong(processorMaxTranslogBytes); + out.writeVInt(maxWriteSize); + out.writeVInt(maxConcurrentWrites); + out.writeVInt(maxBufferSize); } @Override @@ -147,16 +189,20 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; - return batchSize == request.batchSize && - concurrentProcessors == request.concurrentProcessors && + return maxReadSize == request.maxReadSize && + maxConcurrentReads == request.maxConcurrentReads && processorMaxTranslogBytes == request.processorMaxTranslogBytes && + maxWriteSize == request.maxWriteSize && + maxConcurrentWrites == request.maxConcurrentWrites && + maxBufferSize == request.maxBufferSize && Objects.equals(leaderIndex, request.leaderIndex) && Objects.equals(followIndex, request.followIndex); } @Override public int hashCode() { - return Objects.hash(leaderIndex, followIndex, batchSize, concurrentProcessors, processorMaxTranslogBytes); + return Objects.hash(leaderIndex, followIndex, maxReadSize, maxConcurrentReads, processorMaxTranslogBytes, + maxWriteSize, maxConcurrentWrites, maxBufferSize); } } @@ -254,7 +300,8 @@ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMe ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, new ShardId(followIndexMetadata.getIndex(), shardId), new ShardId(leaderIndexMetadata.getIndex(), shardId), - request.batchSize, request.concurrentProcessors, request.processorMaxTranslogBytes, filteredHeaders); + request.maxReadSize, request.maxConcurrentReads, request.processorMaxTranslogBytes, + request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, new ActionListener>() { @Override 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 c72b13a701f41..b746f8dcd4225 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 @@ -54,9 +54,9 @@ public Response newResponse() { public static class Request extends SingleShardRequest { private long minSeqNo; - private long maxSeqNo; + private Long maxSeqNo; private ShardId shardId; - private long maxTranslogsBytes = ShardFollowTasksExecutor.DEFAULT_MAX_TRANSLOG_BYTES; + private long maxTranslogsBytes = ShardFollowNodeTask.DEFAULT_MAX_TRANSLOG_BYTES; public Request(ShardId shardId) { super(shardId.getIndexName()); @@ -78,11 +78,11 @@ public void setMinSeqNo(long minSeqNo) { this.minSeqNo = minSeqNo; } - public long getMaxSeqNo() { + public Long getMaxSeqNo() { return maxSeqNo; } - public void setMaxSeqNo(long maxSeqNo) { + public void setMaxSeqNo(Long maxSeqNo) { this.maxSeqNo = maxSeqNo; } @@ -100,7 +100,7 @@ public ActionRequestValidationException validate() { if (minSeqNo < 0) { validationException = addValidationError("minSeqNo [" + minSeqNo + "] cannot be lower than 0", validationException); } - if (maxSeqNo < minSeqNo) { + if (maxSeqNo != null && maxSeqNo < minSeqNo) { validationException = addValidationError("minSeqNo [" + minSeqNo + "] cannot be larger than maxSeqNo [" + maxSeqNo + "]", validationException); } @@ -115,7 +115,7 @@ public ActionRequestValidationException validate() { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); minSeqNo = in.readVLong(); - maxSeqNo = in.readVLong(); + maxSeqNo = in.readOptionalLong(); shardId = ShardId.readShardId(in); maxTranslogsBytes = in.readVLong(); } @@ -124,7 +124,7 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeVLong(minSeqNo); - out.writeVLong(maxSeqNo); + out.writeOptionalLong(maxSeqNo); shardId.writeTo(out); out.writeVLong(maxTranslogsBytes); } @@ -136,7 +136,7 @@ public boolean equals(final Object o) { if (o == null || getClass() != o.getClass()) return false; final Request request = (Request) o; return minSeqNo == request.minSeqNo && - maxSeqNo == request.maxSeqNo && + Objects.equals(maxSeqNo, request.maxSeqNo) && Objects.equals(shardId, request.shardId) && maxTranslogsBytes == request.maxTranslogsBytes; } @@ -150,13 +150,15 @@ public int hashCode() { public static final class Response extends ActionResponse { private long indexMetadataVersion; + private long leaderGlobalCheckpoint; private Translog.Operation[] operations; Response() { } - Response(long indexMetadataVersion, final Translog.Operation[] operations) { + Response(long indexMetadataVersion, long leaderGlobalCheckpoint, final Translog.Operation[] operations) { this.indexMetadataVersion = indexMetadataVersion; + this.leaderGlobalCheckpoint = leaderGlobalCheckpoint; this.operations = operations; } @@ -164,6 +166,10 @@ public long getIndexMetadataVersion() { return indexMetadataVersion; } + public long getLeaderGlobalCheckpoint() { + return leaderGlobalCheckpoint; + } + public Translog.Operation[] getOperations() { return operations; } @@ -172,6 +178,7 @@ public Translog.Operation[] getOperations() { public void readFrom(final StreamInput in) throws IOException { super.readFrom(in); indexMetadataVersion = in.readVLong(); + leaderGlobalCheckpoint = in.readZLong(); operations = in.readArray(Translog.Operation::readOperation, Translog.Operation[]::new); } @@ -179,6 +186,7 @@ public void readFrom(final StreamInput in) throws IOException { public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); out.writeVLong(indexMetadataVersion); + out.writeZLong(leaderGlobalCheckpoint); out.writeArray(Translog.Operation::writeOperation, operations); } @@ -188,13 +196,15 @@ public boolean equals(final Object o) { if (o == null || getClass() != o.getClass()) return false; final Response response = (Response) o; return indexMetadataVersion == response.indexMetadataVersion && - Arrays.equals(operations, response.operations); + leaderGlobalCheckpoint == response.leaderGlobalCheckpoint && + Arrays.equals(operations, response.operations); } @Override public int hashCode() { int result = 1; result += Objects.hashCode(indexMetadataVersion); + result += Objects.hashCode(leaderGlobalCheckpoint); result += Arrays.hashCode(operations); return result; } @@ -222,14 +232,16 @@ protected Response shardOperation(Request request, ShardId shardId) throws IOExc IndexService indexService = indicesService.indexServiceSafe(request.getShard().getIndex()); IndexShard indexShard = indexService.getShard(request.getShard().id()); final long indexMetaDataVersion = clusterService.state().metaData().index(shardId.getIndex()).getVersion(); + // The following shard generates this request based on the global checkpoint on the primary copy on the leader. // Although this value might not have been synced to all replica copies on the leader, the requesting range // is guaranteed to be at most the local-checkpoint of any shard copies on the leader. - assert request.maxSeqNo <= indexShard.getLocalCheckpoint() : "invalid request from_seqno=[" + request.minSeqNo + "]," + - " to_seqno=[" + request.maxSeqNo + "], local_checkpoint=[" + indexShard.getLocalCheckpoint() + "]"; + assert request.maxSeqNo == null || request.maxSeqNo <= indexShard.getLocalCheckpoint() : "invalid request from_seqno=[" + + request.minSeqNo + "]," + " to_seqno=[" + request.maxSeqNo + "], local_checkpoint=[" + + indexShard.getLocalCheckpoint() + "]"; final Translog.Operation[] operations = getOperationsBetween(indexShard, request.minSeqNo, request.maxSeqNo, request.maxTranslogsBytes); - return new Response(indexMetaDataVersion, operations); + return new Response(indexMetaDataVersion, indexShard.getGlobalCheckpoint(), operations); } @Override @@ -254,14 +266,15 @@ protected Response newResponse() { private static final Translog.Operation[] EMPTY_OPERATIONS_ARRAY = new Translog.Operation[0]; - static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long minSeqNo, long maxSeqNo, + static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long minSeqNo, Long maxSeqNo, long byteLimit) throws IOException { if (indexShard.state() != IndexShardState.STARTED) { throw new IndexShardNotStartedException(indexShard.shardId(), indexShard.state()); } int seenBytes = 0; final List operations = new ArrayList<>(); - try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", minSeqNo, maxSeqNo, true)) { + long max = maxSeqNo != null ? maxSeqNo : minSeqNo + 1000; + try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", minSeqNo, max, true)) { Translog.Operation op; while ((op = snapshot.next()) != null) { if (op.getSource() == null) { @@ -274,6 +287,15 @@ static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long min break; } } + } catch (IllegalStateException e) { + // TODO: handle peek reads better. + // Should this optional upper bound leak into the newLuceneChangesSnapshot(...) method? + if (maxSeqNo != null) { + throw e; + } else if (e.getMessage().contains("prematurely terminated last_seen_seqno") == false) { + // Only fail if there are gaps between the ops. + throw e; + } } return operations.toArray(EMPTY_OPERATIONS_ARRAY); } 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 8777fd0eabe95..5fcfd44d27fd2 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 @@ -5,28 +5,360 @@ */ package org.elasticsearch.xpack.ccr.action; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.NoShardAvailableActionException; +import org.elasticsearch.action.UnavailableShardsException; +import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MappingMetaData; 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.TimeValue; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.transport.ActionTransportException; +import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; +import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; +import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedList; import java.util.Map; import java.util.Objects; -import java.util.concurrent.atomic.AtomicLong; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.LongConsumer; +/** + * The node task that fetch the write operations from a leader shard and + * persists these ops in the follower shard. + */ public class ShardFollowNodeTask extends AllocatedPersistentTask { - private final AtomicLong processedGlobalCheckpoint = new AtomicLong(); + static final int DEFAULT_MAX_READ_SIZE = 1024; + static final int DEFAULT_MAX_WRITE_SIZE = 1024; + static final int RETRY_LIMIT = 10; + static final int DEFAULT_MAX_CONCURRENT_READS = 1; + static final int DEFAULT_MAX_CONCURRENT_WRITES = 1; + static final int DEFAULT_MAX_BUFFER_SIZE = 10240; + static final long DEFAULT_MAX_TRANSLOG_BYTES= Long.MAX_VALUE; + private static final TimeValue RETRY_TIMEOUT = TimeValue.timeValueMillis(500); + + private static final Logger LOGGER = Loggers.getLogger(ShardFollowNodeTask.class); + + final Client leaderClient; + final Client followerClient; + private final ShardFollowTask params; + private final BiConsumer scheduler; + + private volatile long lastRequestedSeqno; + private volatile long leaderGlobalCheckpoint; + + private volatile int numConcurrentReads = 0; + private volatile int numConcurrentWrites = 0; + private volatile long processedGlobalCheckpoint = 0; + private volatile long currentIndexMetadataVersion = 0; + private final AtomicInteger retryCounter = new AtomicInteger(0); + private final Queue buffer = new LinkedList<>(); - ShardFollowNodeTask(long id, String type, String action, String description, TaskId parentTask, Map headers) { + ShardFollowNodeTask(long id, + String type, + String action, + String description, + TaskId parentTask, + Map headers, + Client leaderClient, + Client followerClient, + ShardFollowTask params, + BiConsumer scheduler) { super(id, type, action, description, parentTask, headers); + this.leaderClient = leaderClient; + this.followerClient = followerClient; + this.params = params; + this.scheduler = scheduler; + } + + void start(long leaderGlobalCheckpoint, long followGlobalCheckpoint) { + this.lastRequestedSeqno = followGlobalCheckpoint; + this.processedGlobalCheckpoint = followGlobalCheckpoint; + this.leaderGlobalCheckpoint = leaderGlobalCheckpoint; + + // Forcefully updates follower mapping, this gets us the leader imd version and + // makes sure that leader and follower mapping are identical. + updateMapping(imdVersion -> { + currentIndexMetadataVersion = imdVersion; + LOGGER.info("{} Started to follow leader shard {}, leaderGlobalCheckPoint={}, followGlobalCheckPoint={}", + params.getFollowShardId(), params.getLeaderShardId(), leaderGlobalCheckpoint, followGlobalCheckpoint); + coordinateReads(); + }); + } + + private synchronized void coordinateReads() { + if (isStopped()) { + LOGGER.info("{} shard follow task has been stopped", params.getFollowShardId()); + return; + } + + LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, leaderGlobalCheckpoint={}", + params.getFollowShardId(), lastRequestedSeqno, leaderGlobalCheckpoint); + final long maxReadSize = params.getMaxReadSize(); + final long maxConcurrentReads = params.getMaxConcurrentReads(); + if (lastRequestedSeqno < leaderGlobalCheckpoint) { + while (true) { + if (lastRequestedSeqno >= leaderGlobalCheckpoint) { + LOGGER.debug("{} no new reads to coordinate lastRequestedSeqno [{}] leaderGlobalCheckpoint [{}]", + params.getLeaderShardId(), lastRequestedSeqno, leaderGlobalCheckpoint); + break; + } + if (numConcurrentReads >= maxConcurrentReads) { + LOGGER.debug("{} no new reads, maximum number of concurrent reads have been reached [{}]", + params.getFollowShardId(), numConcurrentReads); + break; + } + if (buffer.size() > params.getMaxBufferSize()) { + LOGGER.debug("{} no new reads, buffer limit has been reached [{}]", params.getFollowShardId(), buffer.size()); + break; + } + numConcurrentReads++; + long from = lastRequestedSeqno + 1; + long to = from + maxReadSize <= leaderGlobalCheckpoint ? from + maxReadSize : leaderGlobalCheckpoint; + LOGGER.debug("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, to); + sendShardChangesRequest(from, to); + lastRequestedSeqno = to; + } + if (numConcurrentReads == 0) { + LOGGER.debug("{} re-scheduling coordinate reads phase", params.getFollowShardId()); + scheduler.accept(TimeValue.timeValueMillis(500), this::coordinateReads); + } + } else { + if (numConcurrentReads == 0) { + LOGGER.debug("{} scheduling peek read", params.getFollowShardId()); + scheduler.accept(TimeValue.timeValueMillis(500), () -> { + synchronized (this) { + // We sneak peek if there is any thing new in the leader primary. + // If there is we will happily accept + numConcurrentReads++; + long from = lastRequestedSeqno + 1; + LOGGER.debug("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); + sendShardChangesRequest(from, null); + } + }); + } + } + } + + private synchronized void coordinateWrites() { + while (true) { + if (buffer.isEmpty()) { + LOGGER.debug("{} no writes to coordinate, because buffer is empty", params.getFollowShardId()); + break; + } + if (numConcurrentWrites >= params.getMaxConcurrentWrites()) { + LOGGER.debug("{} maximum number of concurrent writes have been reached [{}]", + params.getFollowShardId(), numConcurrentWrites); + break; + } + Translog.Operation[] ops = new Translog.Operation[Math.min(params.getMaxWriteSize(), buffer.size())]; + for (int i = 0; i < ops.length; i++) { + ops[i] = buffer.remove(); + } + numConcurrentWrites++; + LOGGER.debug("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops[0].seqNo(), + ops[ops.length - 1].seqNo(), ops.length); + sendBulkShardOperationsRequest(ops); + } + } + + private void sendShardChangesRequest(long from, Long to) { + innerSendShardChangesRequest(from, to, + response -> { + retryCounter.set(0); + handleResponse(from, to, response); + }, + e -> handleFailure(e, () -> sendShardChangesRequest(from, to))); + } + + private synchronized void handleResponse(long from, Long to, ShardChangesAction.Response response) { + maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { + synchronized (ShardFollowNodeTask.this) { + leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getLeaderGlobalCheckpoint()); + if (response.getOperations().length == 0) { + numConcurrentReads--; + if (numConcurrentWrites == 0) { + coordinateWrites(); + } + coordinateReads(); + } else { + Translog.Operation firstOp = response.getOperations()[0]; + assert firstOp.seqNo() == from; + Translog.Operation lastOp = response.getOperations()[response.getOperations().length - 1]; + + LOGGER.debug("{} received [{}/{}]", params.getFollowShardId(), firstOp.seqNo(), lastOp.seqNo()); + buffer.addAll(Arrays.asList(response.getOperations())); + if (to == null) { + lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOp.seqNo()); + LOGGER.debug("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); + numConcurrentReads--; + } else { + if (lastOp.seqNo() < to) { + long newFrom = lastOp.seqNo() + 1; + LOGGER.debug("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", + params.getFollowShardId(), lastOp.seqNo(), to, newFrom, to); + sendShardChangesRequest(newFrom, to); + } else { + numConcurrentReads--; + } + } + if (numConcurrentWrites == 0) { + coordinateWrites(); + } + coordinateReads(); + } + assert numConcurrentReads >= 0; + } + }); + } + + private void sendBulkShardOperationsRequest(Translog.Operation[] operations) { + innerSendBulkShardOperationsRequest(operations, + followerLocalCheckpoint -> { + retryCounter.set(0); + handleResponse(followerLocalCheckpoint); + }, + e -> handleFailure(e, () -> sendBulkShardOperationsRequest(operations)) + ); + } + + private synchronized void handleResponse(long followerLocalCheckpoint) { + processedGlobalCheckpoint = Math.max(processedGlobalCheckpoint, followerLocalCheckpoint); + numConcurrentWrites--; + assert numConcurrentWrites >= 0; + coordinateWrites(); + } + + private void maybeUpdateMapping(Long minimumRequiredIndexMetadataVersion, Runnable task) { + assert Thread.holdsLock(this); + if (currentIndexMetadataVersion >= minimumRequiredIndexMetadataVersion) { + LOGGER.trace("{} index metadata version [{}] is higher or equal than minimum required index metadata version [{}]", + params.getFollowShardId(), currentIndexMetadataVersion, minimumRequiredIndexMetadataVersion); + task.run(); + } else { + LOGGER.debug("{} updating mapping, index metadata version [{}] is lower than minimum required index metadata version [{}]", + params.getFollowShardId(), currentIndexMetadataVersion, minimumRequiredIndexMetadataVersion); + updateMapping(imdVersion -> { + retryCounter.set(0); + currentIndexMetadataVersion = imdVersion; + task.run(); + }); + } + } + + private void handleFailure(Exception e, Runnable task) { + assert e != null; + if (shouldRetry(e)) { + if (isStopped() == false && retryCounter.incrementAndGet() <= RETRY_LIMIT) { + LOGGER.warn("error during follow shard task, retrying...", e); + scheduler.accept(RETRY_TIMEOUT, task); + } else { + markAsFailed(new ElasticsearchException("retrying failed [" + retryCounter.get() + + "] times, aborting...", e)); + } + } else { + markAsFailed(e); + } + } + + private boolean shouldRetry(Exception e) { + // TODO: What other exceptions should be retried? + return NetworkExceptionHelper.isConnectException(e) || + NetworkExceptionHelper.isCloseConnectionException(e) || + e instanceof ActionTransportException || + e instanceof NodeClosedException || + e instanceof UnavailableShardsException || + e instanceof NoShardAvailableActionException; + } + + // These methods are protected for testing purposes: + protected void updateMapping(LongConsumer handler) { + Index leaderIndex = params.getLeaderShardId().getIndex(); + Index followIndex = params.getFollowShardId().getIndex(); + + ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.clear(); + clusterStateRequest.metaData(true); + clusterStateRequest.indices(leaderIndex.getName()); + + leaderClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { + IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().getIndexSafe(leaderIndex); + assert indexMetaData.getMappings().size() == 1; + MappingMetaData mappingMetaData = indexMetaData.getMappings().iterator().next().value; + + PutMappingRequest putMappingRequest = new PutMappingRequest(followIndex.getName()); + putMappingRequest.type(mappingMetaData.type()); + putMappingRequest.source(mappingMetaData.source().string(), XContentType.JSON); + followerClient.admin().indices().putMapping(putMappingRequest, ActionListener.wrap( + putMappingResponse -> handler.accept(indexMetaData.getVersion()), + e -> handleFailure(e, () -> updateMapping(handler)))); + }, e -> handleFailure(e, () -> updateMapping(handler)))); + } + + protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, + LongConsumer handler, + Consumer errorHandler) { + final BulkShardOperationsRequest request = new BulkShardOperationsRequest(params.getFollowShardId(), operations); + followerClient.execute(BulkShardOperationsAction.INSTANCE, request, + new ActionListener() { + @Override + public void onResponse(BulkShardOperationsResponse response) { + handler.accept(response.getLocalCheckpoint()); + } + + @Override + public void onFailure(Exception e) { + errorHandler.accept(e); + } + } + ); + } + + protected void innerSendShardChangesRequest(long from, + Long to, + Consumer handler, + Consumer errorHandler) { + ShardChangesAction.Request request = new ShardChangesAction.Request(params.getLeaderShardId()); + request.setMinSeqNo(from); + request.setMaxSeqNo(to); + request.setMaxTranslogsBytes(params.getMaxTranslogBytes()); + leaderClient.execute(ShardChangesAction.INSTANCE, request, new ActionListener() { + @Override + public void onResponse(ShardChangesAction.Response response) { + handler.accept(response); + } + + @Override + public void onFailure(Exception e) { + errorHandler.accept(e); + } + }); } @Override @@ -34,17 +366,13 @@ protected void onCancelled() { markAsCompleted(); } - public boolean isRunning() { - return isCancelled() == false && isCompleted() == false; - } - - void updateProcessedGlobalCheckpoint(long processedGlobalCheckpoint) { - this.processedGlobalCheckpoint.set(processedGlobalCheckpoint); + protected boolean isStopped() { + return isCancelled() || isCompleted(); } @Override - public Task.Status getStatus() { - return new Status(processedGlobalCheckpoint.get()); + public Status getStatus() { + return new Status(processedGlobalCheckpoint, numConcurrentReads, numConcurrentWrites); } public static class Status implements Task.Status { @@ -52,28 +380,46 @@ public static class Status implements Task.Status { public static final String NAME = "shard-follow-node-task-status"; static final ParseField PROCESSED_GLOBAL_CHECKPOINT_FIELD = new ParseField("processed_global_checkpoint"); + 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 ConstructingObjectParser PARSER = - new ConstructingObjectParser<>(NAME, args -> new Status((Long) args[0])); + new ConstructingObjectParser<>(NAME, args -> new Status((long) args[0], (int) args[1], (int) args[2])); static { PARSER.declareLong(ConstructingObjectParser.constructorArg(), PROCESSED_GLOBAL_CHECKPOINT_FIELD); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_READS_FIELD); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_WRITES_FIELD); } private final long processedGlobalCheckpoint; + private final int numberOfConcurrentReads; + private final int numberOfConcurrentWrites; - Status(long processedGlobalCheckpoint) { + Status(long processedGlobalCheckpoint, int numberOfConcurrentReads, int numberOfConcurrentWrites) { this.processedGlobalCheckpoint = processedGlobalCheckpoint; + this.numberOfConcurrentReads = numberOfConcurrentReads; + this.numberOfConcurrentWrites = numberOfConcurrentWrites; } public Status(StreamInput in) throws IOException { this.processedGlobalCheckpoint = in.readZLong(); + this.numberOfConcurrentReads = in.readVInt(); + this.numberOfConcurrentWrites = in.readVInt(); } public long getProcessedGlobalCheckpoint() { return processedGlobalCheckpoint; } + public int getNumberOfConcurrentReads() { + return numberOfConcurrentReads; + } + + public int getNumberOfConcurrentWrites() { + return numberOfConcurrentWrites; + } + @Override public String getWriteableName() { return NAME; @@ -82,6 +428,8 @@ public String getWriteableName() { @Override public void writeTo(StreamOutput out) throws IOException { out.writeZLong(processedGlobalCheckpoint); + out.writeVInt(numberOfConcurrentReads); + out.writeVInt(numberOfConcurrentWrites); } @Override @@ -90,6 +438,12 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws { builder.field(PROCESSED_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), processedGlobalCheckpoint); } + { + builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads); + } + { + builder.field(NUMBER_OF_CONCURRENT_WRITES_FIELD.getPreferredName(), numberOfConcurrentWrites); + } builder.endObject(); return builder; } @@ -103,12 +457,14 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Status status = (Status) o; - return processedGlobalCheckpoint == status.processedGlobalCheckpoint; + return processedGlobalCheckpoint == status.processedGlobalCheckpoint && + numberOfConcurrentReads == status.numberOfConcurrentReads && + numberOfConcurrentWrites == status.numberOfConcurrentWrites; } @Override public int hashCode() { - return Objects.hash(processedGlobalCheckpoint); + return Objects.hash(processedGlobalCheckpoint, numberOfConcurrentReads, numberOfConcurrentWrites); } public String toString() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index e97f8e7dc0c3c..a92172b390708 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -40,15 +40,18 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { static final ParseField LEADER_SHARD_INDEX_UUID_FIELD = new ParseField("leader_shard_index_uuid"); static final ParseField LEADER_SHARD_SHARDID_FIELD = new ParseField("leader_shard_shard"); static final ParseField HEADERS = new ParseField("headers"); - public static final ParseField MAX_CHUNK_SIZE = new ParseField("max_chunk_size"); - public static final ParseField NUM_CONCURRENT_CHUNKS = new ParseField("max_concurrent_chunks"); - public static final ParseField PROCESSOR_MAX_TRANSLOG_BYTES_PER_REQUEST = new ParseField("processor_max_translog_bytes"); + public static final ParseField MAX_READ_SIZE = new ParseField("max_read_size"); + public static final ParseField MAX_CONCURRENT_READS = new ParseField("max_concurrent_reads"); + public static final ParseField MAX_TRANSLOG_BYTES_PER_REQUEST = new ParseField("max_translog_bytes"); + public static final ParseField MAX_WRITE_SIZE = new ParseField("max_write_size"); + public static final ParseField MAX_CONCURRENT_WRITES = new ParseField("max_concurrent_writes"); + public static final ParseField MAX_BUFFER_SIZE = new ParseField("max_buffer_size"); @SuppressWarnings("unchecked") - public static ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, + private static ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, (a) -> new ShardFollowTask((String) a[0], new ShardId((String) a[1], (String) a[2], (int) a[3]), - new ShardId((String) a[4], (String) a[5], (int) a[6]), (long) a[7], (int) a[8], (long) a[9], - (Map) a[10])); + new ShardId((String) a[4], (String) a[5], (int) a[6]), (int) a[7], (int) a[8], (long) a[9], + (int) a[10], (int) a[11], (int) a[12], (Map) a[13])); static { PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), LEADER_CLUSTER_ALIAS_FIELD); @@ -58,28 +61,38 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_SHARD_INDEX_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_SHARD_INDEX_UUID_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), LEADER_SHARD_SHARDID_FIELD); - PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_CHUNK_SIZE); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUM_CONCURRENT_CHUNKS); - PARSER.declareLong(ConstructingObjectParser.constructorArg(), PROCESSOR_MAX_TRANSLOG_BYTES_PER_REQUEST); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_READ_SIZE); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READS); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_TRANSLOG_BYTES_PER_REQUEST); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_SIZE); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITES); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_BUFFER_SIZE); PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> p.mapStrings(), HEADERS); } private final String leaderClusterAlias; private final ShardId followShardId; private final ShardId leaderShardId; - private final long maxChunkSize; - private final int numConcurrentChunks; - private final long processorMaxTranslogBytes; + private final int maxReadSize; + private final int maxConcurrentReads; + private final long maxTranslogBytes; + private final int maxWriteSize; + private final int maxConcurrentWrites; + private final int maxBufferSize; private final Map headers; - ShardFollowTask(String leaderClusterAlias, ShardId followShardId, ShardId leaderShardId, long maxChunkSize, - int numConcurrentChunks, long processorMaxTranslogBytes, Map headers) { + ShardFollowTask(String leaderClusterAlias, ShardId followShardId, ShardId leaderShardId, int maxReadSize, + int maxConcurrentReads, long maxTranslogBytes, int maxWriteSize, int maxConcurrentWrites, + int maxBufferSize, Map headers) { this.leaderClusterAlias = leaderClusterAlias; this.followShardId = followShardId; this.leaderShardId = leaderShardId; - this.maxChunkSize = maxChunkSize; - this.numConcurrentChunks = numConcurrentChunks; - this.processorMaxTranslogBytes = processorMaxTranslogBytes; + this.maxReadSize = maxReadSize; + this.maxConcurrentReads = maxConcurrentReads; + this.maxTranslogBytes = maxTranslogBytes; + this.maxWriteSize = maxWriteSize; + this.maxConcurrentWrites = maxConcurrentWrites; + this.maxBufferSize = maxBufferSize; this.headers = headers != null ? Collections.unmodifiableMap(headers) : Collections.emptyMap(); } @@ -87,9 +100,12 @@ public ShardFollowTask(StreamInput in) throws IOException { this.leaderClusterAlias = in.readOptionalString(); this.followShardId = ShardId.readShardId(in); this.leaderShardId = ShardId.readShardId(in); - this.maxChunkSize = in.readVLong(); - this.numConcurrentChunks = in.readVInt(); - this.processorMaxTranslogBytes = in.readVLong(); + this.maxReadSize = in.readVInt(); + this.maxConcurrentReads = in.readVInt(); + this.maxTranslogBytes = in.readVLong(); + this.maxWriteSize = in.readVInt(); + this.maxConcurrentWrites= in.readVInt(); + this.maxBufferSize = in.readVInt(); this.headers = Collections.unmodifiableMap(in.readMap(StreamInput::readString, StreamInput::readString)); } @@ -105,16 +121,28 @@ public ShardId getLeaderShardId() { return leaderShardId; } - public long getMaxChunkSize() { - return maxChunkSize; + public int getMaxReadSize() { + return maxReadSize; } - public int getNumConcurrentChunks() { - return numConcurrentChunks; + public int getMaxWriteSize() { + return maxWriteSize; } - public long getProcessorMaxTranslogBytes() { - return processorMaxTranslogBytes; + public int getMaxConcurrentReads() { + return maxConcurrentReads; + } + + public int getMaxConcurrentWrites() { + return maxConcurrentWrites; + } + + public int getMaxBufferSize() { + return maxBufferSize; + } + + public long getMaxTranslogBytes() { + return maxTranslogBytes; } public Map getHeaders() { @@ -131,9 +159,12 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(leaderClusterAlias); followShardId.writeTo(out); leaderShardId.writeTo(out); - out.writeVLong(maxChunkSize); - out.writeVInt(numConcurrentChunks); - out.writeVLong(processorMaxTranslogBytes); + out.writeVLong(maxReadSize); + out.writeVInt(maxConcurrentReads); + out.writeVLong(maxTranslogBytes); + out.writeVInt(maxWriteSize); + out.writeVInt(maxConcurrentWrites); + out.writeVInt(maxBufferSize); out.writeMap(headers, StreamOutput::writeString, StreamOutput::writeString); } @@ -144,19 +175,50 @@ public static ShardFollowTask fromXContent(XContentParser parser) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - if (leaderClusterAlias != null) { - builder.field(LEADER_CLUSTER_ALIAS_FIELD.getPreferredName(), leaderClusterAlias); + { + if (leaderClusterAlias != null) { + builder.field(LEADER_CLUSTER_ALIAS_FIELD.getPreferredName(), leaderClusterAlias); + } + } + { + builder.field(FOLLOW_SHARD_INDEX_FIELD.getPreferredName(), followShardId.getIndex().getName()); + } + { + builder.field(FOLLOW_SHARD_INDEX_UUID_FIELD.getPreferredName(), followShardId.getIndex().getUUID()); + } + { + builder.field(FOLLOW_SHARD_SHARDID_FIELD.getPreferredName(), followShardId.id()); + } + { + builder.field(LEADER_SHARD_INDEX_FIELD.getPreferredName(), leaderShardId.getIndex().getName()); + } + { + builder.field(LEADER_SHARD_INDEX_UUID_FIELD.getPreferredName(), leaderShardId.getIndex().getUUID()); + } + { + builder.field(LEADER_SHARD_SHARDID_FIELD.getPreferredName(), leaderShardId.id()); + } + { + builder.field(MAX_READ_SIZE.getPreferredName(), maxReadSize); + } + { + builder.field(MAX_CONCURRENT_READS.getPreferredName(), maxConcurrentReads); + } + { + builder.field(MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName(), maxTranslogBytes); + } + { + builder.field(MAX_WRITE_SIZE.getPreferredName(), maxWriteSize); + } + { + builder.field(MAX_CONCURRENT_WRITES.getPreferredName(), maxConcurrentWrites); + } + { + builder.field(MAX_BUFFER_SIZE.getPreferredName(), maxBufferSize); + } + { + builder.field(HEADERS.getPreferredName(), headers); } - builder.field(FOLLOW_SHARD_INDEX_FIELD.getPreferredName(), followShardId.getIndex().getName()); - builder.field(FOLLOW_SHARD_INDEX_UUID_FIELD.getPreferredName(), followShardId.getIndex().getUUID()); - builder.field(FOLLOW_SHARD_SHARDID_FIELD.getPreferredName(), followShardId.id()); - builder.field(LEADER_SHARD_INDEX_FIELD.getPreferredName(), leaderShardId.getIndex().getName()); - builder.field(LEADER_SHARD_INDEX_UUID_FIELD.getPreferredName(), leaderShardId.getIndex().getUUID()); - builder.field(LEADER_SHARD_SHARDID_FIELD.getPreferredName(), leaderShardId.id()); - builder.field(MAX_CHUNK_SIZE.getPreferredName(), maxChunkSize); - builder.field(NUM_CONCURRENT_CHUNKS.getPreferredName(), numConcurrentChunks); - builder.field(PROCESSOR_MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName(), processorMaxTranslogBytes); - builder.field(HEADERS.getPreferredName(), headers); return builder.endObject(); } @@ -168,16 +230,19 @@ public boolean equals(Object o) { return Objects.equals(leaderClusterAlias, that.leaderClusterAlias) && Objects.equals(followShardId, that.followShardId) && Objects.equals(leaderShardId, that.leaderShardId) && - maxChunkSize == that.maxChunkSize && - numConcurrentChunks == that.numConcurrentChunks && - processorMaxTranslogBytes == that.processorMaxTranslogBytes && + maxReadSize == that.maxReadSize && + maxConcurrentReads == that.maxConcurrentReads && + maxWriteSize == that.maxWriteSize && + maxConcurrentWrites == that.maxConcurrentWrites && + maxTranslogBytes == that.maxTranslogBytes && + maxBufferSize == that.maxBufferSize && Objects.equals(headers, that.headers); } @Override public int hashCode() { - return Objects.hash(leaderClusterAlias, followShardId, leaderShardId, maxChunkSize, numConcurrentChunks, - processorMaxTranslogBytes, headers); + return Objects.hash(leaderClusterAlias, followShardId, leaderShardId, maxReadSize, maxConcurrentReads, + maxWriteSize, maxConcurrentWrites, maxTranslogBytes, maxBufferSize, headers); } public String toString() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 895f53846f65a..16d5cfcb69ccc 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -5,13 +5,8 @@ */ package org.elasticsearch.xpack.ccr.action; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; -import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; @@ -21,20 +16,11 @@ import org.elasticsearch.client.Client; import org.elasticsearch.client.FilterClient; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.NetworkExceptionHelper; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.Index; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.translog.Translog; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -42,20 +28,10 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.Ccr; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; import java.util.Arrays; import java.util.Map; import java.util.Optional; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.Executor; -import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.LongConsumer; @@ -64,12 +40,6 @@ public class ShardFollowTasksExecutor extends PersistentTasksExecutor { - static final long DEFAULT_BATCH_SIZE = 1024; - static final int PROCESSOR_RETRY_LIMIT = 16; - static final int DEFAULT_CONCURRENT_PROCESSORS = 1; - static final long DEFAULT_MAX_TRANSLOG_BYTES= Long.MAX_VALUE; - private static final TimeValue RETRY_TIMEOUT = TimeValue.timeValueMillis(500); - private final Client client; private final ThreadPool threadPool; @@ -100,86 +70,34 @@ public void validate(ShardFollowTask params, ClusterState clusterState) { protected AllocatedPersistentTask createTask(long id, String type, String action, TaskId parentTaskId, PersistentTasksCustomMetaData.PersistentTask taskInProgress, Map headers) { - return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers); + ShardFollowTask params = taskInProgress.getParams(); + logger.info("{} Creating node task to track leader shard {}, params [{}]", + params.getFollowShardId(), params.getLeaderShardId(), params); + + final Client leaderClient; + if (params.getLeaderClusterAlias() != null) { + leaderClient = wrapClient(client.getRemoteClusterClient(params.getLeaderClusterAlias()), params); + } else { + leaderClient = wrapClient(client, params); + } + Client followerClient = wrapClient(client, params); + BiConsumer scheduler = + (delay, command) -> threadPool.schedule(delay, Ccr.CCR_THREAD_POOL_NAME, command); + return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers, + leaderClient, followerClient, params, scheduler); } @Override protected void nodeOperation(final AllocatedPersistentTask task, final ShardFollowTask params, final PersistentTaskState state) { ShardFollowNodeTask shardFollowNodeTask = (ShardFollowNodeTask) task; - Client leaderClient = wrapClient(params.getLeaderClusterAlias() != null ? - this.client.getRemoteClusterClient(params.getLeaderClusterAlias()) : this.client, params); - Client followerClient = wrapClient(this.client, params); - IndexMetadataVersionChecker imdVersionChecker = new IndexMetadataVersionChecker(params.getLeaderShardId().getIndex(), - params.getFollowShardId().getIndex(), client, leaderClient); - logger.info("[{}] initial leader mapping with follower mapping syncing", params); - imdVersionChecker.updateMapping(1L /* Force update, version is initially 0L */, e -> { - if (e == null) { - logger.info("Starting shard following [{}]", params); - fetchGlobalCheckpoint(followerClient, params.getFollowShardId(), - followGlobalCheckPoint -> { - shardFollowNodeTask.updateProcessedGlobalCheckpoint(followGlobalCheckPoint); - prepare(leaderClient, followerClient, shardFollowNodeTask, params, followGlobalCheckPoint, imdVersionChecker); - }, task::markAsFailed); - } else { - shardFollowNodeTask.markAsFailed(e); - } - }); - } - - void prepare(Client leaderClient, Client followerClient, ShardFollowNodeTask task, ShardFollowTask params, - long followGlobalCheckPoint, - IndexMetadataVersionChecker imdVersionChecker) { - if (task.isRunning() == false) { - // TODO: need better cancellation control - return; - } - - final ShardId leaderShard = params.getLeaderShardId(); - final ShardId followerShard = params.getFollowShardId(); - fetchGlobalCheckpoint(leaderClient, leaderShard, leaderGlobalCheckPoint -> { - // TODO: check if both indices have the same history uuid - if (leaderGlobalCheckPoint == followGlobalCheckPoint) { - logger.debug("{} no write operations to fetch", followerShard); - retry(leaderClient, followerClient, task, params, followGlobalCheckPoint, imdVersionChecker); - } else { - assert followGlobalCheckPoint < leaderGlobalCheckPoint : "followGlobalCheckPoint [" + followGlobalCheckPoint + - "] is not below leaderGlobalCheckPoint [" + leaderGlobalCheckPoint + "]"; - logger.debug("{} fetching write operations, leaderGlobalCheckPoint={}, followGlobalCheckPoint={}", followerShard, - leaderGlobalCheckPoint, followGlobalCheckPoint); - Executor ccrExecutor = threadPool.executor(Ccr.CCR_THREAD_POOL_NAME); - Consumer handler = e -> { - if (e == null) { - task.updateProcessedGlobalCheckpoint(leaderGlobalCheckPoint); - prepare(leaderClient, followerClient, task, params, leaderGlobalCheckPoint, imdVersionChecker); - } else { - task.markAsFailed(e); - } - }; - ChunksCoordinator coordinator = new ChunksCoordinator(followerClient, leaderClient, ccrExecutor, imdVersionChecker, - params.getMaxChunkSize(), params.getNumConcurrentChunks(), params.getProcessorMaxTranslogBytes(), leaderShard, - followerShard, handler); - coordinator.createChucks(followGlobalCheckPoint, leaderGlobalCheckPoint); - coordinator.start(); - } + logger.info("{} Started to track leader shard {}", params.getFollowShardId(), params.getLeaderShardId()); + fetchGlobalCheckpoint(shardFollowNodeTask.leaderClient, params.getLeaderShardId(), leaderGlobalCheckPoint -> { + fetchGlobalCheckpoint(shardFollowNodeTask.followerClient, params.getFollowShardId(), followGlobalCheckPoint -> { + shardFollowNodeTask.start(leaderGlobalCheckPoint, followGlobalCheckPoint); + }, task::markAsFailed); }, task::markAsFailed); } - private void retry(Client leaderClient, Client followerClient, ShardFollowNodeTask task, ShardFollowTask params, - long followGlobalCheckPoint, - IndexMetadataVersionChecker imdVersionChecker) { - threadPool.schedule(RETRY_TIMEOUT, Ccr.CCR_THREAD_POOL_NAME, new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - task.markAsFailed(e); - } - - @Override - protected void doRun() throws Exception { - prepare(leaderClient, followerClient, task, params, followGlobalCheckPoint, imdVersionChecker); - } - }); - } - private void fetchGlobalCheckpoint(Client client, ShardId shardId, LongConsumer handler, Consumer errorHandler) { client.admin().indices().stats(new IndicesStatsRequest().indices(shardId.getIndexName()), ActionListener.wrap(r -> { IndexStats indexStats = r.getIndex(shardId.getIndexName()); @@ -197,229 +115,7 @@ private void fetchGlobalCheckpoint(Client client, ShardId shardId, LongConsumer }, errorHandler)); } - static class ChunksCoordinator { - - private static final Logger LOGGER = Loggers.getLogger(ChunksCoordinator.class); - - private final Client followerClient; - private final Client leaderClient; - private final Executor ccrExecutor; - private final IndexMetadataVersionChecker imdVersionChecker; - - private final long batchSize; - private final int concurrentProcessors; - private final long processorMaxTranslogBytes; - private final ShardId leaderShard; - private final ShardId followerShard; - private final Consumer handler; - - private final CountDown countDown; - private final Queue chunks = new ConcurrentLinkedQueue<>(); - private final AtomicReference failureHolder = new AtomicReference<>(); - - ChunksCoordinator(Client followerClient, Client leaderClient, Executor ccrExecutor, IndexMetadataVersionChecker imdVersionChecker, - long batchSize, int concurrentProcessors, long processorMaxTranslogBytes, ShardId leaderShard, - ShardId followerShard, Consumer handler) { - this.followerClient = followerClient; - this.leaderClient = leaderClient; - this.ccrExecutor = ccrExecutor; - this.imdVersionChecker = imdVersionChecker; - this.batchSize = batchSize; - this.concurrentProcessors = concurrentProcessors; - this.processorMaxTranslogBytes = processorMaxTranslogBytes; - this.leaderShard = leaderShard; - this.followerShard = followerShard; - this.handler = handler; - this.countDown = new CountDown(concurrentProcessors); - } - - /** - * Creates chunks of the specified range, inclusive. - * - * @param from the lower end of the range (inclusive) - * @param to the upper end of the range (inclusive) - */ - void createChucks(final long from, final long to) { - LOGGER.debug("{} Creating chunks for operation range [{}] to [{}]", leaderShard, from, to); - for (long i = from; i < to; i += batchSize) { - long v2 = i + batchSize <= to ? i + batchSize - 1 : to; - chunks.add(new long[]{i, v2}); - } - } - - void start() { - LOGGER.debug("{} Start coordination of [{}] chunks with [{}] concurrent processors", - leaderShard, chunks.size(), concurrentProcessors); - for (int i = 0; i < concurrentProcessors; i++) { - ccrExecutor.execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - assert e != null; - LOGGER.error(() -> new ParameterizedMessage("{} Failure starting processor", leaderShard), e); - postProcessChuck(e); - } - - @Override - protected void doRun() throws Exception { - processNextChunk(); - } - }); - } - } - - void processNextChunk() { - long[] chunk = chunks.poll(); - if (chunk == null) { - postProcessChuck(null); - return; - } - LOGGER.debug("{} Processing chunk [{}/{}]", leaderShard, chunk[0], chunk[1]); - Consumer processorHandler = e -> { - if (e == null) { - LOGGER.debug("{} Successfully processed chunk [{}/{}]", leaderShard, chunk[0], chunk[1]); - processNextChunk(); - } else { - LOGGER.error(() -> new ParameterizedMessage("{} Failure processing chunk [{}/{}]", - leaderShard, chunk[0], chunk[1]), e); - postProcessChuck(e); - } - }; - ChunkProcessor processor = new ChunkProcessor(leaderClient, followerClient, chunks, ccrExecutor, imdVersionChecker, - leaderShard, followerShard, processorHandler); - processor.start(chunk[0], chunk[1], processorMaxTranslogBytes); - } - - void postProcessChuck(Exception e) { - if (failureHolder.compareAndSet(null, e) == false) { - Exception firstFailure = failureHolder.get(); - firstFailure.addSuppressed(e); - } - if (countDown.countDown()) { - handler.accept(failureHolder.get()); - } - } - - Queue getChunks() { - return chunks; - } - - } - - static class ChunkProcessor { - - private final Client leaderClient; - private final Client followerClient; - private final Queue chunks; - private final Executor ccrExecutor; - private final BiConsumer> indexVersionChecker; - - private final ShardId leaderShard; - private final ShardId followerShard; - private final Consumer handler; - final AtomicInteger retryCounter = new AtomicInteger(0); - - ChunkProcessor(Client leaderClient, Client followerClient, Queue chunks, Executor ccrExecutor, - BiConsumer> indexVersionChecker, - ShardId leaderShard, ShardId followerShard, Consumer handler) { - this.leaderClient = leaderClient; - this.followerClient = followerClient; - this.chunks = chunks; - this.ccrExecutor = ccrExecutor; - this.indexVersionChecker = indexVersionChecker; - this.leaderShard = leaderShard; - this.followerShard = followerShard; - this.handler = handler; - } - - void start(final long from, final long to, final long maxTranslogsBytes) { - ShardChangesAction.Request request = new ShardChangesAction.Request(leaderShard); - // Treat -1 as 0, because shard changes api min_seq_no is inclusive and therefore it doesn't allow a negative min_seq_no - // (If no indexing has happened in leader shard then global checkpoint is -1.) - request.setMinSeqNo(Math.max(0, from)); - request.setMaxSeqNo(to); - request.setMaxTranslogsBytes(maxTranslogsBytes); - leaderClient.execute(ShardChangesAction.INSTANCE, request, new ActionListener() { - @Override - public void onResponse(ShardChangesAction.Response response) { - handleResponse(to, response); - } - - @Override - public void onFailure(Exception e) { - assert e != null; - if (shouldRetry(e)) { - if (retryCounter.incrementAndGet() <= PROCESSOR_RETRY_LIMIT) { - start(from, to, maxTranslogsBytes); - } else { - handler.accept(new ElasticsearchException("retrying failed [" + retryCounter.get() + - "] times, aborting...", e)); - } - } else { - handler.accept(e); - } - } - }); - } - - void handleResponse(final long to, final ShardChangesAction.Response response) { - if (response.getOperations().length != 0) { - Translog.Operation lastOp = response.getOperations()[response.getOperations().length - 1]; - boolean maxByteLimitReached = lastOp.seqNo() < to; - if (maxByteLimitReached) { - // add a new entry to the queue for the operations that couldn't be fetched in the current shard changes api call: - chunks.add(new long[]{lastOp.seqNo() + 1, to}); - } - } - ccrExecutor.execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - assert e != null; - handler.accept(e); - } - - @Override - protected void doRun() throws Exception { - indexVersionChecker.accept(response.getIndexMetadataVersion(), e -> { - if (e != null) { - if (shouldRetry(e) && retryCounter.incrementAndGet() <= PROCESSOR_RETRY_LIMIT) { - handleResponse(to, response); - } else { - handler.accept(new ElasticsearchException("retrying failed [" + retryCounter.get() + - "] times, aborting...", e)); - } - return; - } - final BulkShardOperationsRequest request = new BulkShardOperationsRequest(followerShard, response.getOperations()); - followerClient.execute(BulkShardOperationsAction.INSTANCE, request, - new ActionListener() { - @Override - public void onResponse(final BulkShardOperationsResponse bulkShardOperationsResponse) { - handler.accept(null); - } - - @Override - public void onFailure(final Exception e) { - // No retry mechanism here, because if a failure is being redirected to this place it is considered - // non recoverable. - assert e != null; - handler.accept(e); - } - } - ); - }); - } - }); - } - - boolean shouldRetry(Exception e) { - // TODO: What other exceptions should be retried? - return NetworkExceptionHelper.isConnectException(e) || - NetworkExceptionHelper.isCloseConnectionException(e); - } - - } - - static Client wrapClient(Client client, ShardFollowTask shardFollowTask) { + private static Client wrapClient(Client client, ShardFollowTask shardFollowTask) { if (shardFollowTask.getHeaders().isEmpty()) { return client; } else { @@ -446,76 +142,4 @@ private static ThreadContext.StoredContext stashWithHeaders(ThreadContext thread return storedContext; } - static final class IndexMetadataVersionChecker implements BiConsumer> { - - private static final Logger LOGGER = Loggers.getLogger(IndexMetadataVersionChecker.class); - - private final Client followClient; - private final Client leaderClient; - private final Index leaderIndex; - private final Index followIndex; - private final AtomicLong currentIndexMetadataVersion; - private final Semaphore updateMappingSemaphore = new Semaphore(1); - - IndexMetadataVersionChecker(Index leaderIndex, Index followIndex, Client followClient, Client leaderClient) { - this.followClient = followClient; - this.leaderIndex = leaderIndex; - this.followIndex = followIndex; - this.leaderClient = leaderClient; - this.currentIndexMetadataVersion = new AtomicLong(); - } - - public void accept(Long minimumRequiredIndexMetadataVersion, Consumer handler) { - if (currentIndexMetadataVersion.get() >= minimumRequiredIndexMetadataVersion) { - LOGGER.debug("Current index metadata version [{}] is higher or equal than minimum required index metadata version [{}]", - currentIndexMetadataVersion.get(), minimumRequiredIndexMetadataVersion); - handler.accept(null); - } else { - updateMapping(minimumRequiredIndexMetadataVersion, handler); - } - } - - void updateMapping(long minimumRequiredIndexMetadataVersion, Consumer handler) { - try { - updateMappingSemaphore.acquire(); - } catch (InterruptedException e) { - handler.accept(e); - return; - } - if (currentIndexMetadataVersion.get() >= minimumRequiredIndexMetadataVersion) { - updateMappingSemaphore.release(); - LOGGER.debug("Current index metadata version [{}] is higher or equal than minimum required index metadata version [{}]", - currentIndexMetadataVersion.get(), minimumRequiredIndexMetadataVersion); - handler.accept(null); - return; - } - - ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); - clusterStateRequest.clear(); - clusterStateRequest.metaData(true); - clusterStateRequest.indices(leaderIndex.getName()); - - leaderClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { - IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().getIndexSafe(leaderIndex); - assert indexMetaData.getMappings().size() == 1; - MappingMetaData mappingMetaData = indexMetaData.getMappings().iterator().next().value; - - PutMappingRequest putMappingRequest = new PutMappingRequest(followIndex.getName()); - putMappingRequest.type(mappingMetaData.type()); - putMappingRequest.source(mappingMetaData.source().string(), XContentType.JSON); - followClient.admin().indices().putMapping(putMappingRequest, ActionListener.wrap(putMappingResponse -> { - currentIndexMetadataVersion.set(indexMetaData.getVersion()); - updateMappingSemaphore.release(); - handler.accept(null); - }, e -> { - updateMappingSemaphore.release(); - handler.accept(e); - })); - }, e -> { - updateMappingSemaphore.release(); - handler.accept(e); - })); - } - } - } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java index 62612e4bb4b3a..c4d26d96905b1 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java @@ -7,12 +7,39 @@ import org.elasticsearch.action.support.WriteResponse; import org.elasticsearch.action.support.replication.ReplicationResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; public final class BulkShardOperationsResponse extends ReplicationResponse implements WriteResponse { + private long localCheckpoint; + + BulkShardOperationsResponse(long localCheckPoint) { + this.localCheckpoint = localCheckPoint; + } + + BulkShardOperationsResponse() { + } + + public long getLocalCheckpoint() { + return localCheckpoint; + } + @Override public void setForcedRefresh(final boolean forcedRefresh) { + } + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + localCheckpoint = in.readZLong(); } + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeZLong(localCheckpoint); + } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index 81cbf042037c1..9048d121a0e96 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -103,7 +103,8 @@ static WritePrimaryResult(replicaRequest, new BulkShardOperationsResponse(), location, null, primary, logger); + long localCheckPoint = primary.getLocalCheckpoint(); + return new WritePrimaryResult<>(replicaRequest, new BulkShardOperationsResponse(localCheckPoint), location, null, primary, logger); } @Override 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 235308f902926..cc1a82f5787f5 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 @@ -40,16 +40,22 @@ static Request createRequest(RestRequest restRequest) { Request request = new Request(); request.setLeaderIndex(restRequest.param("leader_index")); request.setFollowIndex(restRequest.param("index")); - if (restRequest.hasParam(ShardFollowTask.MAX_CHUNK_SIZE.getPreferredName())) { - request.setBatchSize(Long.valueOf(restRequest.param(ShardFollowTask.MAX_CHUNK_SIZE.getPreferredName()))); + if (restRequest.hasParam(ShardFollowTask.MAX_READ_SIZE.getPreferredName())) { + request.setMaxReadSize(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_READ_SIZE.getPreferredName()))); } - if (restRequest.hasParam(ShardFollowTask.NUM_CONCURRENT_CHUNKS.getPreferredName())) { - request.setConcurrentProcessors(Integer.valueOf(restRequest.param(ShardFollowTask.NUM_CONCURRENT_CHUNKS.getPreferredName()))); + if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName())) { + request.setMaxConcurrentReads(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName()))); } - if (restRequest.hasParam(ShardFollowTask.PROCESSOR_MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName())) { - long value = Long.valueOf(restRequest.param(ShardFollowTask.PROCESSOR_MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName())); + if (restRequest.hasParam(ShardFollowTask.MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName())) { + long value = Long.valueOf(restRequest.param(ShardFollowTask.MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName())); request.setProcessorMaxTranslogBytes(value); } + if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())) { + request.setMaxWriteSize(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName()))); + } + if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_WRITES.getPreferredName())) { + request.setMaxConcurrentWrites(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_WRITES.getPreferredName()))); + } return request; } } 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 ba9855b58736e..7c58be62e5c5e 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 @@ -9,13 +9,20 @@ import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.analysis.common.CommonAnalysisPlugin; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; @@ -24,6 +31,7 @@ import org.elasticsearch.index.translog.Translog; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.MockHttpTransport; @@ -37,6 +45,7 @@ import org.elasticsearch.xpack.core.XPackSettings; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -45,10 +54,14 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import static java.util.Collections.singletonMap; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -90,8 +103,8 @@ protected Collection> transportClientPlugins() { // this emulates what the CCR persistent task will do for pulling public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { client().admin().indices().prepareCreate("index") - .setSettings(Settings.builder().put("index.number_of_shards", 1)) - .get(); + .setSettings(Settings.builder().put("index.number_of_shards", 1)) + .get(); client().prepareIndex("index", "doc", "1").setSource("{}", XContentType.JSON).get(); client().prepareIndex("index", "doc", "2").setSource("{}", XContentType.JSON).get(); @@ -147,9 +160,9 @@ public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { public void testFollowIndex() throws Exception { final int numberOfPrimaryShards = randomIntBetween(1, 3); final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, - Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); - ensureGreen("index1"); + ensureYellow("index1"); final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); followRequest.setLeaderIndex("index1"); @@ -209,9 +222,9 @@ public void testFollowIndex() throws Exception { public void testSyncMappings() throws Exception { final String leaderIndexSettings = getIndexSettings(2, - Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); - ensureGreen("index1"); + ensureYellow("index1"); final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); followRequest.setLeaderIndex("index1"); @@ -229,7 +242,7 @@ public void testSyncMappings() throws Exception { assertBusy(() -> assertThat(client().prepareSearch("index2").get().getHits().totalHits, equalTo(firstBatchNumDocs))); MappingMetaData mappingMetaData = client().admin().indices().prepareGetMappings("index2").get().getMappings() - .get("index2").get("doc"); + .get("index2").get("doc"); assertThat(XContentMapValues.extractValue("properties.f.type", mappingMetaData.sourceAsMap()), equalTo("integer")); assertThat(XContentMapValues.extractValue("properties.k", mappingMetaData.sourceAsMap()), nullValue()); @@ -240,41 +253,136 @@ public void testSyncMappings() throws Exception { } assertBusy(() -> assertThat(client().prepareSearch("index2").get().getHits().totalHits, - equalTo(firstBatchNumDocs + secondBatchNumDocs))); + equalTo(firstBatchNumDocs + secondBatchNumDocs))); mappingMetaData = client().admin().indices().prepareGetMappings("index2").get().getMappings() - .get("index2").get("doc"); + .get("index2").get("doc"); assertThat(XContentMapValues.extractValue("properties.f.type", mappingMetaData.sourceAsMap()), equalTo("integer")); assertThat(XContentMapValues.extractValue("properties.k.type", mappingMetaData.sourceAsMap()), equalTo("long")); + unfollowIndex("index2"); + } - final UnfollowIndexAction.Request unfollowRequest = new UnfollowIndexAction.Request(); - unfollowRequest.setFollowIndex("index2"); - client().execute(UnfollowIndexAction.INSTANCE, unfollowRequest).get(); + public void testFollowIndex_backlog() throws Exception { + String leaderIndexSettings = getIndexSettings(3, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); + BulkProcessor.Listener listener = new BulkProcessor.Listener() { + @Override + public void beforeBulk(long executionId, BulkRequest request) {} - assertBusy(() -> { - final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); - final PersistentTasksCustomMetaData tasks = clusterState.getMetaData().custom(PersistentTasksCustomMetaData.TYPE); - assertThat(tasks.tasks().size(), equalTo(0)); + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {} - ListTasksRequest listTasksRequest = new ListTasksRequest(); - listTasksRequest.setDetailed(true); - ListTasksResponse listTasksResponse = client().admin().cluster().listTasks(listTasksRequest).get(); - int numNodeTasks = 0; - for (TaskInfo taskInfo : listTasksResponse.getTasks()) { - if (taskInfo.getAction().startsWith(ListTasksAction.NAME) == false) { - numNodeTasks++; + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) {} + }; + BulkProcessor bulkProcessor = BulkProcessor.builder(client(), listener) + .setBulkActions(100) + .setConcurrentRequests(4) + .build(); + AtomicBoolean run = new AtomicBoolean(true); + Thread thread = new Thread(() -> { + int counter = 0; + while (run.get()) { + final String source = String.format(Locale.ROOT, "{\"f\":%d}", counter++); + IndexRequest indexRequest = new IndexRequest("index1", "doc") + .source(source, XContentType.JSON) + .timeout(TimeValue.timeValueSeconds(1)); + bulkProcessor.add(indexRequest); + } + }); + thread.start(); + + // Waiting for some document being index before following the index: + int maxReadSize = randomIntBetween(128, 2048); + long numDocsIndexed = Math.min(3000 * 2, randomLongBetween(maxReadSize, maxReadSize * 10)); + atLeastDocsIndexed("index1", numDocsIndexed / 3); + + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); + followRequest.setLeaderIndex("index1"); + followRequest.setFollowIndex("index2"); + followRequest.setMaxReadSize(maxReadSize); + followRequest.setMaxConcurrentReads(randomIntBetween(2, 10)); + followRequest.setMaxConcurrentWrites(randomIntBetween(2, 10)); + CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(); + createAndFollowRequest.setFollowRequest(followRequest); + client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); + + atLeastDocsIndexed("index1", numDocsIndexed); + run.set(false); + thread.join(); + assertThat(bulkProcessor.awaitClose(1L, TimeUnit.MINUTES), is(true)); + + assertSameDocCount("index1", "index2"); + unfollowIndex("index2"); + } + + public void testFollowIndexAndCloseNode() throws Exception { + internalCluster().ensureAtLeastNumDataNodes(3); + String leaderIndexSettings = getIndexSettings(3, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); + + String followerIndexSettings = getIndexSettings(3, singletonMap(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index2").setSource(followerIndexSettings, XContentType.JSON)); + ensureGreen("index1", "index2"); + + AtomicBoolean run = new AtomicBoolean(true); + Thread thread = new Thread(() -> { + int counter = 0; + while (run.get()) { + final String source = String.format(Locale.ROOT, "{\"f\":%d}", counter++); + try { + client().prepareIndex("index1", "doc") + .setSource(source, XContentType.JSON) + .setTimeout(TimeValue.timeValueSeconds(1)) + .get(); + } catch (Exception e) { + logger.error("Error while indexing into leader index", e); } } - assertThat(numNodeTasks, equalTo(0)); }); + thread.start(); + + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); + followRequest.setLeaderIndex("index1"); + followRequest.setFollowIndex("index2"); + followRequest.setMaxReadSize(randomIntBetween(32, 2048)); + followRequest.setMaxConcurrentReads(randomIntBetween(2, 10)); + client().execute(FollowIndexAction.INSTANCE, followRequest).get(); + + long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxReadSize(), followRequest.getMaxReadSize() * 10)); + long minNumDocsReplicated = maxNumDocsReplicated / 3L; + logger.info("waiting for at least [{}] documents to be indexed and then stop a random data node", minNumDocsReplicated); + awaitBusy(() -> { + SearchRequest request = new SearchRequest("index2"); + request.source(new SearchSourceBuilder().size(0)); + SearchResponse response = client().search(request).actionGet(); + if (response.getHits().getTotalHits() >= minNumDocsReplicated) { + try { + internalCluster().stopRandomNonMasterNode(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return true; + } else { + return false; + } + }, 30, TimeUnit.SECONDS); + + logger.info("waiting for at least [{}] documents to be indexed", maxNumDocsReplicated); + atLeastDocsIndexed("index2", maxNumDocsReplicated); + run.set(false); + thread.join(); + + assertSameDocCount("index1", "index2"); + unfollowIndex("index2"); } public void testFollowIndexWithNestedField() throws Exception { final String leaderIndexSettings = - getIndexSettingsWithNestedMapping(1, Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + getIndexSettingsWithNestedMapping(1, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); final String followerIndexSettings = - getIndexSettingsWithNestedMapping(1, Collections.singletonMap(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), "true")); + getIndexSettingsWithNestedMapping(1, singletonMap(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), "true")); assertAcked(client().admin().indices().prepareCreate("index2").setSource(followerIndexSettings, XContentType.JSON)); ensureGreen("index1", "index2"); @@ -311,16 +419,7 @@ public void testFollowIndexWithNestedField() throws Exception { equalTo(Collections.singletonList(value))); }); } - - final UnfollowIndexAction.Request unfollowRequest = new UnfollowIndexAction.Request(); - unfollowRequest.setFollowIndex("index2"); - client().execute(UnfollowIndexAction.INSTANCE, unfollowRequest).get(); - - assertBusy(() -> { - final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); - final PersistentTasksCustomMetaData tasks = clusterState.getMetaData().custom(PersistentTasksCustomMetaData.TYPE); - assertThat(tasks.tasks().size(), equalTo(0)); - }); + unfollowIndex("index2"); } public void testUnfollowNonExistingIndex() { @@ -347,6 +446,43 @@ public void testFollowNonExistentIndex() throws Exception { expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest).actionGet()); } + public void testFollowIndex_lowMaxTranslogBytes() throws Exception { + final String leaderIndexSettings = getIndexSettings(1, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); + ensureYellow("index1"); + + final int numDocs = 1024; + logger.info("Indexing [{}] docs", numDocs); + for (int i = 0; i < numDocs; i++) { + final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); + client().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); + } + + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); + followRequest.setProcessorMaxTranslogBytes(1024); + followRequest.setLeaderIndex("index1"); + followRequest.setFollowIndex("index2"); + + final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(); + createAndFollowRequest.setFollowRequest(followRequest); + client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); + + final Map firstBatchNumDocsPerShard = new HashMap<>(); + final ShardStats[] firstBatchShardStats = client().admin().indices().prepareStats("index1").get().getIndex("index1").getShards(); + for (final ShardStats shardStats : firstBatchShardStats) { + if (shardStats.getShardRouting().primary()) { + long value = shardStats.getStats().getIndexing().getTotal().getIndexCount() - 1; + firstBatchNumDocsPerShard.put(shardStats.getShardRouting().shardId(), value); + } + } + + assertBusy(assertTask(1, firstBatchNumDocsPerShard)); + for (int i = 0; i < numDocs; i++) { + assertBusy(assertExpectedDocumentRunnable(i)); + } + unfollowIndex("index2"); + } + private CheckedRunnable assertTask(final int numberOfPrimaryShards, final Map numDocsPerShard) { return () -> { final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); @@ -362,7 +498,7 @@ private CheckedRunnable assertTask(final int numberOfPrimaryShards, f List taskInfos = listTasksResponse.getTasks(); assertThat(taskInfos.size(), equalTo(numberOfPrimaryShards)); Collection> shardFollowTasks = - taskMetadata.findTasks(ShardFollowTask.NAME, Objects::nonNull); + taskMetadata.findTasks(ShardFollowTask.NAME, Objects::nonNull); for (PersistentTasksCustomMetaData.PersistentTask shardFollowTask : shardFollowTasks) { final ShardFollowTask shardFollowTaskParams = (ShardFollowTask) shardFollowTask.getParams(); TaskInfo taskInfo = null; @@ -376,9 +512,9 @@ private CheckedRunnable assertTask(final int numberOfPrimaryShards, f assertThat(taskInfo, notNullValue()); ShardFollowNodeTask.Status status = (ShardFollowNodeTask.Status) taskInfo.getStatus(); assertThat(status, notNullValue()); - assertThat( - status.getProcessedGlobalCheckpoint(), - equalTo(numDocsPerShard.get(shardFollowTaskParams.getLeaderShardId()))); + assertThat("incorrect global checkpoint " + shardFollowTaskParams, + status.getProcessedGlobalCheckpoint(), + equalTo(numDocsPerShard.get(shardFollowTaskParams.getLeaderShardId()))); } }; } @@ -422,6 +558,7 @@ private String getIndexSettings(final int numberOfPrimaryShards, final Map additionalSetting : additionalIndexSettings.entrySet()) { builder.field(additionalSetting.getKey(), additionalSetting.getValue()); } @@ -502,4 +639,29 @@ private String getIndexSettingsWithNestedMapping(final int numberOfPrimaryShards } return settings; } + + private void atLeastDocsIndexed(String index, long numDocsReplicated) throws InterruptedException { + logger.info("waiting for at least [{}] documents to be indexed into index [{}]", numDocsReplicated, index); + awaitBusy(() -> { + refresh(index); + SearchRequest request = new SearchRequest(index); + request.source(new SearchSourceBuilder().size(0)); + SearchResponse response = client().search(request).actionGet(); + return response.getHits().getTotalHits() >= numDocsReplicated; + }, 30, TimeUnit.SECONDS); + } + + private void assertSameDocCount(String index1, String index2) throws Exception { + refresh(index1); + SearchRequest request1 = new SearchRequest(index1); + request1.source(new SearchSourceBuilder().size(0)); + SearchResponse response1 = client().search(request1).actionGet(); + assertBusy(() -> { + refresh(index2); + SearchRequest request2 = new SearchRequest(index2); + request2.source(new SearchSourceBuilder().size(0)); + SearchResponse response2 = client().search(request2).actionGet(); + assertThat(response2.getHits().getTotalHits(), equalTo(response1.getHits().getTotalHits())); + }); + } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ChunksCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ChunksCoordinatorTests.java deleted file mode 100644 index 9af0d93e9e2bc..0000000000000 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ChunksCoordinatorTests.java +++ /dev/null @@ -1,407 +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.ActionListener; -import org.elasticsearch.client.AdminClient; -import org.elasticsearch.client.Client; -import org.elasticsearch.client.ClusterAdminClient; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.ccr.action.ShardFollowTasksExecutor.ChunkProcessor; -import org.elasticsearch.xpack.ccr.action.ShardFollowTasksExecutor.ChunksCoordinator; -import org.elasticsearch.xpack.ccr.action.ShardFollowTasksExecutor.IndexMetadataVersionChecker; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; - -import java.net.ConnectException; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; -import java.util.function.Consumer; - -import static org.elasticsearch.xpack.ccr.action.ShardFollowTasksExecutor.PROCESSOR_RETRY_LIMIT; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.hamcrest.CoreMatchers.sameInstance; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.same; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class ChunksCoordinatorTests extends ESTestCase { - - public void testCreateChunks() { - Client client = mock(Client.class); - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - ChunksCoordinator coordinator = new ChunksCoordinator(client, client, ccrExecutor, checker, 1024, 1, - Long.MAX_VALUE, leaderShardId, followShardId, e -> {}); - coordinator.createChucks(0, 1023); - List result = new ArrayList<>(coordinator.getChunks()); - assertThat(result.size(), equalTo(1)); - assertThat(result.get(0)[0], equalTo(0L)); - assertThat(result.get(0)[1], equalTo(1023L)); - - coordinator.getChunks().clear(); - coordinator.createChucks(0, 2047); - result = new ArrayList<>(coordinator.getChunks()); - assertThat(result.size(), equalTo(2)); - assertThat(result.get(0)[0], equalTo(0L)); - assertThat(result.get(0)[1], equalTo(1023L)); - assertThat(result.get(1)[0], equalTo(1024L)); - assertThat(result.get(1)[1], equalTo(2047L)); - - coordinator.getChunks().clear(); - coordinator.createChucks(0, 4095); - result = new ArrayList<>(coordinator.getChunks()); - assertThat(result.size(), equalTo(4)); - assertThat(result.get(0)[0], equalTo(0L)); - assertThat(result.get(0)[1], equalTo(1023L)); - assertThat(result.get(1)[0], equalTo(1024L)); - assertThat(result.get(1)[1], equalTo(2047L)); - assertThat(result.get(2)[0], equalTo(2048L)); - assertThat(result.get(2)[1], equalTo(3071L)); - assertThat(result.get(3)[0], equalTo(3072L)); - assertThat(result.get(3)[1], equalTo(4095L)); - - coordinator.getChunks().clear(); - coordinator.createChucks(4096, 8196); - result = new ArrayList<>(coordinator.getChunks()); - assertThat(result.size(), equalTo(5)); - assertThat(result.get(0)[0], equalTo(4096L)); - assertThat(result.get(0)[1], equalTo(5119L)); - assertThat(result.get(1)[0], equalTo(5120L)); - assertThat(result.get(1)[1], equalTo(6143L)); - assertThat(result.get(2)[0], equalTo(6144L)); - assertThat(result.get(2)[1], equalTo(7167L)); - assertThat(result.get(3)[0], equalTo(7168L)); - assertThat(result.get(3)[1], equalTo(8191L)); - assertThat(result.get(4)[0], equalTo(8192L)); - assertThat(result.get(4)[1], equalTo(8196L)); - } - - public void testCoordinator() throws Exception { - Client client = createClientMock(); - - mockShardChangesApiCall(client); - mockBulkShardOperationsApiCall(client); - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - - Consumer handler = e -> assertThat(e, nullValue()); - int concurrentProcessors = randomIntBetween(1, 4); - int batchSize = randomIntBetween(1, 1000); - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - ChunksCoordinator coordinator = new ChunksCoordinator(client, client, ccrExecutor, checker, batchSize, - concurrentProcessors, Long.MAX_VALUE, leaderShardId, followShardId, handler); - - int numberOfOps = randomIntBetween(batchSize, batchSize * 20); - long from = randomInt(1000); - long to = from + numberOfOps - 1; - coordinator.createChucks(from, to); - int expectedNumberOfChunks = numberOfOps / batchSize; - if (numberOfOps % batchSize > 0) { - expectedNumberOfChunks++; - } - assertThat(coordinator.getChunks().size(), equalTo(expectedNumberOfChunks)); - - coordinator.start(); - assertThat(coordinator.getChunks().size(), equalTo(0)); - verify(client, times(expectedNumberOfChunks)).execute(same(ShardChangesAction.INSTANCE), - any(ShardChangesAction.Request.class), any()); - verify(client, times(expectedNumberOfChunks)).execute(same(BulkShardOperationsAction.INSTANCE), - any(BulkShardOperationsRequest.class), any()); - } - - public void testCoordinator_failure() throws Exception { - Exception expectedException = new RuntimeException("throw me"); - Client client = createClientMock(); - boolean shardChangesActionApiCallFailed; - if (randomBoolean()) { - shardChangesActionApiCallFailed = true; - doThrow(expectedException).when(client).execute(same(ShardChangesAction.INSTANCE), - any(ShardChangesAction.Request.class), any()); - } else { - shardChangesActionApiCallFailed = false; - mockShardChangesApiCall(client); - doThrow(expectedException).when(client).execute(same(BulkShardOperationsAction.INSTANCE), - any(BulkShardOperationsRequest.class), any()); - } - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - - Consumer handler = e -> { - assertThat(e, notNullValue()); - assertThat(e, sameInstance(expectedException)); - }; - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - ChunksCoordinator coordinator = new ChunksCoordinator(client, client, ccrExecutor, checker, 10, 1, Long.MAX_VALUE, - leaderShardId, followShardId, handler); - coordinator.createChucks(0, 19); - assertThat(coordinator.getChunks().size(), equalTo(2)); - - coordinator.start(); - assertThat(coordinator.getChunks().size(), equalTo(1)); - verify(client, times(1)).execute(same(ShardChangesAction.INSTANCE), any(ShardChangesAction.Request.class), - any()); - verify(client, times(shardChangesActionApiCallFailed ? 0 : 1)).execute(same(BulkShardOperationsAction.INSTANCE), - any(BulkShardOperationsRequest.class), any()); - } - - public void testCoordinator_concurrent() throws Exception { - Client client = createClientMock(); - mockShardChangesApiCall(client); - mockBulkShardOperationsApiCall(client); - Executor ccrExecutor = command -> new Thread(command).start(); - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - - AtomicBoolean calledOnceChecker = new AtomicBoolean(false); - AtomicReference failureHolder = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - Consumer handler = e -> { - if (failureHolder.compareAndSet(null, e) == false) { - // This handler should only be called once, irregardless of the number of concurrent processors - calledOnceChecker.set(true); - } - latch.countDown(); - }; - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - ChunksCoordinator coordinator = new ChunksCoordinator(client, client, ccrExecutor, checker, 1000, 4, Long.MAX_VALUE, - leaderShardId, followShardId, handler); - coordinator.createChucks(0, 999999); - assertThat(coordinator.getChunks().size(), equalTo(1000)); - - coordinator.start(); - latch.await(); - assertThat(coordinator.getChunks().size(), equalTo(0)); - verify(client, times(1000)).execute(same(ShardChangesAction.INSTANCE), any(ShardChangesAction.Request.class), any()); - verify(client, times(1000)).execute(same(BulkShardOperationsAction.INSTANCE), any(BulkShardOperationsRequest.class), any()); - assertThat(calledOnceChecker.get(), is(false)); - } - - public void testChunkProcessor() { - Client client = createClientMock(); - Queue chunks = new LinkedList<>(); - mockShardChangesApiCall(client); - mockBulkShardOperationsApiCall(client); - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - - boolean[] invoked = new boolean[1]; - Exception[] exception = new Exception[1]; - Consumer handler = e -> {invoked[0] = true;exception[0] = e;}; - ChunkProcessor chunkProcessor = new ChunkProcessor(client, client, chunks, ccrExecutor, checker, leaderShardId, - followShardId, handler); - chunkProcessor.start(0, 10, Long.MAX_VALUE); - assertThat(invoked[0], is(true)); - assertThat(exception[0], nullValue()); - } - - public void testChunkProcessorRetry() { - Client client = createClientMock(); - Queue chunks = new LinkedList<>(); - mockBulkShardOperationsApiCall(client); - int testRetryLimit = randomIntBetween(1, PROCESSOR_RETRY_LIMIT - 1); - mockShardCangesApiCallWithRetry(client, testRetryLimit, new ConnectException("connection exception")); - - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - - boolean[] invoked = new boolean[1]; - Exception[] exception = new Exception[1]; - Consumer handler = e -> {invoked[0] = true;exception[0] = e;}; - ChunkProcessor chunkProcessor = new ChunkProcessor(client, client, chunks, ccrExecutor, checker, leaderShardId, - followShardId, handler); - chunkProcessor.start(0, 10, Long.MAX_VALUE); - assertThat(invoked[0], is(true)); - assertThat(exception[0], nullValue()); - assertThat(chunkProcessor.retryCounter.get(), equalTo(testRetryLimit + 1)); - } - - public void testChunkProcessorRetryTooManyTimes() { - Client client = createClientMock(); - Queue chunks = new LinkedList<>(); - mockBulkShardOperationsApiCall(client); - int testRetryLimit = PROCESSOR_RETRY_LIMIT + 1; - mockShardCangesApiCallWithRetry(client, testRetryLimit, new ConnectException("connection exception")); - - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - - boolean[] invoked = new boolean[1]; - Exception[] exception = new Exception[1]; - Consumer handler = e -> {invoked[0] = true;exception[0] = e;}; - ChunkProcessor chunkProcessor = new ChunkProcessor(client, client, chunks, ccrExecutor, checker, leaderShardId, - followShardId, handler); - chunkProcessor.start(0, 10, Long.MAX_VALUE); - assertThat(invoked[0], is(true)); - assertThat(exception[0], notNullValue()); - assertThat(exception[0].getMessage(), equalTo("retrying failed [17] times, aborting...")); - assertThat(exception[0].getCause().getMessage(), equalTo("connection exception")); - assertThat(chunkProcessor.retryCounter.get(), equalTo(testRetryLimit)); - } - - public void testChunkProcessorNoneRetryableError() { - Client client = createClientMock(); - Queue chunks = new LinkedList<>(); - mockBulkShardOperationsApiCall(client); - mockShardCangesApiCallWithRetry(client, 3, new RuntimeException("unexpected")); - - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - - boolean[] invoked = new boolean[1]; - Exception[] exception = new Exception[1]; - Consumer handler = e -> {invoked[0] = true;exception[0] = e;}; - ChunkProcessor chunkProcessor = new ChunkProcessor(client, client, chunks, ccrExecutor, checker, leaderShardId, - followShardId, handler); - chunkProcessor.start(0, 10, Long.MAX_VALUE); - assertThat(invoked[0], is(true)); - assertThat(exception[0], notNullValue()); - assertThat(exception[0].getMessage(), equalTo("unexpected")); - assertThat(chunkProcessor.retryCounter.get(), equalTo(0)); - } - - public void testChunkProcessorExceedMaxTranslogsBytes() { - long from = 0; - long to = 20; - long actualTo = 10; - Client client = createClientMock(); - Queue chunks = new LinkedList<>(); - doAnswer(invocation -> { - Object[] args = invocation.getArguments(); - assert args.length == 3; - @SuppressWarnings("unchecked") - ActionListener listener = (ActionListener) args[2]; - - List operations = new ArrayList<>(); - for (int i = 0; i <= actualTo; i++) { - operations.add(new Translog.NoOp(i, 1, "test")); - } - listener.onResponse(new ShardChangesAction.Response(1L, operations.toArray(new Translog.Operation[0]))); - return null; - }).when(client).execute(same(ShardChangesAction.INSTANCE), any(ShardChangesAction.Request.class), any()); - - mockBulkShardOperationsApiCall(client); - Executor ccrExecutor = Runnable::run; - ShardId leaderShardId = new ShardId("index1", "index1", 0); - ShardId followShardId = new ShardId("index2", "index1", 0); - IndexMetadataVersionChecker checker = new IndexMetadataVersionChecker(leaderShardId.getIndex(), - followShardId.getIndex(), client, client); - - boolean[] invoked = new boolean[1]; - Exception[] exception = new Exception[1]; - Consumer handler = e -> {invoked[0] = true;exception[0] = e;}; - BiConsumer> versionChecker = (indexVersiuon, consumer) -> consumer.accept(null); - ChunkProcessor chunkProcessor = - new ChunkProcessor(client, client, chunks, ccrExecutor, versionChecker, leaderShardId, followShardId, handler); - chunkProcessor.start(from, to, Long.MAX_VALUE); - assertThat(invoked[0], is(true)); - assertThat(exception[0], nullValue()); - assertThat(chunks.size(), equalTo(1)); - assertThat(chunks.peek()[0], equalTo(11L)); - assertThat(chunks.peek()[1], equalTo(20L)); - } - - private Client createClientMock() { - Client client = mock(Client.class); - ClusterAdminClient clusterAdminClient = mock(ClusterAdminClient.class); - AdminClient adminClient = mock(AdminClient.class); - when(adminClient.cluster()).thenReturn(clusterAdminClient); - when(client.admin()).thenReturn(adminClient); - return client; - } - - private void mockShardCangesApiCallWithRetry(Client client, int testRetryLimit, Exception e) { - int[] retryCounter = new int[1]; - doAnswer(invocation -> { - Object[] args = invocation.getArguments(); - assert args.length == 3; - ShardChangesAction.Request request = (ShardChangesAction.Request) args[1]; - @SuppressWarnings("unchecked") - ActionListener listener = (ActionListener) args[2]; - if (retryCounter[0]++ <= testRetryLimit) { - listener.onFailure(e); - } else { - long delta = request.getMaxSeqNo() - request.getMinSeqNo(); - Translog.Operation[] operations = new Translog.Operation[(int) delta]; - for (int i = 0; i < operations.length; i++) { - operations[i] = new Translog.NoOp(request.getMinSeqNo() + i, 1, "test"); - } - ShardChangesAction.Response response = new ShardChangesAction.Response(0L, operations); - listener.onResponse(response); - } - return null; - }).when(client).execute(same(ShardChangesAction.INSTANCE), any(ShardChangesAction.Request.class), any()); - } - - private void mockShardChangesApiCall(Client client) { - doAnswer(invocation -> { - Object[] args = invocation.getArguments(); - assert args.length == 3; - ShardChangesAction.Request request = (ShardChangesAction.Request) args[1]; - @SuppressWarnings("unchecked") - ActionListener listener = (ActionListener) args[2]; - - List operations = new ArrayList<>(); - for (long i = request.getMinSeqNo(); i <= request.getMaxSeqNo(); i++) { - operations.add(new Translog.NoOp(request.getMinSeqNo() + i, 1, "test")); - } - ShardChangesAction.Response response = new ShardChangesAction.Response(0L, operations.toArray(new Translog.Operation[0])); - listener.onResponse(response); - return null; - }).when(client).execute(same(ShardChangesAction.INSTANCE), any(ShardChangesAction.Request.class), any()); - } - - private void mockBulkShardOperationsApiCall(Client client) { - doAnswer(invocation -> { - Object[] args = invocation.getArguments(); - assert args.length == 3; - @SuppressWarnings("unchecked") - ActionListener listener = (ActionListener) args[2]; - listener.onResponse(new BulkShardOperationsResponse()); - return null; - }).when(client).execute(same(BulkShardOperationsAction.INSTANCE), any(BulkShardOperationsRequest.class), any()); - } - -} 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 9a0557b369a77..73c4a192a63f5 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 @@ -23,9 +23,13 @@ static FollowIndexAction.Request createTestRequest() { FollowIndexAction.Request request = new FollowIndexAction.Request(); request.setLeaderIndex(randomAlphaOfLength(4)); request.setFollowIndex(randomAlphaOfLength(4)); - request.setBatchSize(randomNonNegativeLong()); - request.setConcurrentProcessors(randomIntBetween(0, Integer.MAX_VALUE)); + request.setMaxReadSize(randomIntBetween(1, Integer.MAX_VALUE)); + request.setMaxConcurrentReads(randomIntBetween(1, Integer.MAX_VALUE)); request.setProcessorMaxTranslogBytes(randomNonNegativeLong()); + request.setMaxWriteSize(randomIntBetween(1, Integer.MAX_VALUE)); + request.setMaxConcurrentWrites(randomIntBetween(1, Integer.MAX_VALUE)); + request.setMaxConcurrentWrites(randomIntBetween(1, Integer.MAX_VALUE)); + request.setMaxBufferSize(randomIntBetween(1, Integer.MAX_VALUE)); return request; } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index 7e06ce120a86a..56440dc53f57b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -48,7 +48,7 @@ public void testGetOperationsBetween() throws Exception { IndexShard indexShard = indexService.getShard(0); for (int iter = 0; iter < iters; iter++) { int min = randomIntBetween(0, numWrites - 1); - int max = randomIntBetween(min, numWrites - 1); + long max = randomIntBetween(min, numWrites - 1); final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, min, max, Long.MAX_VALUE); final List seenSeqNos = Arrays.stream(operations).map(Translog.Operation::seqNo).collect(Collectors.toList()); final List expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toList()); @@ -57,13 +57,13 @@ public void testGetOperationsBetween() throws Exception { // get operations for a range no operations exists: Exception e = expectThrows(IllegalStateException.class, - () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites, numWrites + 1, Long.MAX_VALUE)); + () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites, (long) numWrites + 1, Long.MAX_VALUE)); assertThat(e.getMessage(), containsString("Not all operations between min_seqno [" + numWrites + "] and max_seqno [" + (numWrites + 1) +"] found")); // get operations for a range some operations do not exist: e = expectThrows(IllegalStateException.class, - () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, numWrites + 10, Long.MAX_VALUE)); + () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, (long) numWrites + 10, Long.MAX_VALUE)); assertThat(e.getMessage(), containsString("Not all operations between min_seqno [" + (numWrites - 10) + "] and max_seqno [" + (numWrites + 10) +"] found")); } @@ -81,7 +81,7 @@ public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { ShardRouting shardRouting = TestShardRouting.newShardRouting("index", 0, "_node_id", true, ShardRoutingState.INITIALIZING); Mockito.when(indexShard.routingEntry()).thenReturn(shardRouting); - expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, 0, 1, Long.MAX_VALUE)); + expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, 0, 1L, Long.MAX_VALUE)); } public void testGetOperationsBetweenExceedByteLimit() throws Exception { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java index 3f30545576046..dfb5b11b94801 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java @@ -32,9 +32,10 @@ public void testValidate() { assertThat(request.validate().getMessage(), containsString("minSeqNo [-1] cannot be lower than 0")); request.setMinSeqNo(4); + request.setMaxSeqNo(0L); assertThat(request.validate().getMessage(), containsString("minSeqNo [4] cannot be larger than maxSeqNo [0]")); - request.setMaxSeqNo(8); + request.setMaxSeqNo(8L); assertThat(request.validate(), nullValue()); } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesResponseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesResponseTests.java index 4902917ab53e3..2b992c8bc6926 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesResponseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesResponseTests.java @@ -13,12 +13,13 @@ public class ShardChangesResponseTests extends AbstractStreamableTestCase failureHolder = new AtomicReference<>(); + + public void testDefaults() throws Exception { + task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_READ_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, + ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, 10000, + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE); + task.start(randomIntBetween(-1, 2048), -1); + + assertBusy(() -> { + assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + }); + assertThat(mappingUpdateCounter.get(), equalTo(1)); + } + + public void testHitBufferLimit() throws Exception { + // Setting buffer limit to 100, so that we are sure the limit will be met + task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_READ_SIZE, 3, + ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, 1, 10000, 100); + task.start(-1, -1); + + assertBusy(() -> { + assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + }); + } + + public void testConcurrentReadsAndWrites() throws Exception { + task = createShardFollowTask(randomIntBetween(32, 2048), randomIntBetween(2, 10), randomIntBetween(32, 2048), + randomIntBetween(2, 10), 50000, 10240); + task.start(randomIntBetween(-1, 2048), -1); + + assertBusy(() -> { + assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(50000L)); + }); + } + + public void testMappingUpdate() throws Exception { + task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024); + task.start(-1, -1); + + assertBusy(() -> { + assertThat(task.getStatus().getProcessedGlobalCheckpoint(), greaterThanOrEqualTo(1000L)); + }); + imdVersion.set(2L); + assertBusy(() -> { + assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + }); + assertThat(mappingUpdateCounter.get(), equalTo(2)); + } + + public void testOccasionalApiFailure() throws Exception { + task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024); + task.start(-1, -1); + randomlyFailWithRetryableError.set(true); + assertBusy(() -> { + assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + }); + assertThat(failedRequests.get(), greaterThan(0)); + } + + public void testNotAllExpectedOpsReturned() throws Exception { + task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024); + task.start(-1, -1); + randomlyTruncateRequests.set(true); + assertBusy(() -> { + assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + }); + assertThat(truncatedRequests.get(), greaterThan(0)); + } + + ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentReads, int maxWriteSize, + int maxConcurrentWrites, int leaderGlobalCheckpoint, int bufferLimit) { + imdVersion = new AtomicLong(1L); + mappingUpdateCounter = new AtomicInteger(0); + randomlyTruncateRequests = new AtomicBoolean(false); + truncatedRequests = new AtomicInteger(); + randomlyFailWithRetryableError = new AtomicBoolean(false); + failedRequests = new AtomicInteger(0); + AtomicBoolean stopped = new AtomicBoolean(false); + ShardFollowTask params = new ShardFollowTask(null, new ShardId("follow_index", "", 0), + new ShardId("leader_index", "", 0), maxReadSize, maxConcurrentReads, ShardFollowNodeTask.DEFAULT_MAX_TRANSLOG_BYTES, + maxWriteSize, maxConcurrentWrites, bufferLimit, Collections.emptyMap()); + + BiConsumer scheduler = (delay, task) -> { + try { + Thread.sleep(delay.millis()); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + Thread thread = new Thread(task); + thread.start(); + }; + AtomicInteger readCounter = new AtomicInteger(); + AtomicInteger writeCounter = new AtomicInteger(); + LocalCheckpointTracker tracker = new LocalCheckpointTracker(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); + return new ShardFollowNodeTask(1L, "type", ShardFollowTask.NAME, "description", null, + Collections.emptyMap(), null, null, params, scheduler) { + + @Override + protected void updateMapping(LongConsumer handler) { + mappingUpdateCounter.incrementAndGet(); + handler.accept(imdVersion.get()); + } + + @Override + protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, LongConsumer handler, + Consumer errorHandler) { + if (randomlyFailWithRetryableError.get() && readCounter.incrementAndGet() % 5 == 0) { + failedRequests.incrementAndGet(); + errorHandler.accept(new UnavailableShardsException(params.getFollowShardId(), "test error")); + return; + } + + for(Translog.Operation op : operations) { + tracker.markSeqNoAsCompleted(op.seqNo()); + } + + // Emulate network thread and avoid SO: + Thread thread = new Thread(() -> handler.accept(tracker.getCheckpoint())); + thread.start(); + } + + @Override + protected void innerSendShardChangesRequest(long from, Long to, Consumer handler, + Consumer errorHandler) { + if (randomlyFailWithRetryableError.get() && writeCounter.incrementAndGet() % 5 == 0) { + failedRequests.incrementAndGet(); + errorHandler.accept(new UnavailableShardsException(params.getFollowShardId(), "test error")); + return; + } + + if (from < 0) { + errorHandler.accept(new IllegalArgumentException()); + return; + } + + ShardChangesAction.Response response; + if (from >= leaderGlobalCheckpoint) { + response = new ShardChangesAction.Response(1L, leaderGlobalCheckpoint, new Translog.Operation[0]); + } else { + int size = to == null ? 100 : (int) (to - from + 1); + if (randomlyTruncateRequests.get() && size > 10 && truncatedRequests.get() < 5) { + truncatedRequests.incrementAndGet(); + size = size / 2; + } + Translog.Operation[] ops = new Translog.Operation[size]; + for (int i = 0; i < ops.length; i++) { + ops[i] = new Translog.Index("doc", UUIDs.randomBase64UUID(), from + i, 0, "{}".getBytes(StandardCharsets.UTF_8)); + } + response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, ops); + } + // Emulate network thread and avoid SO: + Thread thread = new Thread(() -> handler.accept(response)); + thread.start(); + } + + @Override + protected boolean isStopped() { + return stopped.get(); + } + + @Override + public void markAsCompleted() { + stopped.set(true); + } + + @Override + public void markAsFailed(Exception e) { + stopped.set(true); + failureHolder.set(e); + } + }; + } + + @After + public void cancelNodeTask() throws Exception { + if (task != null){ + task.markAsCompleted(); + assertThat(failureHolder.get(), nullValue()); + assertBusy(() -> { + ShardFollowNodeTask.Status status = task.getStatus(); + assertThat(status.getNumberOfConcurrentReads(), equalTo(0)); + assertThat(status.getNumberOfConcurrentWrites(), equalTo(0)); + }); + } + } + +} diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java index b128e88e63a5a..5f6370318f7f8 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java @@ -23,11 +23,16 @@ protected ShardFollowTask doParseInstance(XContentParser parser) throws IOExcept @Override protected ShardFollowTask createTestInstance() { return new ShardFollowTask( - randomAlphaOfLength(4), - new ShardId(randomAlphaOfLength(4), randomAlphaOfLength(4), randomInt(5)), - new ShardId(randomAlphaOfLength(4), randomAlphaOfLength(4), randomInt(5)), - randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), - randomIntBetween(1, Integer.MAX_VALUE), randomBoolean() ? null : Collections.singletonMap("key", "value")); + randomAlphaOfLength(4), + new ShardId(randomAlphaOfLength(4), randomAlphaOfLength(4), randomInt(5)), + new ShardId(randomAlphaOfLength(4), randomAlphaOfLength(4), randomInt(5)), + randomIntBetween(1, Integer.MAX_VALUE), + randomIntBetween(1, Integer.MAX_VALUE), + randomNonNegativeLong(), + randomIntBetween(1, Integer.MAX_VALUE), + randomIntBetween(1, Integer.MAX_VALUE), + randomIntBetween(1, Integer.MAX_VALUE), + randomBoolean() ? null : Collections.singletonMap("key", "value")); } @Override From 9dec9202a843f760ae012fd721315bd1d1535e89 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 09:57:53 +0200 Subject: [PATCH 02/41] Made to param required like before and changed validation in shard changes api and LuceneChangesSnapshot to allow a response to not return up to `toSeqNo`. --- .../index/engine/LuceneChangesSnapshot.java | 7 +--- .../xpack/ccr/action/ShardChangesAction.java | 34 +++++-------------- .../xpack/ccr/action/ShardFollowNodeTask.java | 24 ++++++------- .../ccr/action/ShardChangesActionTests.java | 18 ++++------ .../ccr/action/ShardChangesRequestTests.java | 3 +- 5 files changed, 30 insertions(+), 56 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index aa3cff4a6054e..e054dcd274d59 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -139,12 +139,7 @@ public Translog.Operation next() throws IOException { } private void rangeCheck(Translog.Operation op) { - if (op == null) { - if (lastSeenSeqNo < toSeqNo) { - throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + - "and max_seqno [" + toSeqNo + "] found; prematurely terminated last_seen_seqno [" + lastSeenSeqNo + "]"); - } - } else { + if (op != null) { final long expectedSeqNo = lastSeenSeqNo + 1; if (op.seqNo() != expectedSeqNo) { throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + 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 b746f8dcd4225..317594dc2bf6f 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 @@ -54,7 +54,7 @@ public Response newResponse() { public static class Request extends SingleShardRequest { private long minSeqNo; - private Long maxSeqNo; + private long maxSeqNo; private ShardId shardId; private long maxTranslogsBytes = ShardFollowNodeTask.DEFAULT_MAX_TRANSLOG_BYTES; @@ -78,11 +78,11 @@ public void setMinSeqNo(long minSeqNo) { this.minSeqNo = minSeqNo; } - public Long getMaxSeqNo() { + public long getMaxSeqNo() { return maxSeqNo; } - public void setMaxSeqNo(Long maxSeqNo) { + public void setMaxSeqNo(long maxSeqNo) { this.maxSeqNo = maxSeqNo; } @@ -100,7 +100,7 @@ public ActionRequestValidationException validate() { if (minSeqNo < 0) { validationException = addValidationError("minSeqNo [" + minSeqNo + "] cannot be lower than 0", validationException); } - if (maxSeqNo != null && maxSeqNo < minSeqNo) { + if (maxSeqNo < minSeqNo) { validationException = addValidationError("minSeqNo [" + minSeqNo + "] cannot be larger than maxSeqNo [" + maxSeqNo + "]", validationException); } @@ -115,7 +115,7 @@ public ActionRequestValidationException validate() { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); minSeqNo = in.readVLong(); - maxSeqNo = in.readOptionalLong(); + maxSeqNo = in.readVLong(); shardId = ShardId.readShardId(in); maxTranslogsBytes = in.readVLong(); } @@ -124,7 +124,7 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeVLong(minSeqNo); - out.writeOptionalLong(maxSeqNo); + out.writeVLong(maxSeqNo); shardId.writeTo(out); out.writeVLong(maxTranslogsBytes); } @@ -136,7 +136,7 @@ public boolean equals(final Object o) { if (o == null || getClass() != o.getClass()) return false; final Request request = (Request) o; return minSeqNo == request.minSeqNo && - Objects.equals(maxSeqNo, request.maxSeqNo) && + maxSeqNo == request.maxSeqNo && Objects.equals(shardId, request.shardId) && maxTranslogsBytes == request.maxTranslogsBytes; } @@ -233,12 +233,6 @@ protected Response shardOperation(Request request, ShardId shardId) throws IOExc IndexShard indexShard = indexService.getShard(request.getShard().id()); final long indexMetaDataVersion = clusterService.state().metaData().index(shardId.getIndex()).getVersion(); - // The following shard generates this request based on the global checkpoint on the primary copy on the leader. - // Although this value might not have been synced to all replica copies on the leader, the requesting range - // is guaranteed to be at most the local-checkpoint of any shard copies on the leader. - assert request.maxSeqNo == null || request.maxSeqNo <= indexShard.getLocalCheckpoint() : "invalid request from_seqno=[" + - request.minSeqNo + "]," + " to_seqno=[" + request.maxSeqNo + "], local_checkpoint=[" + - indexShard.getLocalCheckpoint() + "]"; final Translog.Operation[] operations = getOperationsBetween(indexShard, request.minSeqNo, request.maxSeqNo, request.maxTranslogsBytes); return new Response(indexMetaDataVersion, indexShard.getGlobalCheckpoint(), operations); @@ -266,15 +260,14 @@ protected Response newResponse() { private static final Translog.Operation[] EMPTY_OPERATIONS_ARRAY = new Translog.Operation[0]; - static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long minSeqNo, Long maxSeqNo, + static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long minSeqNo, long maxSeqNo, long byteLimit) throws IOException { if (indexShard.state() != IndexShardState.STARTED) { throw new IndexShardNotStartedException(indexShard.shardId(), indexShard.state()); } int seenBytes = 0; final List operations = new ArrayList<>(); - long max = maxSeqNo != null ? maxSeqNo : minSeqNo + 1000; - try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", minSeqNo, max, true)) { + try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", minSeqNo, maxSeqNo, true)) { Translog.Operation op; while ((op = snapshot.next()) != null) { if (op.getSource() == null) { @@ -287,15 +280,6 @@ static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long min break; } } - } catch (IllegalStateException e) { - // TODO: handle peek reads better. - // Should this optional upper bound leak into the newLuceneChangesSnapshot(...) method? - if (maxSeqNo != null) { - throw e; - } else if (e.getMessage().contains("prematurely terminated last_seen_seqno") == false) { - // Only fail if there are gaps between the ops. - throw e; - } } return operations.toArray(EMPTY_OPERATIONS_ARRAY); } 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 5fcfd44d27fd2..4d333fd3b2bab 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 @@ -142,7 +142,7 @@ private synchronized void coordinateReads() { long from = lastRequestedSeqno + 1; long to = from + maxReadSize <= leaderGlobalCheckpoint ? from + maxReadSize : leaderGlobalCheckpoint; LOGGER.debug("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, to); - sendShardChangesRequest(from, to); + sendShardChangesRequest(from, to, true); lastRequestedSeqno = to; } if (numConcurrentReads == 0) { @@ -159,7 +159,7 @@ private synchronized void coordinateReads() { numConcurrentReads++; long from = lastRequestedSeqno + 1; LOGGER.debug("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, null); + sendShardChangesRequest(from, from + maxReadSize, false); } }); } @@ -188,16 +188,16 @@ private synchronized void coordinateWrites() { } } - private void sendShardChangesRequest(long from, Long to) { + private void sendShardChangesRequest(long from, long to, boolean bla) { innerSendShardChangesRequest(from, to, response -> { retryCounter.set(0); - handleResponse(from, to, response); + handleResponse(from, to, bla, response); }, - e -> handleFailure(e, () -> sendShardChangesRequest(from, to))); + e -> handleFailure(e, () -> sendShardChangesRequest(from, to, bla))); } - private synchronized void handleResponse(long from, Long to, ShardChangesAction.Response response) { + private synchronized void handleResponse(long from, long to, boolean checkLastOpReceived, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getLeaderGlobalCheckpoint()); @@ -214,19 +214,19 @@ private synchronized void handleResponse(long from, Long to, ShardChangesAction. LOGGER.debug("{} received [{}/{}]", params.getFollowShardId(), firstOp.seqNo(), lastOp.seqNo()); buffer.addAll(Arrays.asList(response.getOperations())); - if (to == null) { - lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOp.seqNo()); - LOGGER.debug("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); - numConcurrentReads--; - } else { + if (checkLastOpReceived) { if (lastOp.seqNo() < to) { long newFrom = lastOp.seqNo() + 1; LOGGER.debug("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", params.getFollowShardId(), lastOp.seqNo(), to, newFrom, to); - sendShardChangesRequest(newFrom, to); + sendShardChangesRequest(newFrom, to, true); } else { numConcurrentReads--; } + } else { + lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOp.seqNo()); + LOGGER.debug("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); + numConcurrentReads--; } if (numConcurrentWrites == 0) { coordinateWrites(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index 56440dc53f57b..2ecf49d2d7be8 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -25,7 +25,6 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; public class ShardChangesActionTests extends ESSingleNodeTestCase { @@ -48,24 +47,21 @@ public void testGetOperationsBetween() throws Exception { IndexShard indexShard = indexService.getShard(0); for (int iter = 0; iter < iters; iter++) { int min = randomIntBetween(0, numWrites - 1); - long max = randomIntBetween(min, numWrites - 1); + int max = randomIntBetween(min, numWrites - 1); final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, min, max, Long.MAX_VALUE); final List seenSeqNos = Arrays.stream(operations).map(Translog.Operation::seqNo).collect(Collectors.toList()); final List expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toList()); assertThat(seenSeqNos, equalTo(expectedSeqNos)); } + // get operations for a range no operations exists: - Exception e = expectThrows(IllegalStateException.class, - () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites, (long) numWrites + 1, Long.MAX_VALUE)); - assertThat(e.getMessage(), containsString("Not all operations between min_seqno [" + numWrites + "] and max_seqno [" + - (numWrites + 1) +"] found")); + Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, numWrites, numWrites + 1, Long.MAX_VALUE); + assertThat(operations.length, equalTo(0)); // get operations for a range some operations do not exist: - e = expectThrows(IllegalStateException.class, - () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, (long) numWrites + 10, Long.MAX_VALUE)); - assertThat(e.getMessage(), containsString("Not all operations between min_seqno [" + (numWrites - 10) + "] and max_seqno [" + - (numWrites + 10) +"] found")); + operations = ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, numWrites + 10, Long.MAX_VALUE); + assertThat(operations.length, equalTo(10)); } public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { @@ -81,7 +77,7 @@ public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { ShardRouting shardRouting = TestShardRouting.newShardRouting("index", 0, "_node_id", true, ShardRoutingState.INITIALIZING); Mockito.when(indexShard.routingEntry()).thenReturn(shardRouting); - expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, 0, 1L, Long.MAX_VALUE)); + expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, 0, 1, Long.MAX_VALUE)); } public void testGetOperationsBetweenExceedByteLimit() throws Exception { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java index dfb5b11b94801..3f30545576046 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java @@ -32,10 +32,9 @@ public void testValidate() { assertThat(request.validate().getMessage(), containsString("minSeqNo [-1] cannot be lower than 0")); request.setMinSeqNo(4); - request.setMaxSeqNo(0L); assertThat(request.validate().getMessage(), containsString("minSeqNo [4] cannot be larger than maxSeqNo [0]")); - request.setMaxSeqNo(8L); + request.setMaxSeqNo(8); assertThat(request.validate(), nullValue()); } } From 7ae1f1df23af69acfe069401d29708e2fd275d0c Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 10:07:51 +0200 Subject: [PATCH 03/41] changed log levels --- .../xpack/ccr/action/ShardFollowNodeTask.java | 30 ++++++++++--------- .../ccr/action/ShardFollowNodeTaskTests.java | 2 ++ 2 files changed, 18 insertions(+), 14 deletions(-) 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 4d333fd3b2bab..1228e44b5d8f7 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 @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.ccr.action; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; @@ -125,40 +126,40 @@ private synchronized void coordinateReads() { if (lastRequestedSeqno < leaderGlobalCheckpoint) { while (true) { if (lastRequestedSeqno >= leaderGlobalCheckpoint) { - LOGGER.debug("{} no new reads to coordinate lastRequestedSeqno [{}] leaderGlobalCheckpoint [{}]", + LOGGER.trace("{} no new reads to coordinate lastRequestedSeqno [{}] leaderGlobalCheckpoint [{}]", params.getLeaderShardId(), lastRequestedSeqno, leaderGlobalCheckpoint); break; } if (numConcurrentReads >= maxConcurrentReads) { - LOGGER.debug("{} no new reads, maximum number of concurrent reads have been reached [{}]", + LOGGER.trace("{} no new reads, maximum number of concurrent reads have been reached [{}]", params.getFollowShardId(), numConcurrentReads); break; } if (buffer.size() > params.getMaxBufferSize()) { - LOGGER.debug("{} no new reads, buffer limit has been reached [{}]", params.getFollowShardId(), buffer.size()); + LOGGER.trace("{} no new reads, buffer limit has been reached [{}]", params.getFollowShardId(), buffer.size()); break; } numConcurrentReads++; long from = lastRequestedSeqno + 1; long to = from + maxReadSize <= leaderGlobalCheckpoint ? from + maxReadSize : leaderGlobalCheckpoint; - LOGGER.debug("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, to); + LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, to); sendShardChangesRequest(from, to, true); lastRequestedSeqno = to; } if (numConcurrentReads == 0) { - LOGGER.debug("{} re-scheduling coordinate reads phase", params.getFollowShardId()); + LOGGER.trace("{} re-scheduling coordinate reads phase", params.getFollowShardId()); scheduler.accept(TimeValue.timeValueMillis(500), this::coordinateReads); } } else { if (numConcurrentReads == 0) { - LOGGER.debug("{} scheduling peek read", params.getFollowShardId()); + LOGGER.trace("{} scheduling peek read", params.getFollowShardId()); scheduler.accept(TimeValue.timeValueMillis(500), () -> { synchronized (this) { // We sneak peek if there is any thing new in the leader primary. // If there is we will happily accept numConcurrentReads++; long from = lastRequestedSeqno + 1; - LOGGER.debug("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); + LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); sendShardChangesRequest(from, from + maxReadSize, false); } }); @@ -169,11 +170,11 @@ private synchronized void coordinateReads() { private synchronized void coordinateWrites() { while (true) { if (buffer.isEmpty()) { - LOGGER.debug("{} no writes to coordinate, because buffer is empty", params.getFollowShardId()); + LOGGER.trace("{} no writes to coordinate, because buffer is empty", params.getFollowShardId()); break; } if (numConcurrentWrites >= params.getMaxConcurrentWrites()) { - LOGGER.debug("{} maximum number of concurrent writes have been reached [{}]", + LOGGER.trace("{} maximum number of concurrent writes have been reached [{}]", params.getFollowShardId(), numConcurrentWrites); break; } @@ -182,7 +183,7 @@ private synchronized void coordinateWrites() { ops[i] = buffer.remove(); } numConcurrentWrites++; - LOGGER.debug("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops[0].seqNo(), + LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops[0].seqNo(), ops[ops.length - 1].seqNo(), ops.length); sendBulkShardOperationsRequest(ops); } @@ -212,12 +213,12 @@ private synchronized void handleResponse(long from, long to, boolean checkLastOp assert firstOp.seqNo() == from; Translog.Operation lastOp = response.getOperations()[response.getOperations().length - 1]; - LOGGER.debug("{} received [{}/{}]", params.getFollowShardId(), firstOp.seqNo(), lastOp.seqNo()); + LOGGER.trace("{} received [{}/{}]", params.getFollowShardId(), firstOp.seqNo(), lastOp.seqNo()); buffer.addAll(Arrays.asList(response.getOperations())); if (checkLastOpReceived) { if (lastOp.seqNo() < to) { long newFrom = lastOp.seqNo() + 1; - LOGGER.debug("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", + LOGGER.trace("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", params.getFollowShardId(), lastOp.seqNo(), to, newFrom, to); sendShardChangesRequest(newFrom, to, true); } else { @@ -225,7 +226,7 @@ private synchronized void handleResponse(long from, long to, boolean checkLastOp } } else { lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOp.seqNo()); - LOGGER.debug("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); + LOGGER.trace("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); numConcurrentReads--; } if (numConcurrentWrites == 0) { @@ -276,7 +277,8 @@ private void handleFailure(Exception e, Runnable task) { assert e != null; if (shouldRetry(e)) { if (isStopped() == false && retryCounter.incrementAndGet() <= RETRY_LIMIT) { - LOGGER.warn("error during follow shard task, retrying...", e); + LOGGER.debug(new ParameterizedMessage("{} error during follow shard task, retrying...", + new Object[]{params.getFollowShardId()}, e)); scheduler.accept(RETRY_TIMEOUT, task); } else { markAsFailed(new ElasticsearchException("retrying failed [" + retryCounter.get() + 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 0a7653cec8ded..0686cf4eef4f3 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.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.After; import java.nio.charset.StandardCharsets; @@ -30,6 +31,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.nullValue; +@TestLogging("org.elasticsearch.xpack.ccr.action:TRACE") public class ShardFollowNodeTaskTests extends ESTestCase { private ShardFollowNodeTask task; From 5510731f0f577b3451f9690a236c8a8eb87666f4 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 10:21:28 +0200 Subject: [PATCH 04/41] renamed maxTranslogBytes to maxOperationSizeInBytes --- .../xpack/ccr/action/FollowIndexAction.java | 18 +++++++------- .../xpack/ccr/action/ShardChangesAction.java | 24 +++++++++---------- .../xpack/ccr/action/ShardFollowNodeTask.java | 4 ++-- .../xpack/ccr/action/ShardFollowTask.java | 24 +++++++++---------- .../xpack/ccr/rest/RestFollowIndexAction.java | 6 ++--- .../xpack/ccr/ShardChangesIT.java | 2 +- .../ccr/action/FollowIndexRequestTests.java | 2 +- .../ccr/action/ShardFollowNodeTaskTests.java | 2 +- 8 files changed, 41 insertions(+), 41 deletions(-) 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 index 499d93023e459..a4ce794f93ebc 100644 --- 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 @@ -73,7 +73,7 @@ public static class Request extends ActionRequest { private String followIndex; private int maxReadSize = ShardFollowNodeTask.DEFAULT_MAX_READ_SIZE; private int maxConcurrentReads = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS; - private long processorMaxTranslogBytes = ShardFollowNodeTask.DEFAULT_MAX_TRANSLOG_BYTES; + private long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; private int maxWriteSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE; private int maxConcurrentWrites = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES; private int maxBufferSize = ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE; @@ -113,11 +113,11 @@ public void setMaxConcurrentReads(int maxConcurrentReads) { this.maxConcurrentReads = maxConcurrentReads; } - public void setProcessorMaxTranslogBytes(long processorMaxTranslogBytes) { - if (processorMaxTranslogBytes <= 0) { + public void setMaxOperationSizeInBytes(long maxOperationSizeInBytes) { + if (maxOperationSizeInBytes <= 0) { throw new IllegalArgumentException("processor_max_translog_bytes must be larger than 0"); } - this.processorMaxTranslogBytes = processorMaxTranslogBytes; + this.maxOperationSizeInBytes = maxOperationSizeInBytes; } public int getMaxWriteSize() { @@ -165,7 +165,7 @@ public void readFrom(StreamInput in) throws IOException { followIndex = in.readString(); maxReadSize = in.readVInt(); maxConcurrentReads = in.readVInt(); - processorMaxTranslogBytes = in.readVLong(); + maxOperationSizeInBytes = in.readVLong(); maxWriteSize = in.readVInt(); maxConcurrentWrites = in.readVInt(); maxBufferSize = in.readVInt(); @@ -178,7 +178,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(followIndex); out.writeVInt(maxReadSize); out.writeVInt(maxConcurrentReads); - out.writeVLong(processorMaxTranslogBytes); + out.writeVLong(maxOperationSizeInBytes); out.writeVInt(maxWriteSize); out.writeVInt(maxConcurrentWrites); out.writeVInt(maxBufferSize); @@ -191,7 +191,7 @@ public boolean equals(Object o) { Request request = (Request) o; return maxReadSize == request.maxReadSize && maxConcurrentReads == request.maxConcurrentReads && - processorMaxTranslogBytes == request.processorMaxTranslogBytes && + maxOperationSizeInBytes == request.maxOperationSizeInBytes && maxWriteSize == request.maxWriteSize && maxConcurrentWrites == request.maxConcurrentWrites && maxBufferSize == request.maxBufferSize && @@ -201,7 +201,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(leaderIndex, followIndex, maxReadSize, maxConcurrentReads, processorMaxTranslogBytes, + return Objects.hash(leaderIndex, followIndex, maxReadSize, maxConcurrentReads, maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize); } } @@ -300,7 +300,7 @@ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMe ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, new ShardId(followIndexMetadata.getIndex(), shardId), new ShardId(leaderIndexMetadata.getIndex(), shardId), - request.maxReadSize, request.maxConcurrentReads, request.processorMaxTranslogBytes, + request.maxReadSize, request.maxConcurrentReads, request.maxOperationSizeInBytes, request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, new ActionListener>() { 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 317594dc2bf6f..cea84647cc349 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 @@ -56,7 +56,7 @@ public static class Request extends SingleShardRequest { private long minSeqNo; private long maxSeqNo; private ShardId shardId; - private long maxTranslogsBytes = ShardFollowNodeTask.DEFAULT_MAX_TRANSLOG_BYTES; + private long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; public Request(ShardId shardId) { super(shardId.getIndexName()); @@ -86,12 +86,12 @@ public void setMaxSeqNo(long maxSeqNo) { this.maxSeqNo = maxSeqNo; } - public long getMaxTranslogsBytes() { - return maxTranslogsBytes; + public long getMaxOperationSizeInBytes() { + return maxOperationSizeInBytes; } - public void setMaxTranslogsBytes(long maxTranslogsBytes) { - this.maxTranslogsBytes = maxTranslogsBytes; + public void setMaxOperationSizeInBytes(long maxOperationSizeInBytes) { + this.maxOperationSizeInBytes = maxOperationSizeInBytes; } @Override @@ -104,8 +104,8 @@ public ActionRequestValidationException validate() { validationException = addValidationError("minSeqNo [" + minSeqNo + "] cannot be larger than maxSeqNo [" + maxSeqNo + "]", validationException); } - if (maxTranslogsBytes <= 0) { - validationException = addValidationError("maxTranslogsBytes [" + maxTranslogsBytes + "] must be larger than 0", + if (maxOperationSizeInBytes <= 0) { + validationException = addValidationError("maxOperationSizeInBytes [" + maxOperationSizeInBytes + "] must be larger than 0", validationException); } return validationException; @@ -117,7 +117,7 @@ public void readFrom(StreamInput in) throws IOException { minSeqNo = in.readVLong(); maxSeqNo = in.readVLong(); shardId = ShardId.readShardId(in); - maxTranslogsBytes = in.readVLong(); + maxOperationSizeInBytes = in.readVLong(); } @Override @@ -126,7 +126,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(minSeqNo); out.writeVLong(maxSeqNo); shardId.writeTo(out); - out.writeVLong(maxTranslogsBytes); + out.writeVLong(maxOperationSizeInBytes); } @@ -138,12 +138,12 @@ public boolean equals(final Object o) { return minSeqNo == request.minSeqNo && maxSeqNo == request.maxSeqNo && Objects.equals(shardId, request.shardId) && - maxTranslogsBytes == request.maxTranslogsBytes; + maxOperationSizeInBytes == request.maxOperationSizeInBytes; } @Override public int hashCode() { - return Objects.hash(minSeqNo, maxSeqNo, shardId, maxTranslogsBytes); + return Objects.hash(minSeqNo, maxSeqNo, shardId, maxOperationSizeInBytes); } } @@ -234,7 +234,7 @@ protected Response shardOperation(Request request, ShardId shardId) throws IOExc final long indexMetaDataVersion = clusterService.state().metaData().index(shardId.getIndex()).getVersion(); final Translog.Operation[] operations = - getOperationsBetween(indexShard, request.minSeqNo, request.maxSeqNo, request.maxTranslogsBytes); + getOperationsBetween(indexShard, request.minSeqNo, request.maxSeqNo, request.maxOperationSizeInBytes); return new Response(indexMetaDataVersion, indexShard.getGlobalCheckpoint(), operations); } 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 1228e44b5d8f7..10162d9653b76 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 @@ -61,7 +61,7 @@ public class ShardFollowNodeTask extends AllocatedPersistentTask { static final int DEFAULT_MAX_CONCURRENT_READS = 1; static final int DEFAULT_MAX_CONCURRENT_WRITES = 1; static final int DEFAULT_MAX_BUFFER_SIZE = 10240; - static final long DEFAULT_MAX_TRANSLOG_BYTES= Long.MAX_VALUE; + static final long DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES = Long.MAX_VALUE; private static final TimeValue RETRY_TIMEOUT = TimeValue.timeValueMillis(500); private static final Logger LOGGER = Loggers.getLogger(ShardFollowNodeTask.class); @@ -349,7 +349,7 @@ protected void innerSendShardChangesRequest(long from, ShardChangesAction.Request request = new ShardChangesAction.Request(params.getLeaderShardId()); request.setMinSeqNo(from); request.setMaxSeqNo(to); - request.setMaxTranslogsBytes(params.getMaxTranslogBytes()); + request.setMaxOperationSizeInBytes(params.getMaxOperationSizeInBytes()); leaderClient.execute(ShardChangesAction.INSTANCE, request, new ActionListener() { @Override public void onResponse(ShardChangesAction.Response response) { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index a92172b390708..23860995c0a4d 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -42,7 +42,7 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { static final ParseField HEADERS = new ParseField("headers"); public static final ParseField MAX_READ_SIZE = new ParseField("max_read_size"); public static final ParseField MAX_CONCURRENT_READS = new ParseField("max_concurrent_reads"); - public static final ParseField MAX_TRANSLOG_BYTES_PER_REQUEST = new ParseField("max_translog_bytes"); + public static final ParseField MAX_OPERATION_SIZE_IN_BYTES = new ParseField("max_operation_size_in_bytes"); public static final ParseField MAX_WRITE_SIZE = new ParseField("max_write_size"); public static final ParseField MAX_CONCURRENT_WRITES = new ParseField("max_concurrent_writes"); public static final ParseField MAX_BUFFER_SIZE = new ParseField("max_buffer_size"); @@ -63,7 +63,7 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { PARSER.declareInt(ConstructingObjectParser.constructorArg(), LEADER_SHARD_SHARDID_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_READ_SIZE); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READS); - PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_TRANSLOG_BYTES_PER_REQUEST); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_OPERATION_SIZE_IN_BYTES); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_SIZE); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITES); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_BUFFER_SIZE); @@ -75,21 +75,21 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { private final ShardId leaderShardId; private final int maxReadSize; private final int maxConcurrentReads; - private final long maxTranslogBytes; + private final long maxOperationSizeInBytes; private final int maxWriteSize; private final int maxConcurrentWrites; private final int maxBufferSize; private final Map headers; ShardFollowTask(String leaderClusterAlias, ShardId followShardId, ShardId leaderShardId, int maxReadSize, - int maxConcurrentReads, long maxTranslogBytes, int maxWriteSize, int maxConcurrentWrites, + int maxConcurrentReads, long maxOperationSizeInBytes, int maxWriteSize, int maxConcurrentWrites, int maxBufferSize, Map headers) { this.leaderClusterAlias = leaderClusterAlias; this.followShardId = followShardId; this.leaderShardId = leaderShardId; this.maxReadSize = maxReadSize; this.maxConcurrentReads = maxConcurrentReads; - this.maxTranslogBytes = maxTranslogBytes; + this.maxOperationSizeInBytes = maxOperationSizeInBytes; this.maxWriteSize = maxWriteSize; this.maxConcurrentWrites = maxConcurrentWrites; this.maxBufferSize = maxBufferSize; @@ -102,7 +102,7 @@ public ShardFollowTask(StreamInput in) throws IOException { this.leaderShardId = ShardId.readShardId(in); this.maxReadSize = in.readVInt(); this.maxConcurrentReads = in.readVInt(); - this.maxTranslogBytes = in.readVLong(); + this.maxOperationSizeInBytes = in.readVLong(); this.maxWriteSize = in.readVInt(); this.maxConcurrentWrites= in.readVInt(); this.maxBufferSize = in.readVInt(); @@ -141,8 +141,8 @@ public int getMaxBufferSize() { return maxBufferSize; } - public long getMaxTranslogBytes() { - return maxTranslogBytes; + public long getMaxOperationSizeInBytes() { + return maxOperationSizeInBytes; } public Map getHeaders() { @@ -161,7 +161,7 @@ public void writeTo(StreamOutput out) throws IOException { leaderShardId.writeTo(out); out.writeVLong(maxReadSize); out.writeVInt(maxConcurrentReads); - out.writeVLong(maxTranslogBytes); + out.writeVLong(maxOperationSizeInBytes); out.writeVInt(maxWriteSize); out.writeVInt(maxConcurrentWrites); out.writeVInt(maxBufferSize); @@ -205,7 +205,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(MAX_CONCURRENT_READS.getPreferredName(), maxConcurrentReads); } { - builder.field(MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName(), maxTranslogBytes); + builder.field(MAX_OPERATION_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); } { builder.field(MAX_WRITE_SIZE.getPreferredName(), maxWriteSize); @@ -234,7 +234,7 @@ public boolean equals(Object o) { maxConcurrentReads == that.maxConcurrentReads && maxWriteSize == that.maxWriteSize && maxConcurrentWrites == that.maxConcurrentWrites && - maxTranslogBytes == that.maxTranslogBytes && + maxOperationSizeInBytes == that.maxOperationSizeInBytes && maxBufferSize == that.maxBufferSize && Objects.equals(headers, that.headers); } @@ -242,7 +242,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(leaderClusterAlias, followShardId, leaderShardId, maxReadSize, maxConcurrentReads, - maxWriteSize, maxConcurrentWrites, maxTranslogBytes, maxBufferSize, headers); + maxWriteSize, maxConcurrentWrites, maxOperationSizeInBytes, maxBufferSize, headers); } public String toString() { 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 cc1a82f5787f5..34b31c229fbf6 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 @@ -46,9 +46,9 @@ static Request createRequest(RestRequest restRequest) { if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName())) { request.setMaxConcurrentReads(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName()))); } - if (restRequest.hasParam(ShardFollowTask.MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName())) { - long value = Long.valueOf(restRequest.param(ShardFollowTask.MAX_TRANSLOG_BYTES_PER_REQUEST.getPreferredName())); - request.setProcessorMaxTranslogBytes(value); + if (restRequest.hasParam(ShardFollowTask.MAX_OPERATION_SIZE_IN_BYTES.getPreferredName())) { + long value = Long.valueOf(restRequest.param(ShardFollowTask.MAX_OPERATION_SIZE_IN_BYTES.getPreferredName())); + request.setMaxOperationSizeInBytes(value); } if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())) { request.setMaxWriteSize(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName()))); 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 7c58be62e5c5e..1455ee3945931 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 @@ -459,7 +459,7 @@ public void testFollowIndex_lowMaxTranslogBytes() throws Exception { } final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); - followRequest.setProcessorMaxTranslogBytes(1024); + followRequest.setMaxOperationSizeInBytes(1024); followRequest.setLeaderIndex("index1"); followRequest.setFollowIndex("index2"); 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 73c4a192a63f5..9bd28958e8313 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 @@ -25,7 +25,7 @@ static FollowIndexAction.Request createTestRequest() { request.setFollowIndex(randomAlphaOfLength(4)); request.setMaxReadSize(randomIntBetween(1, Integer.MAX_VALUE)); request.setMaxConcurrentReads(randomIntBetween(1, Integer.MAX_VALUE)); - request.setProcessorMaxTranslogBytes(randomNonNegativeLong()); + request.setMaxOperationSizeInBytes(randomNonNegativeLong()); request.setMaxWriteSize(randomIntBetween(1, Integer.MAX_VALUE)); request.setMaxConcurrentWrites(randomIntBetween(1, Integer.MAX_VALUE)); request.setMaxConcurrentWrites(randomIntBetween(1, Integer.MAX_VALUE)); 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 0686cf4eef4f3..b9ec0808cf9ce 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 @@ -124,7 +124,7 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead failedRequests = new AtomicInteger(0); AtomicBoolean stopped = new AtomicBoolean(false); ShardFollowTask params = new ShardFollowTask(null, new ShardId("follow_index", "", 0), - new ShardId("leader_index", "", 0), maxReadSize, maxConcurrentReads, ShardFollowNodeTask.DEFAULT_MAX_TRANSLOG_BYTES, + new ShardId("leader_index", "", 0), maxReadSize, maxConcurrentReads, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, maxWriteSize, maxConcurrentWrites, bufferLimit, Collections.emptyMap()); BiConsumer scheduler = (delay, task) -> { From f3d58e07379747e4488c6de2f488d72e2259dceb Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 12:29:28 +0200 Subject: [PATCH 05/41] changed shard changes api to be size based instead of range based --- .../xpack/ccr/action/ShardChangesAction.java | 59 ++++++++++--------- .../xpack/ccr/action/ShardFollowNodeTask.java | 57 +++++++++++------- .../xpack/ccr/ShardChangesIT.java | 8 +-- .../ccr/action/ShardChangesActionTests.java | 17 +----- .../ccr/action/ShardChangesRequestTests.java | 15 ++--- .../ccr/action/ShardFollowNodeTaskTests.java | 8 +-- 6 files changed, 84 insertions(+), 80 deletions(-) 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 cea84647cc349..e6ece0275e697 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 @@ -53,8 +53,8 @@ public Response newResponse() { public static class Request extends SingleShardRequest { - private long minSeqNo; - private long maxSeqNo; + private long fromSeqNo; + private long maxOperationCount; private ShardId shardId; private long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; @@ -70,20 +70,20 @@ public ShardId getShard() { return shardId; } - public long getMinSeqNo() { - return minSeqNo; + public long getFromSeqNo() { + return fromSeqNo; } - public void setMinSeqNo(long minSeqNo) { - this.minSeqNo = minSeqNo; + public void setFromSeqNo(long fromSeqNo) { + this.fromSeqNo = fromSeqNo; } - public long getMaxSeqNo() { - return maxSeqNo; + public long getMaxOperationCount() { + return maxOperationCount; } - public void setMaxSeqNo(long maxSeqNo) { - this.maxSeqNo = maxSeqNo; + public void setMaxOperationCount(long maxOperationCount) { + this.maxOperationCount = maxOperationCount; } public long getMaxOperationSizeInBytes() { @@ -97,12 +97,12 @@ public void setMaxOperationSizeInBytes(long maxOperationSizeInBytes) { @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; - if (minSeqNo < 0) { - validationException = addValidationError("minSeqNo [" + minSeqNo + "] cannot be lower than 0", validationException); + if (fromSeqNo < 0) { + validationException = addValidationError("fromSeqNo [" + fromSeqNo + "] cannot be lower than 0", validationException); } - if (maxSeqNo < minSeqNo) { - validationException = addValidationError("minSeqNo [" + minSeqNo + "] cannot be larger than maxSeqNo [" - + maxSeqNo + "]", validationException); + if (maxOperationCount < 0) { + validationException = addValidationError("maxOperationCount [" + maxOperationCount + + "] cannot be lower than 0", validationException); } if (maxOperationSizeInBytes <= 0) { validationException = addValidationError("maxOperationSizeInBytes [" + maxOperationSizeInBytes + "] must be larger than 0", @@ -114,8 +114,8 @@ public ActionRequestValidationException validate() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - minSeqNo = in.readVLong(); - maxSeqNo = in.readVLong(); + fromSeqNo = in.readVLong(); + maxOperationCount = in.readVLong(); shardId = ShardId.readShardId(in); maxOperationSizeInBytes = in.readVLong(); } @@ -123,8 +123,8 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeVLong(minSeqNo); - out.writeVLong(maxSeqNo); + out.writeVLong(fromSeqNo); + out.writeVLong(maxOperationCount); shardId.writeTo(out); out.writeVLong(maxOperationSizeInBytes); } @@ -135,15 +135,15 @@ public boolean equals(final Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; final Request request = (Request) o; - return minSeqNo == request.minSeqNo && - maxSeqNo == request.maxSeqNo && + return fromSeqNo == request.fromSeqNo && + maxOperationCount == request.maxOperationCount && Objects.equals(shardId, request.shardId) && maxOperationSizeInBytes == request.maxOperationSizeInBytes; } @Override public int hashCode() { - return Objects.hash(minSeqNo, maxSeqNo, shardId, maxOperationSizeInBytes); + return Objects.hash(fromSeqNo, maxOperationCount, shardId, maxOperationSizeInBytes); } } @@ -234,7 +234,7 @@ protected Response shardOperation(Request request, ShardId shardId) throws IOExc final long indexMetaDataVersion = clusterService.state().metaData().index(shardId.getIndex()).getVersion(); final Translog.Operation[] operations = - getOperationsBetween(indexShard, request.minSeqNo, request.maxSeqNo, request.maxOperationSizeInBytes); + getOperationsBetween(indexShard, request.fromSeqNo, request.maxOperationCount, request.maxOperationSizeInBytes); return new Response(indexMetaDataVersion, indexShard.getGlobalCheckpoint(), operations); } @@ -260,23 +260,24 @@ protected Response newResponse() { private static final Translog.Operation[] EMPTY_OPERATIONS_ARRAY = new Translog.Operation[0]; - static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long minSeqNo, long maxSeqNo, - long byteLimit) throws IOException { + static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long fromSeqNo, long maxOperationCount, + long maxOperationSizeInBytes) throws IOException { if (indexShard.state() != IndexShardState.STARTED) { throw new IndexShardNotStartedException(indexShard.shardId(), indexShard.state()); } int seenBytes = 0; + long toSeqNo = fromSeqNo + maxOperationCount; final List operations = new ArrayList<>(); - try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", minSeqNo, maxSeqNo, true)) { + try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", fromSeqNo, toSeqNo, true)) { Translog.Operation op; while ((op = snapshot.next()) != null) { if (op.getSource() == null) { - throw new IllegalStateException("source not found for operation: " + op + " minSeqNo: " + minSeqNo + " maxSeqNo: " + - maxSeqNo); + throw new IllegalStateException("source not found for operation: " + op + " fromSeqNo: " + fromSeqNo + + " maxOperationCount: " + maxOperationCount); } operations.add(op); seenBytes += op.estimateSize(); - if (seenBytes > byteLimit) { + if (seenBytes > maxOperationSizeInBytes) { break; } } 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 10162d9653b76..326dc7e252573 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 @@ -141,10 +141,15 @@ private synchronized void coordinateReads() { } numConcurrentReads++; long from = lastRequestedSeqno + 1; - long to = from + maxReadSize <= leaderGlobalCheckpoint ? from + maxReadSize : leaderGlobalCheckpoint; - LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, to); - sendShardChangesRequest(from, to, true); - lastRequestedSeqno = to; + long size; + if (from + maxReadSize <= leaderGlobalCheckpoint) { + size = maxReadSize; + } else { + size = leaderGlobalCheckpoint - from; + } + LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, size); + sendShardChangesRequest(from, size, true); + lastRequestedSeqno = from + size; } if (numConcurrentReads == 0) { LOGGER.trace("{} re-scheduling coordinate reads phase", params.getFollowShardId()); @@ -160,7 +165,7 @@ private synchronized void coordinateReads() { numConcurrentReads++; long from = lastRequestedSeqno + 1; LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, from + maxReadSize, false); + sendShardChangesRequest(from, maxReadSize, false); } }); } @@ -189,25 +194,31 @@ private synchronized void coordinateWrites() { } } - private void sendShardChangesRequest(long from, long to, boolean bla) { - innerSendShardChangesRequest(from, to, + private void sendShardChangesRequest(long from, long size, boolean checkLastOpReceived) { + innerSendShardChangesRequest(from, size, response -> { retryCounter.set(0); - handleResponse(from, to, bla, response); + handleResponse(from, size, checkLastOpReceived, response); }, - e -> handleFailure(e, () -> sendShardChangesRequest(from, to, bla))); + e -> handleFailure(e, () -> sendShardChangesRequest(from, size, checkLastOpReceived))); } - private synchronized void handleResponse(long from, long to, boolean checkLastOpReceived, ShardChangesAction.Response response) { + private synchronized void handleResponse(long from, long size, boolean checkLastOpReceived, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getLeaderGlobalCheckpoint()); if (response.getOperations().length == 0) { - numConcurrentReads--; - if (numConcurrentWrites == 0) { - coordinateWrites(); + if (checkLastOpReceived) { + LOGGER.trace("{} received no ops while [{}/{}] was expected, re-executing read...", + params.getFollowShardId(), from, size); + sendShardChangesRequest(from, size, true); + } else { + numConcurrentReads--; + if (numConcurrentWrites == 0) { + coordinateWrites(); + } + coordinateReads(); } - coordinateReads(); } else { Translog.Operation firstOp = response.getOperations()[0]; assert firstOp.seqNo() == from; @@ -216,11 +227,13 @@ private synchronized void handleResponse(long from, long to, boolean checkLastOp LOGGER.trace("{} received [{}/{}]", params.getFollowShardId(), firstOp.seqNo(), lastOp.seqNo()); buffer.addAll(Arrays.asList(response.getOperations())); if (checkLastOpReceived) { - if (lastOp.seqNo() < to) { + long expectedLastSeqNo = from + size; + if (lastOp.seqNo() < expectedLastSeqNo) { long newFrom = lastOp.seqNo() + 1; + long newSize = expectedLastSeqNo - lastOp.seqNo(); LOGGER.trace("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", - params.getFollowShardId(), lastOp.seqNo(), to, newFrom, to); - sendShardChangesRequest(newFrom, to, true); + params.getFollowShardId(), lastOp.seqNo(), expectedLastSeqNo, newFrom, size); + sendShardChangesRequest(newFrom, newSize, true); } else { numConcurrentReads--; } @@ -343,12 +356,12 @@ public void onFailure(Exception e) { } protected void innerSendShardChangesRequest(long from, - Long to, - Consumer handler, - Consumer errorHandler) { + long size, + Consumer handler, + Consumer errorHandler) { ShardChangesAction.Request request = new ShardChangesAction.Request(params.getLeaderShardId()); - request.setMinSeqNo(from); - request.setMaxSeqNo(to); + request.setFromSeqNo(from); + request.setMaxOperationCount(size); request.setMaxOperationSizeInBytes(params.getMaxOperationSizeInBytes()); leaderClient.execute(ShardChangesAction.INSTANCE, request, new ActionListener() { @Override 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 1455ee3945931..c616d24b376b8 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 @@ -115,8 +115,8 @@ public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { assertThat(globalCheckPoint, equalTo(2L)); ShardChangesAction.Request request = new ShardChangesAction.Request(shardStats.getShardRouting().shardId()); - request.setMinSeqNo(0L); - request.setMaxSeqNo(globalCheckPoint); + request.setFromSeqNo(0L); + request.setMaxOperationCount(3L); ShardChangesAction.Response response = client().execute(ShardChangesAction.INSTANCE, request).get(); assertThat(response.getOperations().length, equalTo(3)); Translog.Index operation = (Translog.Index) response.getOperations()[0]; @@ -140,8 +140,8 @@ public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { assertThat(globalCheckPoint, equalTo(5L)); request = new ShardChangesAction.Request(shardStats.getShardRouting().shardId()); - request.setMinSeqNo(3L); - request.setMaxSeqNo(globalCheckPoint); + request.setFromSeqNo(3L); + request.setMaxOperationCount(3L); response = client().execute(ShardChangesAction.INSTANCE, request).get(); assertThat(response.getOperations().length, equalTo(3)); operation = (Translog.Index) response.getOperations()[0]; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index 2ecf49d2d7be8..037d464ddef0b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -5,12 +5,9 @@ */ package org.elasticsearch.xpack.ccr.action; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; -import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; @@ -35,7 +32,6 @@ public void testGetOperationsBetween() throws Exception { .put("index.number_of_replicas", 0) .build(); final IndexService indexService = createIndex("index", settings); - IndexMetaData indexMetaData = indexService.getMetaData(); final int numWrites = randomIntBetween(2, 8192); for (int i = 0; i < numWrites; i++) { @@ -48,7 +44,8 @@ public void testGetOperationsBetween() throws Exception { for (int iter = 0; iter < iters; iter++) { int min = randomIntBetween(0, numWrites - 1); int max = randomIntBetween(min, numWrites - 1); - final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, min, max, Long.MAX_VALUE); + int size = max - min; + final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, min, size, Long.MAX_VALUE); final List seenSeqNos = Arrays.stream(operations).map(Translog.Operation::seqNo).collect(Collectors.toList()); final List expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toList()); assertThat(seenSeqNos, equalTo(expectedSeqNos)); @@ -65,14 +62,6 @@ public void testGetOperationsBetween() throws Exception { } public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { - IndexMetaData indexMetaData = IndexMetaData.builder("index") - .settings(Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) - .build()) - .build(); IndexShard indexShard = Mockito.mock(IndexShard.class); ShardRouting shardRouting = TestShardRouting.newShardRouting("index", 0, "_node_id", true, ShardRoutingState.INITIALIZING); @@ -93,7 +82,7 @@ public void testGetOperationsBetweenExceedByteLimit() throws Exception { } final IndexShard indexShard = indexService.getShard(0); - final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, 0, numWrites - 1, 256); + final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, 0, 12, 256); assertThat(operations.length, equalTo(12)); assertThat(operations[0].seqNo(), equalTo(0L)); assertThat(operations[1].seqNo(), equalTo(1L)); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java index 3f30545576046..94eddc0e85446 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java @@ -16,8 +16,8 @@ public class ShardChangesRequestTests extends AbstractStreamableTestCase handler, + protected void innerSendShardChangesRequest(long from, long size, Consumer handler, Consumer errorHandler) { if (randomlyFailWithRetryableError.get() && writeCounter.incrementAndGet() % 5 == 0) { failedRequests.incrementAndGet(); @@ -184,12 +185,11 @@ protected void innerSendShardChangesRequest(long from, Long to, Consumer= leaderGlobalCheckpoint) { response = new ShardChangesAction.Response(1L, leaderGlobalCheckpoint, new Translog.Operation[0]); } else { - int size = to == null ? 100 : (int) (to - from + 1); if (randomlyTruncateRequests.get() && size > 10 && truncatedRequests.get() < 5) { truncatedRequests.incrementAndGet(); size = size / 2; } - Translog.Operation[] ops = new Translog.Operation[size]; + Translog.Operation[] ops = new Translog.Operation[(int) size + 1]; for (int i = 0; i < ops.length; i++) { ops[i] = new Translog.Index("doc", UUIDs.randomBase64UUID(), from + i, 0, "{}".getBytes(StandardCharsets.UTF_8)); } From 0330e5f10fd85b00435556bb0a24a67f2293d68d Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 16:08:25 +0200 Subject: [PATCH 06/41] start the node task only with followGlobalCheckpoint --- .../xpack/ccr/action/ShardFollowNodeTask.java | 35 +++++++++---------- .../ccr/action/ShardFollowTasksExecutor.java | 7 ++-- .../ccr/action/ShardFollowNodeTaskTests.java | 32 +++++++++-------- 3 files changed, 36 insertions(+), 38 deletions(-) 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 326dc7e252573..c2577540ffee5 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 @@ -98,17 +98,17 @@ public class ShardFollowNodeTask extends AllocatedPersistentTask { this.scheduler = scheduler; } - void start(long leaderGlobalCheckpoint, long followGlobalCheckpoint) { + void start(long followGlobalCheckpoint) { this.lastRequestedSeqno = followGlobalCheckpoint; this.processedGlobalCheckpoint = followGlobalCheckpoint; - this.leaderGlobalCheckpoint = leaderGlobalCheckpoint; + this.leaderGlobalCheckpoint = followGlobalCheckpoint; // Forcefully updates follower mapping, this gets us the leader imd version and // makes sure that leader and follower mapping are identical. updateMapping(imdVersion -> { currentIndexMetadataVersion = imdVersion; - LOGGER.info("{} Started to follow leader shard {}, leaderGlobalCheckPoint={}, followGlobalCheckPoint={}", - params.getFollowShardId(), params.getLeaderShardId(), leaderGlobalCheckpoint, followGlobalCheckpoint); + LOGGER.info("{} Started to follow leader shard {}, followGlobalCheckPoint={}", + params.getFollowShardId(), params.getLeaderShardId(), followGlobalCheckpoint); coordinateReads(); }); } @@ -148,7 +148,7 @@ private synchronized void coordinateReads() { size = leaderGlobalCheckpoint - from; } LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, size); - sendShardChangesRequest(from, size, true); + sendShardChangesRequest(from, size, from + size); lastRequestedSeqno = from + size; } if (numConcurrentReads == 0) { @@ -165,7 +165,7 @@ private synchronized void coordinateReads() { numConcurrentReads++; long from = lastRequestedSeqno + 1; LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, maxReadSize, false); + sendShardChangesRequest(from, maxReadSize, null); } }); } @@ -194,24 +194,24 @@ private synchronized void coordinateWrites() { } } - private void sendShardChangesRequest(long from, long size, boolean checkLastOpReceived) { + private void sendShardChangesRequest(long from, long size, Long targetSeqNo) { innerSendShardChangesRequest(from, size, response -> { retryCounter.set(0); - handleResponse(from, size, checkLastOpReceived, response); + handleResponse(from, size, targetSeqNo, response); }, - e -> handleFailure(e, () -> sendShardChangesRequest(from, size, checkLastOpReceived))); + e -> handleFailure(e, () -> sendShardChangesRequest(from, size, targetSeqNo))); } - private synchronized void handleResponse(long from, long size, boolean checkLastOpReceived, ShardChangesAction.Response response) { + private synchronized void handleResponse(long from, long size, Long targetSeqNo, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getLeaderGlobalCheckpoint()); if (response.getOperations().length == 0) { - if (checkLastOpReceived) { + if (targetSeqNo != null) { LOGGER.trace("{} received no ops while [{}/{}] was expected, re-executing read...", params.getFollowShardId(), from, size); - sendShardChangesRequest(from, size, true); + sendShardChangesRequest(from, size, targetSeqNo); } else { numConcurrentReads--; if (numConcurrentWrites == 0) { @@ -226,14 +226,13 @@ private synchronized void handleResponse(long from, long size, boolean checkLast LOGGER.trace("{} received [{}/{}]", params.getFollowShardId(), firstOp.seqNo(), lastOp.seqNo()); buffer.addAll(Arrays.asList(response.getOperations())); - if (checkLastOpReceived) { - long expectedLastSeqNo = from + size; - if (lastOp.seqNo() < expectedLastSeqNo) { + if (targetSeqNo != null) { + if (lastOp.seqNo() < targetSeqNo) { long newFrom = lastOp.seqNo() + 1; - long newSize = expectedLastSeqNo - lastOp.seqNo(); + long newSize = targetSeqNo - lastOp.seqNo(); LOGGER.trace("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", - params.getFollowShardId(), lastOp.seqNo(), expectedLastSeqNo, newFrom, size); - sendShardChangesRequest(newFrom, newSize, true); + params.getFollowShardId(), lastOp.seqNo(), targetSeqNo, newFrom, size); + sendShardChangesRequest(newFrom, newSize, targetSeqNo); } else { numConcurrentReads--; } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 16d5cfcb69ccc..39afa8486eb02 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -91,11 +91,8 @@ protected AllocatedPersistentTask createTask(long id, String type, String action protected void nodeOperation(final AllocatedPersistentTask task, final ShardFollowTask params, final PersistentTaskState state) { ShardFollowNodeTask shardFollowNodeTask = (ShardFollowNodeTask) task; logger.info("{} Started to track leader shard {}", params.getFollowShardId(), params.getLeaderShardId()); - fetchGlobalCheckpoint(shardFollowNodeTask.leaderClient, params.getLeaderShardId(), leaderGlobalCheckPoint -> { - fetchGlobalCheckpoint(shardFollowNodeTask.followerClient, params.getFollowShardId(), followGlobalCheckPoint -> { - shardFollowNodeTask.start(leaderGlobalCheckPoint, followGlobalCheckPoint); - }, task::markAsFailed); - }, task::markAsFailed); + fetchGlobalCheckpoint(shardFollowNodeTask.followerClient, params.getFollowShardId(), shardFollowNodeTask::start, + task::markAsFailed); } private void fetchGlobalCheckpoint(Client client, ShardId shardId, LongConsumer handler, Consumer errorHandler) { 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 73a0912adb868..7b879a0f5f7a9 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 @@ -9,7 +9,6 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.seqno.LocalCheckpointTracker; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.ESTestCase; @@ -48,10 +47,11 @@ public class ShardFollowNodeTaskTests extends ESTestCase { private AtomicReference failureHolder = new AtomicReference<>(); public void testDefaults() throws Exception { + long followGlobalCheckpoint = randomIntBetween(-1, 2048); task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_READ_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, 10000, - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE); - task.start(randomIntBetween(-1, 2048), -1); + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, followGlobalCheckpoint); + task.start(followGlobalCheckpoint); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); @@ -62,8 +62,8 @@ public void testDefaults() throws Exception { public void testHitBufferLimit() throws Exception { // Setting buffer limit to 100, so that we are sure the limit will be met task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_READ_SIZE, 3, - ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, 1, 10000, 100); - task.start(-1, -1); + ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, 1, 10000, 100, -1); + task.start(-1); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); @@ -72,9 +72,10 @@ public void testHitBufferLimit() throws Exception { @TestLogging("org.elasticsearch.xpack.ccr.action:TRACE") public void testConcurrentReadsAndWrites() throws Exception { + long followGlobalCheckpoint = randomIntBetween(-1, 2048); task = createShardFollowTask(randomIntBetween(32, 2048), randomIntBetween(2, 10), randomIntBetween(32, 2048), - randomIntBetween(2, 10), 50000, 10240); - task.start(randomIntBetween(-1, 2048), -1); + randomIntBetween(2, 10), 50000, 10240, followGlobalCheckpoint); + task.start(followGlobalCheckpoint); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(50000L)); @@ -82,8 +83,8 @@ public void testConcurrentReadsAndWrites() throws Exception { } public void testMappingUpdate() throws Exception { - task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024); - task.start(-1, -1); + task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024, -1); + task.start(-1); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), greaterThanOrEqualTo(1000L)); @@ -96,8 +97,8 @@ public void testMappingUpdate() throws Exception { } public void testOccasionalApiFailure() throws Exception { - task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024); - task.start(-1, -1); + task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024, -1); + task.start(-1); randomlyFailWithRetryableError.set(true); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); @@ -106,8 +107,8 @@ public void testOccasionalApiFailure() throws Exception { } public void testNotAllExpectedOpsReturned() throws Exception { - task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024); - task.start(-1, -1); + task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024, -1); + task.start(-1); randomlyTruncateRequests.set(true); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); @@ -116,7 +117,8 @@ public void testNotAllExpectedOpsReturned() throws Exception { } ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentReads, int maxWriteSize, - int maxConcurrentWrites, int leaderGlobalCheckpoint, int bufferLimit) { + int maxConcurrentWrites, int leaderGlobalCheckpoint, int bufferLimit, + long followGlobalCheckpoint) { imdVersion = new AtomicLong(1L); mappingUpdateCounter = new AtomicInteger(0); randomlyTruncateRequests = new AtomicBoolean(false); @@ -139,7 +141,7 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead }; AtomicInteger readCounter = new AtomicInteger(); AtomicInteger writeCounter = new AtomicInteger(); - LocalCheckpointTracker tracker = new LocalCheckpointTracker(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); + LocalCheckpointTracker tracker = new LocalCheckpointTracker(followGlobalCheckpoint, followGlobalCheckpoint); return new ShardFollowNodeTask(1L, "type", ShardFollowTask.NAME, "description", null, Collections.emptyMap(), null, null, params, scheduler) { From adc829f9a00a63deca05ed36b259a68143170a69 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 16:23:26 +0200 Subject: [PATCH 07/41] iter --- .../xpack/ccr/action/ShardFollowNodeTask.java | 53 ++++++++++--------- 1 file changed, 29 insertions(+), 24 deletions(-) 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 c2577540ffee5..f72777f29c2f4 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 @@ -122,23 +122,8 @@ private synchronized void coordinateReads() { LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, leaderGlobalCheckpoint={}", params.getFollowShardId(), lastRequestedSeqno, leaderGlobalCheckpoint); final long maxReadSize = params.getMaxReadSize(); - final long maxConcurrentReads = params.getMaxConcurrentReads(); if (lastRequestedSeqno < leaderGlobalCheckpoint) { - while (true) { - if (lastRequestedSeqno >= leaderGlobalCheckpoint) { - LOGGER.trace("{} no new reads to coordinate lastRequestedSeqno [{}] leaderGlobalCheckpoint [{}]", - params.getLeaderShardId(), lastRequestedSeqno, leaderGlobalCheckpoint); - break; - } - if (numConcurrentReads >= maxConcurrentReads) { - LOGGER.trace("{} no new reads, maximum number of concurrent reads have been reached [{}]", - params.getFollowShardId(), numConcurrentReads); - break; - } - if (buffer.size() > params.getMaxBufferSize()) { - LOGGER.trace("{} no new reads, buffer limit has been reached [{}]", params.getFollowShardId(), buffer.size()); - break; - } + while (hasReadBudget() && lastRequestedSeqno < leaderGlobalCheckpoint) { numConcurrentReads++; long from = lastRequestedSeqno + 1; long size; @@ -172,12 +157,22 @@ private synchronized void coordinateReads() { } } + private boolean hasReadBudget() { + assert Thread.holdsLock(this); + if (numConcurrentReads >= params.getMaxConcurrentReads()) { + LOGGER.trace("{} no new reads, maximum number of concurrent reads have been reached [{}]", + params.getFollowShardId(), numConcurrentReads); + return false; + } + if (buffer.size() > params.getMaxBufferSize()) { + LOGGER.trace("{} no new reads, buffer limit has been reached [{}]", params.getFollowShardId(), buffer.size()); + return false; + } + return true; + } + private synchronized void coordinateWrites() { - while (true) { - if (buffer.isEmpty()) { - LOGGER.trace("{} no writes to coordinate, because buffer is empty", params.getFollowShardId()); - break; - } + while (hasWriteBudget() && buffer.isEmpty() == false) { if (numConcurrentWrites >= params.getMaxConcurrentWrites()) { LOGGER.trace("{} maximum number of concurrent writes have been reached [{}]", params.getFollowShardId(), numConcurrentWrites); @@ -194,6 +189,17 @@ private synchronized void coordinateWrites() { } } + private boolean hasWriteBudget() { + assert Thread.holdsLock(this); + if (numConcurrentWrites >= params.getMaxConcurrentWrites()) { + LOGGER.trace("{} maximum number of concurrent writes have been reached [{}]", + params.getFollowShardId(), numConcurrentWrites); + return false; + } + return true; + } + + private void sendShardChangesRequest(long from, long size, Long targetSeqNo) { innerSendShardChangesRequest(from, size, response -> { @@ -203,7 +209,7 @@ private void sendShardChangesRequest(long from, long size, Long targetSeqNo) { e -> handleFailure(e, () -> sendShardChangesRequest(from, size, targetSeqNo))); } - private synchronized void handleResponse(long from, long size, Long targetSeqNo, ShardChangesAction.Response response) { + private void handleResponse(long from, long size, Long targetSeqNo, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getLeaderGlobalCheckpoint()); @@ -268,8 +274,7 @@ private synchronized void handleResponse(long followerLocalCheckpoint) { coordinateWrites(); } - private void maybeUpdateMapping(Long minimumRequiredIndexMetadataVersion, Runnable task) { - assert Thread.holdsLock(this); + private synchronized void maybeUpdateMapping(Long minimumRequiredIndexMetadataVersion, Runnable task) { if (currentIndexMetadataVersion >= minimumRequiredIndexMetadataVersion) { LOGGER.trace("{} index metadata version [{}] is higher or equal than minimum required index metadata version [{}]", params.getFollowShardId(), currentIndexMetadataVersion, minimumRequiredIndexMetadataVersion); From 0b8e6a7f50f909c45e01a7c07ee1c9fb4a416649 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 16:42:16 +0200 Subject: [PATCH 08/41] coordinate reads should not schedule, but if there is budget at least do a single read. if the shard changes response returns no hits and had no target seqno then it should schedule coordinate reads phase. --- .../xpack/ccr/action/ShardFollowNodeTask.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) 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 f72777f29c2f4..c45f1aa9caa39 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 @@ -142,17 +142,12 @@ private synchronized void coordinateReads() { } } else { if (numConcurrentReads == 0) { - LOGGER.trace("{} scheduling peek read", params.getFollowShardId()); - scheduler.accept(TimeValue.timeValueMillis(500), () -> { - synchronized (this) { - // We sneak peek if there is any thing new in the leader primary. - // If there is we will happily accept - numConcurrentReads++; - long from = lastRequestedSeqno + 1; - LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, maxReadSize, null); - } - }); + // We sneak peek if there is any thing new in the leader primary. + // If there is we will happily accept + numConcurrentReads++; + long from = lastRequestedSeqno + 1; + LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); + sendShardChangesRequest(from, maxReadSize, null); } } } @@ -223,7 +218,8 @@ private void handleResponse(long from, long size, Long targetSeqNo, ShardChanges if (numConcurrentWrites == 0) { coordinateWrites(); } - coordinateReads(); + LOGGER.trace("{} received no ops, scheduling to coordinate reads", params.getFollowShardId()); + scheduler.accept(TimeValue.timeValueMillis(500), this::coordinateReads); } } else { Translog.Operation firstOp = response.getOperations()[0]; From 7b3fb30f0ef3ac0422634fbbc5903b5955cae6bd Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 16:43:30 +0200 Subject: [PATCH 09/41] s/leaderGlobalCheckpoint/globalCheckpoint --- .../xpack/ccr/action/ShardFollowNodeTask.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) 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 c45f1aa9caa39..f2bd8edf94e6f 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 @@ -72,7 +72,7 @@ public class ShardFollowNodeTask extends AllocatedPersistentTask { private final BiConsumer scheduler; private volatile long lastRequestedSeqno; - private volatile long leaderGlobalCheckpoint; + private volatile long globalCheckpoint; private volatile int numConcurrentReads = 0; private volatile int numConcurrentWrites = 0; @@ -101,7 +101,7 @@ public class ShardFollowNodeTask extends AllocatedPersistentTask { void start(long followGlobalCheckpoint) { this.lastRequestedSeqno = followGlobalCheckpoint; this.processedGlobalCheckpoint = followGlobalCheckpoint; - this.leaderGlobalCheckpoint = followGlobalCheckpoint; + this.globalCheckpoint = followGlobalCheckpoint; // Forcefully updates follower mapping, this gets us the leader imd version and // makes sure that leader and follower mapping are identical. @@ -119,18 +119,18 @@ private synchronized void coordinateReads() { return; } - LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, leaderGlobalCheckpoint={}", - params.getFollowShardId(), lastRequestedSeqno, leaderGlobalCheckpoint); + LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, globalCheckpoint={}", + params.getFollowShardId(), lastRequestedSeqno, globalCheckpoint); final long maxReadSize = params.getMaxReadSize(); - if (lastRequestedSeqno < leaderGlobalCheckpoint) { - while (hasReadBudget() && lastRequestedSeqno < leaderGlobalCheckpoint) { + if (lastRequestedSeqno < globalCheckpoint) { + while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { numConcurrentReads++; long from = lastRequestedSeqno + 1; long size; - if (from + maxReadSize <= leaderGlobalCheckpoint) { + if (from + maxReadSize <= globalCheckpoint) { size = maxReadSize; } else { - size = leaderGlobalCheckpoint - from; + size = globalCheckpoint - from; } LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, size); sendShardChangesRequest(from, size, from + size); @@ -207,7 +207,7 @@ private void sendShardChangesRequest(long from, long size, Long targetSeqNo) { private void handleResponse(long from, long size, Long targetSeqNo, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { - leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getLeaderGlobalCheckpoint()); + globalCheckpoint = Math.max(globalCheckpoint, response.getLeaderGlobalCheckpoint()); if (response.getOperations().length == 0) { if (targetSeqNo != null) { LOGGER.trace("{} received no ops while [{}/{}] was expected, re-executing read...", From a0059ae1302df42e2b2b9244726f8ad94c839b14 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 16:53:35 +0200 Subject: [PATCH 10/41] Separate the code that uses client and makes remote calls from ShardFollowNodeTask --- .../xpack/ccr/action/ShardFollowNodeTask.java | 86 ++----------------- .../ccr/action/ShardFollowTasksExecutor.java | 81 ++++++++++++++++- .../ccr/action/ShardFollowNodeTaskTests.java | 3 +- 3 files changed, 85 insertions(+), 85 deletions(-) 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 f2bd8edf94e6f..4a7312fa1b2f3 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 @@ -8,14 +8,8 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.UnavailableShardsException; -import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; -import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -26,17 +20,12 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.Index; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.transport.ActionTransportException; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; import java.io.IOException; import java.util.Arrays; @@ -53,7 +42,7 @@ * The node task that fetch the write operations from a leader shard and * persists these ops in the follower shard. */ -public class ShardFollowNodeTask extends AllocatedPersistentTask { +public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { static final int DEFAULT_MAX_READ_SIZE = 1024; static final int DEFAULT_MAX_WRITE_SIZE = 1024; @@ -66,8 +55,6 @@ public class ShardFollowNodeTask extends AllocatedPersistentTask { private static final Logger LOGGER = Loggers.getLogger(ShardFollowNodeTask.class); - final Client leaderClient; - final Client followerClient; private final ShardFollowTask params; private final BiConsumer scheduler; @@ -87,13 +74,9 @@ public class ShardFollowNodeTask extends AllocatedPersistentTask { String description, TaskId parentTask, Map headers, - Client leaderClient, - Client followerClient, ShardFollowTask params, BiConsumer scheduler) { super(id, type, action, description, parentTask, headers); - this.leaderClient = leaderClient; - this.followerClient = followerClient; this.params = params; this.scheduler = scheduler; } @@ -286,7 +269,7 @@ private synchronized void maybeUpdateMapping(Long minimumRequiredIndexMetadataVe } } - private void handleFailure(Exception e, Runnable task) { + void handleFailure(Exception e, Runnable task) { assert e != null; if (shouldRetry(e)) { if (isStopped() == false && retryCounter.incrementAndGet() <= RETRY_LIMIT) { @@ -313,68 +296,13 @@ private boolean shouldRetry(Exception e) { } // These methods are protected for testing purposes: - protected void updateMapping(LongConsumer handler) { - Index leaderIndex = params.getLeaderShardId().getIndex(); - Index followIndex = params.getFollowShardId().getIndex(); - - ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); - clusterStateRequest.clear(); - clusterStateRequest.metaData(true); - clusterStateRequest.indices(leaderIndex.getName()); - - leaderClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { - IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().getIndexSafe(leaderIndex); - assert indexMetaData.getMappings().size() == 1; - MappingMetaData mappingMetaData = indexMetaData.getMappings().iterator().next().value; - - PutMappingRequest putMappingRequest = new PutMappingRequest(followIndex.getName()); - putMappingRequest.type(mappingMetaData.type()); - putMappingRequest.source(mappingMetaData.source().string(), XContentType.JSON); - followerClient.admin().indices().putMapping(putMappingRequest, ActionListener.wrap( - putMappingResponse -> handler.accept(indexMetaData.getVersion()), - e -> handleFailure(e, () -> updateMapping(handler)))); - }, e -> handleFailure(e, () -> updateMapping(handler)))); - } - - protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, - LongConsumer handler, - Consumer errorHandler) { - final BulkShardOperationsRequest request = new BulkShardOperationsRequest(params.getFollowShardId(), operations); - followerClient.execute(BulkShardOperationsAction.INSTANCE, request, - new ActionListener() { - @Override - public void onResponse(BulkShardOperationsResponse response) { - handler.accept(response.getLocalCheckpoint()); - } - - @Override - public void onFailure(Exception e) { - errorHandler.accept(e); - } - } - ); - } + protected abstract void updateMapping(LongConsumer handler); - protected void innerSendShardChangesRequest(long from, - long size, - Consumer handler, - Consumer errorHandler) { - ShardChangesAction.Request request = new ShardChangesAction.Request(params.getLeaderShardId()); - request.setFromSeqNo(from); - request.setMaxOperationCount(size); - request.setMaxOperationSizeInBytes(params.getMaxOperationSizeInBytes()); - leaderClient.execute(ShardChangesAction.INSTANCE, request, new ActionListener() { - @Override - public void onResponse(ShardChangesAction.Response response) { - handler.accept(response); - } + protected abstract void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, LongConsumer handler, + Consumer errorHandler); - @Override - public void onFailure(Exception e) { - errorHandler.accept(e); - } - }); - } + protected abstract void innerSendShardChangesRequest(long from, long size, Consumer handler, + Consumer errorHandler); @Override protected void onCancelled() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 39afa8486eb02..9a6d0cc729cb3 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -9,6 +9,8 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.ShardStats; @@ -16,11 +18,16 @@ import org.elasticsearch.client.Client; import org.elasticsearch.client.FilterClient; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -28,6 +35,9 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.Ccr; +import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; +import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; +import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; import java.util.Arrays; import java.util.Map; @@ -83,16 +93,79 @@ protected AllocatedPersistentTask createTask(long id, String type, String action Client followerClient = wrapClient(client, params); BiConsumer scheduler = (delay, command) -> threadPool.schedule(delay, Ccr.CCR_THREAD_POOL_NAME, command); - return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers, - leaderClient, followerClient, params, scheduler); + return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers, params, scheduler) { + + @Override + protected void updateMapping(LongConsumer handler) { + Index leaderIndex = params.getLeaderShardId().getIndex(); + Index followIndex = params.getFollowShardId().getIndex(); + + ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.clear(); + clusterStateRequest.metaData(true); + clusterStateRequest.indices(leaderIndex.getName()); + + leaderClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { + IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().getIndexSafe(leaderIndex); + assert indexMetaData.getMappings().size() == 1; + MappingMetaData mappingMetaData = indexMetaData.getMappings().iterator().next().value; + + PutMappingRequest putMappingRequest = new PutMappingRequest(followIndex.getName()); + putMappingRequest.type(mappingMetaData.type()); + putMappingRequest.source(mappingMetaData.source().string(), XContentType.JSON); + followerClient.admin().indices().putMapping(putMappingRequest, ActionListener.wrap( + putMappingResponse -> handler.accept(indexMetaData.getVersion()), + e -> handleFailure(e, () -> updateMapping(handler)))); + }, e -> handleFailure(e, () -> updateMapping(handler)))); + } + + @Override + protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, LongConsumer handler, + Consumer errorHandler) { + final BulkShardOperationsRequest request = new BulkShardOperationsRequest(params.getFollowShardId(), operations); + followerClient.execute(BulkShardOperationsAction.INSTANCE, request, + new ActionListener() { + @Override + public void onResponse(BulkShardOperationsResponse response) { + handler.accept(response.getLocalCheckpoint()); + } + + @Override + public void onFailure(Exception e) { + errorHandler.accept(e); + } + } + ); + } + + @Override + protected void innerSendShardChangesRequest(long from, long size, Consumer handler, + Consumer errorHandler) { + ShardChangesAction.Request request = new ShardChangesAction.Request(params.getLeaderShardId()); + request.setFromSeqNo(from); + request.setMaxOperationCount(size); + request.setMaxOperationSizeInBytes(params.getMaxOperationSizeInBytes()); + leaderClient.execute(ShardChangesAction.INSTANCE, request, new ActionListener() { + @Override + public void onResponse(ShardChangesAction.Response response) { + handler.accept(response); + } + + @Override + public void onFailure(Exception e) { + errorHandler.accept(e); + } + }); + } + }; } @Override protected void nodeOperation(final AllocatedPersistentTask task, final ShardFollowTask params, final PersistentTaskState state) { + Client followerClient = wrapClient(client, params); ShardFollowNodeTask shardFollowNodeTask = (ShardFollowNodeTask) task; logger.info("{} Started to track leader shard {}", params.getFollowShardId(), params.getLeaderShardId()); - fetchGlobalCheckpoint(shardFollowNodeTask.followerClient, params.getFollowShardId(), shardFollowNodeTask::start, - task::markAsFailed); + fetchGlobalCheckpoint(followerClient, params.getFollowShardId(), shardFollowNodeTask::start, task::markAsFailed); } private void fetchGlobalCheckpoint(Client client, ShardId shardId, LongConsumer handler, Consumer errorHandler) { 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 7b879a0f5f7a9..755e6ea3ef176 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 @@ -142,8 +142,7 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead AtomicInteger readCounter = new AtomicInteger(); AtomicInteger writeCounter = new AtomicInteger(); LocalCheckpointTracker tracker = new LocalCheckpointTracker(followGlobalCheckpoint, followGlobalCheckpoint); - return new ShardFollowNodeTask(1L, "type", ShardFollowTask.NAME, "description", null, - Collections.emptyMap(), null, null, params, scheduler) { + return new ShardFollowNodeTask(1L, "type", ShardFollowTask.NAME, "description", null, Collections.emptyMap(), params, scheduler) { @Override protected void updateMapping(LongConsumer handler) { From a356fbea63691649d2912734e19b9674ad785ed7 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 17:03:05 +0200 Subject: [PATCH 11/41] iter, exception handling. --- .../elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 4a7312fa1b2f3..3866118017dc9 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,6 +10,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.UnavailableShardsException; +import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -21,11 +22,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.transport.ActionTransportException; import java.io.IOException; import java.util.Arrays; @@ -289,8 +288,7 @@ private boolean shouldRetry(Exception e) { // TODO: What other exceptions should be retried? return NetworkExceptionHelper.isConnectException(e) || NetworkExceptionHelper.isCloseConnectionException(e) || - e instanceof ActionTransportException || - e instanceof NodeClosedException || + TransportActions.isShardNotAvailableException(e) || e instanceof UnavailableShardsException || e instanceof NoShardAvailableActionException; } From fa6bb6f2cd004ea6dcfbbe469ae170c1e22d298d Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 17:25:50 +0200 Subject: [PATCH 12/41] iter --- .../xpack/ccr/action/ShardFollowNodeTask.java | 10 ++----- .../ccr/action/ShardFollowTasksExecutor.java | 26 ++----------------- 2 files changed, 4 insertions(+), 32 deletions(-) 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 3866118017dc9..aa274962edffd 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 @@ -67,14 +67,8 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { private final AtomicInteger retryCounter = new AtomicInteger(0); private final Queue buffer = new LinkedList<>(); - ShardFollowNodeTask(long id, - String type, - String action, - String description, - TaskId parentTask, - Map headers, - ShardFollowTask params, - BiConsumer scheduler) { + ShardFollowNodeTask(long id, String type, String action, String description, TaskId parentTask, Map headers, + ShardFollowTask params, BiConsumer scheduler) { super(id, type, action, description, parentTask, headers); this.params = params; this.scheduler = scheduler; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 9a6d0cc729cb3..d1a8477fbefde 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -37,7 +37,6 @@ import org.elasticsearch.xpack.ccr.Ccr; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; -import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; import java.util.Arrays; import java.util.Map; @@ -124,18 +123,7 @@ protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operatio Consumer errorHandler) { final BulkShardOperationsRequest request = new BulkShardOperationsRequest(params.getFollowShardId(), operations); followerClient.execute(BulkShardOperationsAction.INSTANCE, request, - new ActionListener() { - @Override - public void onResponse(BulkShardOperationsResponse response) { - handler.accept(response.getLocalCheckpoint()); - } - - @Override - public void onFailure(Exception e) { - errorHandler.accept(e); - } - } - ); + ActionListener.wrap(response -> handler.accept(response.getLocalCheckpoint()), errorHandler)); } @Override @@ -145,17 +133,7 @@ protected void innerSendShardChangesRequest(long from, long size, Consumer() { - @Override - public void onResponse(ShardChangesAction.Response response) { - handler.accept(response); - } - - @Override - public void onFailure(Exception e) { - errorHandler.accept(e); - } - }); + leaderClient.execute(ShardChangesAction.INSTANCE, request, ActionListener.wrap(handler::accept, errorHandler)); } }; } From e73d9904ec376c22cf50f95b4f1ecd016c96eead Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 28 Jun 2018 17:26:15 +0200 Subject: [PATCH 13/41] adjusted test --- .../engine/LuceneChangesSnapshotTests.java | 25 ++++++++----------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index f903032039cad..aeac3cd3de4cd 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -38,7 +38,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; public class LuceneChangesSnapshotTests extends EngineTestCase { @@ -54,9 +53,8 @@ public void testBasics() throws Exception { long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); // Empty engine try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, true)) { - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat(error.getMessage(), - containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + List ops = drainAll(snapshot); + assertThat(ops.size(), equalTo(0)); } try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, false)) { assertThat(snapshot, SnapshotMatchers.size(0)); @@ -97,13 +95,12 @@ searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), f try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { searcher = null; - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat(error.getMessage(), - containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); - }finally { + List ops = drainAll(snapshot); + assertThat(ops.size(), equalTo(0)); + } finally { IOUtils.close(searcher); } - }else { + } else { fromSeqNo = randomLongBetween(0, refreshedSeqNo); toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); @@ -111,17 +108,15 @@ searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), f searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { searcher = null; assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); - }finally { + } finally { IOUtils.close(searcher); } searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { searcher = null; - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat(error.getMessage(), - containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); - }finally { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); + } finally { IOUtils.close(searcher); } toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); @@ -130,7 +125,7 @@ searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), f searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { searcher = null; assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); - }finally { + } finally { IOUtils.close(searcher); } } From 942c777a3336e85b635fb05ef3a45d20f2aa7066 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 29 Jun 2018 08:08:21 +0200 Subject: [PATCH 14/41] improve coordinateReads() --- .../xpack/ccr/action/ShardFollowNodeTask.java | 46 ++++++++----------- .../ccr/action/ShardFollowNodeTaskTests.java | 22 ++++++--- 2 files changed, 35 insertions(+), 33 deletions(-) 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 aa274962edffd..decd372adde64 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 @@ -98,33 +98,27 @@ private synchronized void coordinateReads() { LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, globalCheckpoint={}", params.getFollowShardId(), lastRequestedSeqno, globalCheckpoint); final long maxReadSize = params.getMaxReadSize(); - if (lastRequestedSeqno < globalCheckpoint) { - while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { - numConcurrentReads++; - long from = lastRequestedSeqno + 1; - long size; - if (from + maxReadSize <= globalCheckpoint) { - size = maxReadSize; - } else { - size = globalCheckpoint - from; - } - LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, size); - sendShardChangesRequest(from, size, from + size); - lastRequestedSeqno = from + size; - } - if (numConcurrentReads == 0) { - LOGGER.trace("{} re-scheduling coordinate reads phase", params.getFollowShardId()); - scheduler.accept(TimeValue.timeValueMillis(500), this::coordinateReads); - } - } else { - if (numConcurrentReads == 0) { - // We sneak peek if there is any thing new in the leader primary. - // If there is we will happily accept - numConcurrentReads++; - long from = lastRequestedSeqno + 1; - LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, maxReadSize, null); + while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { + numConcurrentReads++; + long from = lastRequestedSeqno + 1; + long size; + if (from + maxReadSize <= globalCheckpoint) { + size = maxReadSize; + } else { + size = globalCheckpoint - from; } + LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, size); + sendShardChangesRequest(from, size, from + size); + lastRequestedSeqno = from + size; + } + + if (numConcurrentReads == 0) { + // We sneak peek if there is any thing new in the leader primary. + // If there is we will happily accept + numConcurrentReads++; + long from = lastRequestedSeqno + 1; + LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); + sendShardChangesRequest(from, maxReadSize, null); } } 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 755e6ea3ef176..5d5104058e99a 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 @@ -16,7 +16,9 @@ import org.junit.After; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -30,7 +32,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.nullValue; -@TestLogging("org.elasticsearch.xpack.ccr.action:DEBUG") +@TestLogging("org.elasticsearch.xpack.ccr.action:TRACE") public class ShardFollowNodeTaskTests extends ESTestCase { private ShardFollowNodeTask task; @@ -183,18 +185,24 @@ protected void innerSendShardChangesRequest(long from, long size, Consumer= leaderGlobalCheckpoint) { - response = new ShardChangesAction.Response(1L, leaderGlobalCheckpoint, new Translog.Operation[0]); + if (from > leaderGlobalCheckpoint) { + response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, new Translog.Operation[0]); } else { if (randomlyTruncateRequests.get() && size > 10 && truncatedRequests.get() < 5) { truncatedRequests.incrementAndGet(); size = size / 2; } - Translog.Operation[] ops = new Translog.Operation[(int) size + 1]; - for (int i = 0; i < ops.length; i++) { - ops[i] = new Translog.Index("doc", UUIDs.randomBase64UUID(), from + i, 0, "{}".getBytes(StandardCharsets.UTF_8)); + List ops = new ArrayList<>(); + for (int i = 0; i < size; i++) { + long seqNo = from + i; + if (seqNo <= leaderGlobalCheckpoint) { + String id = UUIDs.randomBase64UUID(); + byte[] source = "{}".getBytes(StandardCharsets.UTF_8); + ops.add(new Translog.Index("doc", id, seqNo, 0, source)); + } } - response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, ops); + response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, + ops.toArray(new Translog.Operation[0])); } // Emulate network thread and avoid SO: Thread thread = new Thread(() -> handler.accept(response)); From 9cdf3200c093f6427c83026f6b82da4f13592384 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 29 Jun 2018 08:40:13 +0200 Subject: [PATCH 15/41] improved assert --- .../xpack/ccr/action/ShardFollowTasksExecutor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index d1a8477fbefde..f9bb35a2d1791 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -106,7 +106,8 @@ protected void updateMapping(LongConsumer handler) { leaderClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().getIndexSafe(leaderIndex); - assert indexMetaData.getMappings().size() == 1; + assert indexMetaData.getMappings().size() == 1 : "expected exactly one mapping, but got [" + + indexMetaData.getMappings().size() + "]"; MappingMetaData mappingMetaData = indexMetaData.getMappings().iterator().next().value; PutMappingRequest putMappingRequest = new PutMappingRequest(followIndex.getName()); From e9d61c107375059fe3a98acb5949919e60390c0b Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 2 Jul 2018 14:12:45 +0200 Subject: [PATCH 16/41] iter --- .../index/engine/LuceneChangesSnapshot.java | 4 +- .../elasticsearch/xpack/ccr/CcrSettings.java | 11 +- .../action/CreateAndFollowIndexAction.java | 11 +- .../xpack/ccr/action/FollowIndexAction.java | 112 ++++++--------- .../xpack/ccr/action/ShardChangesAction.java | 42 +++--- .../xpack/ccr/action/ShardFollowNodeTask.java | 135 +++++++----------- .../xpack/ccr/action/ShardFollowTask.java | 6 +- .../ccr/action/ShardFollowTasksExecutor.java | 15 +- .../rest/RestCreateAndFollowIndexAction.java | 3 +- .../xpack/ccr/rest/RestFollowIndexAction.java | 29 ++-- .../xpack/ccr/ShardChangesIT.java | 72 ++++------ .../CreateAndFollowIndexRequestTests.java | 4 +- .../ccr/action/FollowIndexActionTests.java | 7 +- .../ccr/action/FollowIndexRequestTests.java | 14 +- .../ccr/action/ShardChangesActionTests.java | 15 +- .../ccr/action/ShardChangesRequestTests.java | 2 +- .../ccr/action/ShardFollowNodeTaskTests.java | 15 +- 17 files changed, 221 insertions(+), 276 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index 3a0e1ab80b8e4..367b0f822a870 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -142,8 +142,8 @@ private void rangeCheck(Translog.Operation op) { if (op != null) { final long expectedSeqNo = lastSeenSeqNo + 1; if (op.seqNo() != expectedSeqNo) { - throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + - "and max_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); + throw new IllegalStateException("Not all operations between from_seqno [" + fromSeqNo + "] " + + "and to_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java index 6960766bad0cb..da1a302c2e770 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java @@ -7,6 +7,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; +import org.elasticsearch.common.unit.TimeValue; import java.util.Arrays; import java.util.List; @@ -32,6 +33,13 @@ private CcrSettings() { public static final Setting CCR_FOLLOWING_INDEX_SETTING = Setting.boolSetting("index.xpack.ccr.following_index", false, Setting.Property.IndexScope); + /** + * Index setting for controlling the timeout between the next shard changes request when a shard follow + * task is idle. + */ + public static final Setting CCR_IDLE_SHARD_CHANGES_DELAY = Setting.timeSetting( + "index.xpack.ccr.idle_shard_changes_delay", TimeValue.timeValueSeconds(10), Property.NodeScope); + /** * The settings defined by CCR. * @@ -40,7 +48,8 @@ private CcrSettings() { static List> getSettings() { return Arrays.asList( CCR_ENABLED_SETTING, - CCR_FOLLOWING_INDEX_SETTING); + CCR_FOLLOWING_INDEX_SETTING, + CCR_IDLE_SHARD_CHANGES_DELAY); } } 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 index 78d6175f39efb..1a887b6d010bf 100644 --- 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 @@ -65,12 +65,15 @@ public static class Request extends AcknowledgedRequest { private FollowIndexAction.Request followRequest; - public FollowIndexAction.Request getFollowRequest() { - return followRequest; + public Request(FollowIndexAction.Request followRequest) { + this.followRequest = Objects.requireNonNull(followRequest); + } + + Request() { } - public void setFollowRequest(FollowIndexAction.Request followRequest) { - this.followRequest = followRequest; + public FollowIndexAction.Request getFollowRequest() { + return followRequest; } @Override 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 index a4ce794f93ebc..b375866ac4ebe 100644 --- 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 @@ -71,86 +71,64 @@ public static class Request extends ActionRequest { private String leaderIndex; private String followIndex; - private int maxReadSize = ShardFollowNodeTask.DEFAULT_MAX_READ_SIZE; - private int maxConcurrentReads = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS; - private long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; - private int maxWriteSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE; - private int maxConcurrentWrites = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES; - private int maxBufferSize = ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE; - - public String getLeaderIndex() { - return leaderIndex; - } - - public void setLeaderIndex(String leaderIndex) { - this.leaderIndex = leaderIndex; - } - - public String getFollowIndex() { - return followIndex; - } - - public void setFollowIndex(String followIndex) { - this.followIndex = followIndex; - } - - public int getMaxReadSize() { - return maxReadSize; - } - - public void setMaxReadSize(int maxReadSize) { - if (maxReadSize < 1) { - throw new IllegalArgumentException("Illegal batch_size [" + maxReadSize + "]"); + private int maxOperationCount; + private int maxConcurrentReads; + private long maxOperationSizeInBytes; + private int maxWriteSize; + private int maxConcurrentWrites; + private int maxBufferSize; + + public Request(String leaderIndex, String followIndex, int maxOperationCount, int maxConcurrentReads, long maxOperationSizeInBytes, + int maxWriteSize, int maxConcurrentWrites, int maxBufferSize) { + if (maxOperationCount < 1) { + throw new IllegalArgumentException("maxOperationCount must be larger than 0"); } - - this.maxReadSize = maxReadSize; - } - - public void setMaxConcurrentReads(int maxConcurrentReads) { if (maxConcurrentReads < 1) { throw new IllegalArgumentException("concurrent_processors must be larger than 0"); } - this.maxConcurrentReads = maxConcurrentReads; - } - - public void setMaxOperationSizeInBytes(long maxOperationSizeInBytes) { if (maxOperationSizeInBytes <= 0) { throw new IllegalArgumentException("processor_max_translog_bytes must be larger than 0"); } - this.maxOperationSizeInBytes = maxOperationSizeInBytes; - } - - public int getMaxWriteSize() { - return maxWriteSize; - } - - public void setMaxWriteSize(int maxWriteSize) { if (maxWriteSize < 1) { throw new IllegalArgumentException("maxWriteSize must be larger than 0"); } + if (maxConcurrentWrites < 1) { + throw new IllegalArgumentException("maxConcurrentWrites must be larger than 0"); + } + if (maxBufferSize < 1) { + throw new IllegalArgumentException("maxBufferSize must be larger than 0"); + } + + this.leaderIndex = Objects.requireNonNull(leaderIndex); + this.followIndex = Objects.requireNonNull(followIndex); + this.maxOperationCount = maxOperationCount; + this.maxConcurrentReads = maxConcurrentReads; + this.maxOperationSizeInBytes = maxOperationSizeInBytes; this.maxWriteSize = maxWriteSize; + this.maxConcurrentWrites = maxConcurrentWrites; + this.maxBufferSize = maxBufferSize; } - public int getMaxConcurrentWrites() { - return maxConcurrentWrites; + public Request(String leaderIndex, String followIndex) { + this(leaderIndex, followIndex, ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT, + ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, + ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE); } - public void setMaxConcurrentWrites(int maxConcurrentWrites) { - if (maxConcurrentWrites < 1) { - throw new IllegalArgumentException("maxConcurrentWrites must be larger than 0"); - } - this.maxConcurrentWrites = maxConcurrentWrites; + Request() { } - public int getMaxBufferSize() { - return maxBufferSize; + public String getLeaderIndex() { + return leaderIndex; } - public void setMaxBufferSize(int maxBufferSize) { - if (maxBufferSize < 1) { - throw new IllegalArgumentException("maxBufferSize must be larger than 0"); - } - this.maxBufferSize = maxBufferSize; + public String getFollowIndex() { + return followIndex; + } + + public int getMaxOperationCount() { + return maxOperationCount; } @Override @@ -163,7 +141,7 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); leaderIndex = in.readString(); followIndex = in.readString(); - maxReadSize = in.readVInt(); + maxOperationCount = in.readVInt(); maxConcurrentReads = in.readVInt(); maxOperationSizeInBytes = in.readVLong(); maxWriteSize = in.readVInt(); @@ -176,7 +154,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(leaderIndex); out.writeString(followIndex); - out.writeVInt(maxReadSize); + out.writeVInt(maxOperationCount); out.writeVInt(maxConcurrentReads); out.writeVLong(maxOperationSizeInBytes); out.writeVInt(maxWriteSize); @@ -189,7 +167,7 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; - return maxReadSize == request.maxReadSize && + return maxOperationCount == request.maxOperationCount && maxConcurrentReads == request.maxConcurrentReads && maxOperationSizeInBytes == request.maxOperationSizeInBytes && maxWriteSize == request.maxWriteSize && @@ -201,7 +179,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(leaderIndex, followIndex, maxReadSize, maxConcurrentReads, maxOperationSizeInBytes, + return Objects.hash(leaderIndex, followIndex, maxOperationCount, maxConcurrentReads, maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize); } } @@ -243,7 +221,7 @@ protected void doExecute(Task task, Request request, ActionListener li ClusterState localClusterState = clusterService.state(); IndexMetaData followIndexMetadata = localClusterState.getMetaData().index(request.followIndex); - String[] indices = new String[]{request.getLeaderIndex()}; + String[] indices = new String[]{request.leaderIndex}; Map> remoteClusterIndices = remoteClusterService.groupClusterIndices(indices, s -> false); if (remoteClusterIndices.containsKey(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)) { // Following an index in local cluster, so use local cluster state to fetch leader IndexMetaData: @@ -300,7 +278,7 @@ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMe ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, new ShardId(followIndexMetadata.getIndex(), shardId), new ShardId(leaderIndexMetadata.getIndex(), shardId), - request.maxReadSize, request.maxConcurrentReads, request.maxOperationSizeInBytes, + request.maxOperationCount, request.maxConcurrentReads, request.maxOperationSizeInBytes, request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, new ActionListener>() { 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 e6ece0275e697..7723539cf912d 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 @@ -54,7 +54,7 @@ public Response newResponse() { public static class Request extends SingleShardRequest { private long fromSeqNo; - private long maxOperationCount; + private int maxOperationCount; private ShardId shardId; private long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; @@ -78,11 +78,11 @@ public void setFromSeqNo(long fromSeqNo) { this.fromSeqNo = fromSeqNo; } - public long getMaxOperationCount() { + public int getMaxOperationCount() { return maxOperationCount; } - public void setMaxOperationCount(long maxOperationCount) { + public void setMaxOperationCount(int maxOperationCount) { this.maxOperationCount = maxOperationCount; } @@ -115,7 +115,7 @@ public ActionRequestValidationException validate() { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); fromSeqNo = in.readVLong(); - maxOperationCount = in.readVLong(); + maxOperationCount = in.readVInt(); shardId = ShardId.readShardId(in); maxOperationSizeInBytes = in.readVLong(); } @@ -124,7 +124,7 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeVLong(fromSeqNo); - out.writeVLong(maxOperationCount); + out.writeVInt(maxOperationCount); shardId.writeTo(out); out.writeVLong(maxOperationSizeInBytes); } @@ -150,15 +150,15 @@ public int hashCode() { public static final class Response extends ActionResponse { private long indexMetadataVersion; - private long leaderGlobalCheckpoint; + private long leaderCheckpoint; private Translog.Operation[] operations; Response() { } - Response(long indexMetadataVersion, long leaderGlobalCheckpoint, final Translog.Operation[] operations) { + Response(long indexMetadataVersion, long leaderCheckpoint, final Translog.Operation[] operations) { this.indexMetadataVersion = indexMetadataVersion; - this.leaderGlobalCheckpoint = leaderGlobalCheckpoint; + this.leaderCheckpoint = leaderCheckpoint; this.operations = operations; } @@ -166,8 +166,8 @@ public long getIndexMetadataVersion() { return indexMetadataVersion; } - public long getLeaderGlobalCheckpoint() { - return leaderGlobalCheckpoint; + public long getLeaderCheckpoint() { + return leaderCheckpoint; } public Translog.Operation[] getOperations() { @@ -178,7 +178,7 @@ public Translog.Operation[] getOperations() { public void readFrom(final StreamInput in) throws IOException { super.readFrom(in); indexMetadataVersion = in.readVLong(); - leaderGlobalCheckpoint = in.readZLong(); + leaderCheckpoint = in.readZLong(); operations = in.readArray(Translog.Operation::readOperation, Translog.Operation[]::new); } @@ -186,7 +186,7 @@ public void readFrom(final StreamInput in) throws IOException { public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); out.writeVLong(indexMetadataVersion); - out.writeZLong(leaderGlobalCheckpoint); + out.writeZLong(leaderCheckpoint); out.writeArray(Translog.Operation::writeOperation, operations); } @@ -196,7 +196,7 @@ public boolean equals(final Object o) { if (o == null || getClass() != o.getClass()) return false; final Response response = (Response) o; return indexMetadataVersion == response.indexMetadataVersion && - leaderGlobalCheckpoint == response.leaderGlobalCheckpoint && + leaderCheckpoint == response.leaderCheckpoint && Arrays.equals(operations, response.operations); } @@ -204,7 +204,7 @@ public boolean equals(final Object o) { public int hashCode() { int result = 1; result += Objects.hashCode(indexMetadataVersion); - result += Objects.hashCode(leaderGlobalCheckpoint); + result += Objects.hashCode(leaderCheckpoint); result += Arrays.hashCode(operations); return result; } @@ -231,11 +231,12 @@ public TransportAction(Settings settings, protected Response shardOperation(Request request, ShardId shardId) throws IOException { IndexService indexService = indicesService.indexServiceSafe(request.getShard().getIndex()); IndexShard indexShard = indexService.getShard(request.getShard().id()); + long globalCheckpoint = indexShard.getGlobalCheckpoint(); final long indexMetaDataVersion = clusterService.state().metaData().index(shardId.getIndex()).getVersion(); - final Translog.Operation[] operations = - getOperationsBetween(indexShard, request.fromSeqNo, request.maxOperationCount, request.maxOperationSizeInBytes); - return new Response(indexMetaDataVersion, indexShard.getGlobalCheckpoint(), operations); + final Translog.Operation[] operations = getOperationsBetween(indexShard, globalCheckpoint, request.fromSeqNo, + request.maxOperationCount, request.maxOperationSizeInBytes); + return new Response(indexMetaDataVersion, globalCheckpoint, operations); } @Override @@ -260,13 +261,16 @@ protected Response newResponse() { private static final Translog.Operation[] EMPTY_OPERATIONS_ARRAY = new Translog.Operation[0]; - static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long fromSeqNo, long maxOperationCount, + static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long globalCheckpoint, long fromSeqNo, int maxOperationCount, long maxOperationSizeInBytes) throws IOException { if (indexShard.state() != IndexShardState.STARTED) { throw new IndexShardNotStartedException(indexShard.shardId(), indexShard.state()); } + if (fromSeqNo > indexShard.getGlobalCheckpoint()) { + return EMPTY_OPERATIONS_ARRAY; + } int seenBytes = 0; - long toSeqNo = fromSeqNo + maxOperationCount; + long toSeqNo = Math.min(globalCheckpoint, fromSeqNo + maxOperationCount); final List operations = new ArrayList<>(); try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", fromSeqNo, toSeqNo, true)) { Translog.Operation op; 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 decd372adde64..edbd822ec04b4 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 @@ -8,8 +8,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.NoShardAvailableActionException; -import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; @@ -28,9 +26,11 @@ import java.io.IOException; import java.util.Arrays; -import java.util.LinkedList; +import java.util.Comparator; import java.util.Map; import java.util.Objects; +import java.util.OptionalLong; +import java.util.PriorityQueue; import java.util.Queue; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; @@ -43,18 +43,20 @@ */ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { - static final int DEFAULT_MAX_READ_SIZE = 1024; - static final int DEFAULT_MAX_WRITE_SIZE = 1024; - static final int RETRY_LIMIT = 10; - static final int DEFAULT_MAX_CONCURRENT_READS = 1; - static final int DEFAULT_MAX_CONCURRENT_WRITES = 1; - static final int DEFAULT_MAX_BUFFER_SIZE = 10240; - static final long DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES = Long.MAX_VALUE; + public static final int DEFAULT_MAX_OPERATION_COUNT = 1024; + public static final int DEFAULT_MAX_WRITE_SIZE = 1024; + public static final int DEFAULT_MAX_CONCURRENT_READS = 1; + public static final int DEFAULT_MAX_CONCURRENT_WRITES = 1; + public static final int DEFAULT_MAX_BUFFER_SIZE = 10240; + public static final long DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES = Long.MAX_VALUE; + public static final TimeValue DEFAULT_IDLE_SHARD_CHANGES_DELAY = TimeValue.timeValueSeconds(10); + private static final int RETRY_LIMIT = 10; private static final TimeValue RETRY_TIMEOUT = TimeValue.timeValueMillis(500); private static final Logger LOGGER = Loggers.getLogger(ShardFollowNodeTask.class); private final ShardFollowTask params; + private final TimeValue idleShardChangesRequestDelay; private final BiConsumer scheduler; private volatile long lastRequestedSeqno; @@ -65,13 +67,14 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { private volatile long processedGlobalCheckpoint = 0; private volatile long currentIndexMetadataVersion = 0; private final AtomicInteger retryCounter = new AtomicInteger(0); - private final Queue buffer = new LinkedList<>(); + private final Queue buffer = new PriorityQueue<>(Comparator.comparing(Translog.Operation::seqNo).reversed()); ShardFollowNodeTask(long id, String type, String action, String description, TaskId parentTask, Map headers, - ShardFollowTask params, BiConsumer scheduler) { + ShardFollowTask params, BiConsumer scheduler, TimeValue idleShardChangesRequestDelay) { super(id, type, action, description, parentTask, headers); this.params = params; this.scheduler = scheduler; + this.idleShardChangesRequestDelay = idleShardChangesRequestDelay; } void start(long followGlobalCheckpoint) { @@ -83,8 +86,8 @@ void start(long followGlobalCheckpoint) { // makes sure that leader and follower mapping are identical. updateMapping(imdVersion -> { currentIndexMetadataVersion = imdVersion; - LOGGER.info("{} Started to follow leader shard {}, followGlobalCheckPoint={}", - params.getFollowShardId(), params.getLeaderShardId(), followGlobalCheckpoint); + LOGGER.info("{} Started to follow leader shard {}, followGlobalCheckPoint={}, indexMetaDataVersion={}", + params.getFollowShardId(), params.getLeaderShardId(), followGlobalCheckpoint, imdVersion); coordinateReads(); }); } @@ -97,19 +100,14 @@ private synchronized void coordinateReads() { LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, globalCheckpoint={}", params.getFollowShardId(), lastRequestedSeqno, globalCheckpoint); - final long maxReadSize = params.getMaxReadSize(); + final int maxReadSize = params.getMaxReadSize(); while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { numConcurrentReads++; long from = lastRequestedSeqno + 1; - long size; - if (from + maxReadSize <= globalCheckpoint) { - size = maxReadSize; - } else { - size = globalCheckpoint - from; - } - LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, size); - sendShardChangesRequest(from, size, from + size); - lastRequestedSeqno = from + size; + LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, maxReadSize); + long maxRequiredSeqno = Math.min(globalCheckpoint, from + maxReadSize); + sendShardChangesRequest(from, maxReadSize, maxRequiredSeqno); + lastRequestedSeqno = maxRequiredSeqno; } if (numConcurrentReads == 0) { @@ -118,7 +116,7 @@ private synchronized void coordinateReads() { numConcurrentReads++; long from = lastRequestedSeqno + 1; LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, maxReadSize, null); + sendShardChangesRequest(from, maxReadSize, from); } } @@ -138,11 +136,6 @@ private boolean hasReadBudget() { private synchronized void coordinateWrites() { while (hasWriteBudget() && buffer.isEmpty() == false) { - if (numConcurrentWrites >= params.getMaxConcurrentWrites()) { - LOGGER.trace("{} maximum number of concurrent writes have been reached [{}]", - params.getFollowShardId(), numConcurrentWrites); - break; - } Translog.Operation[] ops = new Translog.Operation[Math.min(params.getMaxWriteSize(), buffer.size())]; for (int i = 0; i < ops.length; i++) { ops[i] = buffer.remove(); @@ -165,58 +158,42 @@ private boolean hasWriteBudget() { } - private void sendShardChangesRequest(long from, long size, Long targetSeqNo) { - innerSendShardChangesRequest(from, size, + private void sendShardChangesRequest(long from, int maxOperationCount, long maxRequiredSeqNo) { + innerSendShardChangesRequest(from, maxOperationCount, response -> { retryCounter.set(0); - handleResponse(from, size, targetSeqNo, response); + handleReadResponse(from, maxOperationCount, maxRequiredSeqNo, response); }, - e -> handleFailure(e, () -> sendShardChangesRequest(from, size, targetSeqNo))); + e -> handleFailure(e, () -> sendShardChangesRequest(from, maxOperationCount, maxRequiredSeqNo))); } - private void handleResponse(long from, long size, Long targetSeqNo, ShardChangesAction.Response response) { + private void handleReadResponse(long from, int maxOperationCount, long maxRequiredSeqNo, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { - globalCheckpoint = Math.max(globalCheckpoint, response.getLeaderGlobalCheckpoint()); - if (response.getOperations().length == 0) { - if (targetSeqNo != null) { - LOGGER.trace("{} received no ops while [{}/{}] was expected, re-executing read...", - params.getFollowShardId(), from, size); - sendShardChangesRequest(from, size, targetSeqNo); - } else { - numConcurrentReads--; - if (numConcurrentWrites == 0) { - coordinateWrites(); - } - LOGGER.trace("{} received no ops, scheduling to coordinate reads", params.getFollowShardId()); - scheduler.accept(TimeValue.timeValueMillis(500), this::coordinateReads); - } - } else { + globalCheckpoint = Math.max(globalCheckpoint, response.getLeaderCheckpoint()); + buffer.addAll(Arrays.asList(response.getOperations())); + coordinateWrites(); + + OptionalLong lastOp = Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::seqNo).max(); + if (lastOp.isPresent() && lastOp.getAsLong() < maxRequiredSeqNo) { + long newFrom = lastOp.getAsLong() + 1; + int newSize = (int) (maxRequiredSeqNo - lastOp.getAsLong()); + LOGGER.trace("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", + params.getFollowShardId(), lastOp.getAsLong(), maxRequiredSeqNo, newFrom, maxOperationCount); + sendShardChangesRequest(newFrom, newSize, maxRequiredSeqNo); + return; + } + + numConcurrentReads--; + if (response.getOperations().length != 0) { + LOGGER.trace("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); Translog.Operation firstOp = response.getOperations()[0]; + lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOp.getAsLong()); assert firstOp.seqNo() == from; - Translog.Operation lastOp = response.getOperations()[response.getOperations().length - 1]; - - LOGGER.trace("{} received [{}/{}]", params.getFollowShardId(), firstOp.seqNo(), lastOp.seqNo()); - buffer.addAll(Arrays.asList(response.getOperations())); - if (targetSeqNo != null) { - if (lastOp.seqNo() < targetSeqNo) { - long newFrom = lastOp.seqNo() + 1; - long newSize = targetSeqNo - lastOp.seqNo(); - LOGGER.trace("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", - params.getFollowShardId(), lastOp.seqNo(), targetSeqNo, newFrom, size); - sendShardChangesRequest(newFrom, newSize, targetSeqNo); - } else { - numConcurrentReads--; - } - } else { - lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOp.seqNo()); - LOGGER.trace("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); - numConcurrentReads--; - } - if (numConcurrentWrites == 0) { - coordinateWrites(); - } coordinateReads(); + } else { + LOGGER.trace("{} received no ops, scheduling to coordinate reads", params.getFollowShardId()); + scheduler.accept(idleShardChangesRequestDelay, this::coordinateReads); } assert numConcurrentReads >= 0; } @@ -227,13 +204,13 @@ private void sendBulkShardOperationsRequest(Translog.Operation[] operations) { innerSendBulkShardOperationsRequest(operations, followerLocalCheckpoint -> { retryCounter.set(0); - handleResponse(followerLocalCheckpoint); + handleWriteResponse(followerLocalCheckpoint); }, e -> handleFailure(e, () -> sendBulkShardOperationsRequest(operations)) ); } - private synchronized void handleResponse(long followerLocalCheckpoint) { + private synchronized void handleWriteResponse(long followerLocalCheckpoint) { processedGlobalCheckpoint = Math.max(processedGlobalCheckpoint, followerLocalCheckpoint); numConcurrentWrites--; assert numConcurrentWrites >= 0; @@ -260,8 +237,7 @@ void handleFailure(Exception e, Runnable task) { assert e != null; if (shouldRetry(e)) { if (isStopped() == false && retryCounter.incrementAndGet() <= RETRY_LIMIT) { - LOGGER.debug(new ParameterizedMessage("{} error during follow shard task, retrying...", - new Object[]{params.getFollowShardId()}, e)); + LOGGER.debug(new ParameterizedMessage("{} error during follow shard task, retrying...", params.getFollowShardId()), e); scheduler.accept(RETRY_TIMEOUT, task); } else { markAsFailed(new ElasticsearchException("retrying failed [" + retryCounter.get() + @@ -273,12 +249,9 @@ void handleFailure(Exception e, Runnable task) { } private boolean shouldRetry(Exception e) { - // TODO: What other exceptions should be retried? return NetworkExceptionHelper.isConnectException(e) || NetworkExceptionHelper.isCloseConnectionException(e) || - TransportActions.isShardNotAvailableException(e) || - e instanceof UnavailableShardsException || - e instanceof NoShardAvailableActionException; + TransportActions.isShardNotAvailableException(e); } // These methods are protected for testing purposes: @@ -287,7 +260,7 @@ private boolean shouldRetry(Exception e) { protected abstract void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, LongConsumer handler, Consumer errorHandler); - protected abstract void innerSendShardChangesRequest(long from, long size, Consumer handler, + protected abstract void innerSendShardChangesRequest(long from, int maxOperationCount, Consumer handler, Consumer errorHandler); @Override diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index 23860995c0a4d..96da24779d201 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -40,7 +40,7 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { static final ParseField LEADER_SHARD_INDEX_UUID_FIELD = new ParseField("leader_shard_index_uuid"); static final ParseField LEADER_SHARD_SHARDID_FIELD = new ParseField("leader_shard_shard"); static final ParseField HEADERS = new ParseField("headers"); - public static final ParseField MAX_READ_SIZE = new ParseField("max_read_size"); + public static final ParseField MAX_OPERATION_COUNT = new ParseField("max_operation_count"); public static final ParseField MAX_CONCURRENT_READS = new ParseField("max_concurrent_reads"); public static final ParseField MAX_OPERATION_SIZE_IN_BYTES = new ParseField("max_operation_size_in_bytes"); public static final ParseField MAX_WRITE_SIZE = new ParseField("max_write_size"); @@ -61,7 +61,7 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_SHARD_INDEX_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_SHARD_INDEX_UUID_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), LEADER_SHARD_SHARDID_FIELD); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_READ_SIZE); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_OPERATION_COUNT); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READS); PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_OPERATION_SIZE_IN_BYTES); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_SIZE); @@ -199,7 +199,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(LEADER_SHARD_SHARDID_FIELD.getPreferredName(), leaderShardId.id()); } { - builder.field(MAX_READ_SIZE.getPreferredName(), maxReadSize); + builder.field(MAX_OPERATION_COUNT.getPreferredName(), maxReadSize); } { builder.field(MAX_CONCURRENT_READS.getPreferredName(), maxConcurrentReads); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index f9bb35a2d1791..826921d6093e6 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -35,6 +35,7 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.Ccr; +import org.elasticsearch.xpack.ccr.CcrSettings; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; @@ -47,6 +48,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.elasticsearch.xpack.ccr.action.ShardFollowNodeTask.DEFAULT_IDLE_SHARD_CHANGES_DELAY; + public class ShardFollowTasksExecutor extends PersistentTasksExecutor { private final Client client; @@ -80,19 +83,19 @@ protected AllocatedPersistentTask createTask(long id, String type, String action PersistentTasksCustomMetaData.PersistentTask taskInProgress, Map headers) { ShardFollowTask params = taskInProgress.getParams(); - logger.info("{} Creating node task to track leader shard {}, params [{}]", - params.getFollowShardId(), params.getLeaderShardId(), params); - final Client leaderClient; if (params.getLeaderClusterAlias() != null) { leaderClient = wrapClient(client.getRemoteClusterClient(params.getLeaderClusterAlias()), params); } else { leaderClient = wrapClient(client, params); } + TimeValue idleShardChangesRequestDelay = + settings.getAsTime(CcrSettings.CCR_IDLE_SHARD_CHANGES_DELAY.getKey(), DEFAULT_IDLE_SHARD_CHANGES_DELAY); Client followerClient = wrapClient(client, params); BiConsumer scheduler = (delay, command) -> threadPool.schedule(delay, Ccr.CCR_THREAD_POOL_NAME, command); - return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers, params, scheduler) { + return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers, params, + scheduler, idleShardChangesRequestDelay) { @Override protected void updateMapping(LongConsumer handler) { @@ -128,11 +131,11 @@ protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operatio } @Override - protected void innerSendShardChangesRequest(long from, long size, Consumer handler, + protected void innerSendShardChangesRequest(long from, int maxOperationCount, Consumer handler, Consumer errorHandler) { ShardChangesAction.Request request = new ShardChangesAction.Request(params.getLeaderShardId()); request.setFromSeqNo(from); - request.setMaxOperationCount(size); + request.setMaxOperationCount(maxOperationCount); request.setMaxOperationSizeInBytes(params.getMaxOperationSizeInBytes()); leaderClient.execute(ShardChangesAction.INSTANCE, request, ActionListener.wrap(handler::accept, errorHandler)); } 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 22d3390671935..7683311f812e0 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 @@ -31,8 +31,7 @@ public String getName() { @Override protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { - Request request = new Request(); - request.setFollowRequest(RestFollowIndexAction.createRequest(restRequest)); + Request request = new Request(RestFollowIndexAction.createRequest(restRequest)); return channel -> client.execute(INSTANCE, request, new RestToXContentListener<>(channel)); } } 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 34b31c229fbf6..60a081e198f61 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 @@ -11,6 +11,7 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.ccr.action.ShardFollowNodeTask; import org.elasticsearch.xpack.ccr.action.ShardFollowTask; import java.io.IOException; @@ -37,25 +38,31 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient } static Request createRequest(RestRequest restRequest) { - Request request = new Request(); - request.setLeaderIndex(restRequest.param("leader_index")); - request.setFollowIndex(restRequest.param("index")); - if (restRequest.hasParam(ShardFollowTask.MAX_READ_SIZE.getPreferredName())) { - request.setMaxReadSize(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_READ_SIZE.getPreferredName()))); + int maxOperationCount = ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT; + if (restRequest.hasParam(ShardFollowTask.MAX_OPERATION_COUNT.getPreferredName())) { + maxOperationCount = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_OPERATION_COUNT.getPreferredName())); } + int maxConcurrentReads = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS; if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName())) { - request.setMaxConcurrentReads(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName()))); + maxConcurrentReads = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName())); } + long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; if (restRequest.hasParam(ShardFollowTask.MAX_OPERATION_SIZE_IN_BYTES.getPreferredName())) { - long value = Long.valueOf(restRequest.param(ShardFollowTask.MAX_OPERATION_SIZE_IN_BYTES.getPreferredName())); - request.setMaxOperationSizeInBytes(value); + maxOperationSizeInBytes = Long.valueOf(restRequest.param(ShardFollowTask.MAX_OPERATION_SIZE_IN_BYTES.getPreferredName())); } + int maxWriteSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE; if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())) { - request.setMaxWriteSize(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName()))); + maxWriteSize = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())); } + int maxConcurrentWrites = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES; if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_WRITES.getPreferredName())) { - request.setMaxConcurrentWrites(Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_WRITES.getPreferredName()))); + maxConcurrentWrites = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_WRITES.getPreferredName())); } - return request; + int maxBufferSize = ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE; + if (restRequest.hasParam(ShardFollowTask.MAX_BUFFER_SIZE.getPreferredName())) { + maxBufferSize = Integer.parseInt(restRequest.param(ShardFollowTask.MAX_BUFFER_SIZE.getPreferredName())); + } + return new Request(restRequest.param("leader_index"), restRequest.param("index"), maxOperationCount, maxConcurrentReads, + maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize); } } 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 c616d24b376b8..cd90d59ccc575 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 @@ -77,6 +77,7 @@ protected Settings nodeSettings(int nodeOrdinal) { newSettings.put(XPackSettings.WATCHER_ENABLED.getKey(), false); newSettings.put(XPackSettings.MACHINE_LEARNING_ENABLED.getKey(), false); newSettings.put(XPackSettings.LOGSTASH_ENABLED.getKey(), false); + newSettings.put(CcrSettings.CCR_IDLE_SHARD_CHANGES_TIMEOUT.getKey(), TimeValue.timeValueSeconds(1)); return newSettings.build(); } @@ -116,7 +117,7 @@ public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { ShardChangesAction.Request request = new ShardChangesAction.Request(shardStats.getShardRouting().shardId()); request.setFromSeqNo(0L); - request.setMaxOperationCount(3L); + request.setMaxOperationCount(3); ShardChangesAction.Response response = client().execute(ShardChangesAction.INSTANCE, request).get(); assertThat(response.getOperations().length, equalTo(3)); Translog.Index operation = (Translog.Index) response.getOperations()[0]; @@ -141,7 +142,7 @@ public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { request = new ShardChangesAction.Request(shardStats.getShardRouting().shardId()); request.setFromSeqNo(3L); - request.setMaxOperationCount(3L); + request.setMaxOperationCount(3); response = client().execute(ShardChangesAction.INSTANCE, request).get(); assertThat(response.getOperations().length, equalTo(3)); operation = (Translog.Index) response.getOperations()[0]; @@ -164,12 +165,8 @@ public void testFollowIndex() throws Exception { assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); ensureYellow("index1"); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); - followRequest.setLeaderIndex("index1"); - followRequest.setFollowIndex("index2"); - - final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(); - createAndFollowRequest.setFollowRequest(followRequest); + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2"); + final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); final int firstBatchNumDocs = randomIntBetween(2, 64); @@ -226,12 +223,8 @@ public void testSyncMappings() throws Exception { assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); ensureYellow("index1"); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); - followRequest.setLeaderIndex("index1"); - followRequest.setFollowIndex("index2"); - - final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(); - createAndFollowRequest.setFollowRequest(followRequest); + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2"); + final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); final long firstBatchNumDocs = randomIntBetween(2, 64); @@ -296,14 +289,9 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) long numDocsIndexed = Math.min(3000 * 2, randomLongBetween(maxReadSize, maxReadSize * 10)); atLeastDocsIndexed("index1", numDocsIndexed / 3); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); - followRequest.setLeaderIndex("index1"); - followRequest.setFollowIndex("index2"); - followRequest.setMaxReadSize(maxReadSize); - followRequest.setMaxConcurrentReads(randomIntBetween(2, 10)); - followRequest.setMaxConcurrentWrites(randomIntBetween(2, 10)); - CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(); - createAndFollowRequest.setFollowRequest(followRequest); + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", maxReadSize, + randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(32, 2048), randomIntBetween(2, 10), randomIntBetween(1024, 10240)); + CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); atLeastDocsIndexed("index1", numDocsIndexed); @@ -341,14 +329,12 @@ public void testFollowIndexAndCloseNode() throws Exception { }); thread.start(); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); - followRequest.setLeaderIndex("index1"); - followRequest.setFollowIndex("index2"); - followRequest.setMaxReadSize(randomIntBetween(32, 2048)); - followRequest.setMaxConcurrentReads(randomIntBetween(2, 10)); + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", randomIntBetween(32, 2048), + randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(32, 2048), randomIntBetween(2, 10), + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); - long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxReadSize(), followRequest.getMaxReadSize() * 10)); + long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxOperationCount(), followRequest.getMaxOperationCount() * 10)); long minNumDocsReplicated = maxNumDocsReplicated / 3L; logger.info("waiting for at least [{}] documents to be indexed and then stop a random data node", minNumDocsReplicated); awaitBusy(() -> { @@ -387,9 +373,7 @@ public void testFollowIndexWithNestedField() throws Exception { ensureGreen("index1", "index2"); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); - followRequest.setLeaderIndex("index1"); - followRequest.setFollowIndex("index2"); + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2"); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); final int numDocs = randomIntBetween(2, 64); @@ -431,19 +415,15 @@ public void testUnfollowNonExistingIndex() { public void testFollowNonExistentIndex() throws Exception { assertAcked(client().admin().indices().prepareCreate("test-leader").get()); assertAcked(client().admin().indices().prepareCreate("test-follower").get()); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); // Leader index does not exist. - followRequest.setLeaderIndex("non-existent-leader"); - followRequest.setFollowIndex("test-follower"); - expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest).actionGet()); + FollowIndexAction.Request followRequest1 = new FollowIndexAction.Request("non-existent-leader", "test-follower"); + expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest1).actionGet()); // Follower index does not exist. - followRequest.setLeaderIndex("test-leader"); - followRequest.setFollowIndex("non-existent-follower"); - expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest).actionGet()); + FollowIndexAction.Request followRequest2 = new FollowIndexAction.Request("non-test-leader", "non-existent-follower"); + expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest2).actionGet()); // Both indices do not exist. - followRequest.setLeaderIndex("non-existent-leader"); - followRequest.setFollowIndex("non-existent-follower"); - expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest).actionGet()); + FollowIndexAction.Request followRequest3 = new FollowIndexAction.Request("non-existent-leader", "non-existent-follower"); + expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest3).actionGet()); } public void testFollowIndex_lowMaxTranslogBytes() throws Exception { @@ -458,13 +438,9 @@ public void testFollowIndex_lowMaxTranslogBytes() throws Exception { client().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); } - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request(); - followRequest.setMaxOperationSizeInBytes(1024); - followRequest.setLeaderIndex("index1"); - followRequest.setFollowIndex("index2"); - - final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(); - createAndFollowRequest.setFollowRequest(followRequest); + final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", 1024, 1, 1024, + 1024, 1, 10240); + final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); final Map firstBatchNumDocsPerShard = new HashMap<>(); 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 18900bc852da5..c68d18499658c 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 @@ -16,8 +16,6 @@ protected CreateAndFollowIndexAction.Request createBlankInstance() { @Override protected CreateAndFollowIndexAction.Request createTestInstance() { - CreateAndFollowIndexAction.Request request = new CreateAndFollowIndexAction.Request(); - request.setFollowRequest(FollowIndexRequestTests.createTestRequest()); - return request; + return new CreateAndFollowIndexAction.Request(FollowIndexRequestTests.createTestRequest()); } } 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/FollowIndexActionTests.java index a27294ccf2df5..f0f26b921ac76 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/FollowIndexActionTests.java @@ -22,10 +22,7 @@ public class FollowIndexActionTests extends ESTestCase { public void testValidation() throws IOException { - FollowIndexAction.Request request = new FollowIndexAction.Request(); - request.setLeaderIndex("index1"); - request.setFollowIndex("index2"); - + FollowIndexAction.Request request = new FollowIndexAction.Request("index1", "index2"); { // should fail, because leader index does not exist Exception e = expectThrows(IllegalArgumentException.class, () -> FollowIndexAction.validate(request, null, null, null)); @@ -130,7 +127,7 @@ public void testValidation() throws IOException { FollowIndexAction.validate(request, leaderIMD, followIMD, mapperService); } } - + private static IndexMetaData createIMD(String index, int numShards, Tuple... settings) throws IOException { return createIMD(index, State.OPEN, "{\"properties\": {}}", numShards, settings); } 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 9bd28958e8313..5576357456e8c 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 @@ -20,16 +20,8 @@ protected FollowIndexAction.Request createTestInstance() { } static FollowIndexAction.Request createTestRequest() { - FollowIndexAction.Request request = new FollowIndexAction.Request(); - request.setLeaderIndex(randomAlphaOfLength(4)); - request.setFollowIndex(randomAlphaOfLength(4)); - request.setMaxReadSize(randomIntBetween(1, Integer.MAX_VALUE)); - request.setMaxConcurrentReads(randomIntBetween(1, Integer.MAX_VALUE)); - request.setMaxOperationSizeInBytes(randomNonNegativeLong()); - request.setMaxWriteSize(randomIntBetween(1, Integer.MAX_VALUE)); - request.setMaxConcurrentWrites(randomIntBetween(1, Integer.MAX_VALUE)); - request.setMaxConcurrentWrites(randomIntBetween(1, Integer.MAX_VALUE)); - request.setMaxBufferSize(randomIntBetween(1, Integer.MAX_VALUE)); - return request; + return new FollowIndexAction.Request(randomAlphaOfLength(4), randomAlphaOfLength(4), randomIntBetween(1, Integer.MAX_VALUE), + randomIntBetween(1, Integer.MAX_VALUE), randomNonNegativeLong(), randomIntBetween(1, Integer.MAX_VALUE), + randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE)); } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index 037d464ddef0b..baa4c5ee77aed 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -45,7 +45,8 @@ public void testGetOperationsBetween() throws Exception { int min = randomIntBetween(0, numWrites - 1); int max = randomIntBetween(min, numWrites - 1); int size = max - min; - final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, min, size, Long.MAX_VALUE); + final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, + indexShard.getGlobalCheckpoint(), min, size, Long.MAX_VALUE); final List seenSeqNos = Arrays.stream(operations).map(Translog.Operation::seqNo).collect(Collectors.toList()); final List expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toList()); assertThat(seenSeqNos, equalTo(expectedSeqNos)); @@ -53,11 +54,13 @@ public void testGetOperationsBetween() throws Exception { // get operations for a range no operations exists: - Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, numWrites, numWrites + 1, Long.MAX_VALUE); + Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, indexShard.getGlobalCheckpoint(), + numWrites, numWrites + 1, Long.MAX_VALUE); assertThat(operations.length, equalTo(0)); // get operations for a range some operations do not exist: - operations = ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, numWrites + 10, Long.MAX_VALUE); + operations = ShardChangesAction.getOperationsBetween(indexShard, indexShard.getGlobalCheckpoint(), + numWrites - 10, numWrites + 10, Long.MAX_VALUE); assertThat(operations.length, equalTo(10)); } @@ -66,7 +69,8 @@ public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { ShardRouting shardRouting = TestShardRouting.newShardRouting("index", 0, "_node_id", true, ShardRoutingState.INITIALIZING); Mockito.when(indexShard.routingEntry()).thenReturn(shardRouting); - expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, 0, 1, Long.MAX_VALUE)); + expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, + indexShard.getGlobalCheckpoint(), 0, 1, Long.MAX_VALUE)); } public void testGetOperationsBetweenExceedByteLimit() throws Exception { @@ -82,7 +86,8 @@ public void testGetOperationsBetweenExceedByteLimit() throws Exception { } final IndexShard indexShard = indexService.getShard(0); - final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, 0, 12, 256); + final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, indexShard.getGlobalCheckpoint(), + 0, 12, 256); assertThat(operations.length, equalTo(12)); assertThat(operations[0].seqNo(), equalTo(0L)); assertThat(operations[1].seqNo(), equalTo(1L)); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java index 94eddc0e85446..19585da8851d6 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java @@ -16,7 +16,7 @@ public class ShardChangesRequestTests extends AbstractStreamableTestCase handler, + protected void innerSendShardChangesRequest(long from, int maxOperationCount, Consumer handler, Consumer errorHandler) { if (randomlyFailWithRetryableError.get() && writeCounter.incrementAndGet() % 5 == 0) { failedRequests.incrementAndGet(); @@ -188,12 +189,12 @@ protected void innerSendShardChangesRequest(long from, long size, Consumer leaderGlobalCheckpoint) { response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, new Translog.Operation[0]); } else { - if (randomlyTruncateRequests.get() && size > 10 && truncatedRequests.get() < 5) { + if (randomlyTruncateRequests.get() && maxOperationCount > 10 && truncatedRequests.get() < 5) { truncatedRequests.incrementAndGet(); - size = size / 2; + maxOperationCount = maxOperationCount / 2; } List ops = new ArrayList<>(); - for (int i = 0; i < size; i++) { + for (int i = 0; i < maxOperationCount; i++) { long seqNo = from + i; if (seqNo <= leaderGlobalCheckpoint) { String id = UUIDs.randomBase64UUID(); From 9b8de5fe774f3610f30e9a482e05abce052f33a8 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 2 Jul 2018 16:57:12 +0200 Subject: [PATCH 17/41] fixed compile error and checkstyle violation --- .../java/org/elasticsearch/xpack/ccr/ShardChangesIT.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 cd90d59ccc575..f735ea2d1bf6e 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 @@ -77,7 +77,7 @@ protected Settings nodeSettings(int nodeOrdinal) { newSettings.put(XPackSettings.WATCHER_ENABLED.getKey(), false); newSettings.put(XPackSettings.MACHINE_LEARNING_ENABLED.getKey(), false); newSettings.put(XPackSettings.LOGSTASH_ENABLED.getKey(), false); - newSettings.put(CcrSettings.CCR_IDLE_SHARD_CHANGES_TIMEOUT.getKey(), TimeValue.timeValueSeconds(1)); + newSettings.put(CcrSettings.CCR_IDLE_SHARD_CHANGES_DELAY.getKey(), TimeValue.timeValueSeconds(1)); return newSettings.build(); } @@ -334,7 +334,8 @@ public void testFollowIndexAndCloseNode() throws Exception { ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); - long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxOperationCount(), followRequest.getMaxOperationCount() * 10)); + long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxOperationCount(), + followRequest.getMaxOperationCount() * 10)); long minNumDocsReplicated = maxNumDocsReplicated / 3L; logger.info("waiting for at least [{}] documents to be indexed and then stop a random data node", minNumDocsReplicated); awaitBusy(() -> { From 60fb9a8d4aeb5cbad33e6b62364f4486d97df7f8 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 3 Jul 2018 07:31:01 +0200 Subject: [PATCH 18/41] increase timeouts --- .../test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 5a52970e7f21d..05ba629c9d9d9 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; @@ -64,7 +65,7 @@ public void testFollowIndex() throws Exception { logger.info("Running against follow cluster"); final String followIndexName = "test_index2"; createAndFollowIndex("leader_cluster:" + leaderIndexName, followIndexName); - assertBusy(() -> verifyDocuments(followIndexName, numDocs)); + assertBusy(() -> verifyDocuments(followIndexName, numDocs), 20, TimeUnit.SECONDS); // unfollow and then follow and then index a few docs in leader index: unfollowIndex(followIndexName); followIndex("leader_cluster:" + leaderIndexName, followIndexName); @@ -74,7 +75,7 @@ public void testFollowIndex() throws Exception { index(leaderClient, leaderIndexName, Integer.toString(id + 1), "field", id + 1, "filtered_field", "true"); index(leaderClient, leaderIndexName, Integer.toString(id + 2), "field", id + 2, "filtered_field", "true"); } - assertBusy(() -> verifyDocuments(followIndexName, numDocs + 3)); + assertBusy(() -> verifyDocuments(followIndexName, numDocs + 3), 20, TimeUnit.SECONDS); } } From 2ec6ac45aebbb42c9e7c90d13470adc95105969c Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 10:45:12 +0200 Subject: [PATCH 19/41] rename + jdocs --- .../xpack/ccr/action/ShardChangesAction.java | 13 ++++++++++--- .../xpack/ccr/action/ShardChangesActionTests.java | 10 +++++----- 2 files changed, 15 insertions(+), 8 deletions(-) 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 7723539cf912d..33c055bf1abe2 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 @@ -234,7 +234,7 @@ protected Response shardOperation(Request request, ShardId shardId) throws IOExc long globalCheckpoint = indexShard.getGlobalCheckpoint(); final long indexMetaDataVersion = clusterService.state().metaData().index(shardId.getIndex()).getVersion(); - final Translog.Operation[] operations = getOperationsBetween(indexShard, globalCheckpoint, request.fromSeqNo, + final Translog.Operation[] operations = getOperations(indexShard, globalCheckpoint, request.fromSeqNo, request.maxOperationCount, request.maxOperationSizeInBytes); return new Response(indexMetaDataVersion, globalCheckpoint, operations); } @@ -261,8 +261,15 @@ protected Response newResponse() { private static final Translog.Operation[] EMPTY_OPERATIONS_ARRAY = new Translog.Operation[0]; - static Translog.Operation[] getOperationsBetween(IndexShard indexShard, long globalCheckpoint, long fromSeqNo, int maxOperationCount, - long maxOperationSizeInBytes) throws IOException { + /** + * Returns at most maxOperationCount operations from the specified from sequence number. + * This method will never return operations above the specified globalCheckpoint. + * + * Also if the sum of collected operations' size is above the specified maxOperationSizeInBytes then this method + * stops collecting more operations and returns what has been collected so far. + */ + static Translog.Operation[] getOperations(IndexShard indexShard, long globalCheckpoint, long fromSeqNo, int maxOperationCount, + long maxOperationSizeInBytes) throws IOException { if (indexShard.state() != IndexShardState.STARTED) { throw new IndexShardNotStartedException(indexShard.shardId(), indexShard.state()); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index baa4c5ee77aed..8d7c440ab1d38 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -45,7 +45,7 @@ public void testGetOperationsBetween() throws Exception { int min = randomIntBetween(0, numWrites - 1); int max = randomIntBetween(min, numWrites - 1); int size = max - min; - final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, + final Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(), min, size, Long.MAX_VALUE); final List seenSeqNos = Arrays.stream(operations).map(Translog.Operation::seqNo).collect(Collectors.toList()); final List expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toList()); @@ -54,12 +54,12 @@ public void testGetOperationsBetween() throws Exception { // get operations for a range no operations exists: - Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, indexShard.getGlobalCheckpoint(), + Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(), numWrites, numWrites + 1, Long.MAX_VALUE); assertThat(operations.length, equalTo(0)); // get operations for a range some operations do not exist: - operations = ShardChangesAction.getOperationsBetween(indexShard, indexShard.getGlobalCheckpoint(), + operations = ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(), numWrites - 10, numWrites + 10, Long.MAX_VALUE); assertThat(operations.length, equalTo(10)); } @@ -69,7 +69,7 @@ public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { ShardRouting shardRouting = TestShardRouting.newShardRouting("index", 0, "_node_id", true, ShardRoutingState.INITIALIZING); Mockito.when(indexShard.routingEntry()).thenReturn(shardRouting); - expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, + expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(), 0, 1, Long.MAX_VALUE)); } @@ -86,7 +86,7 @@ public void testGetOperationsBetweenExceedByteLimit() throws Exception { } final IndexShard indexShard = indexService.getShard(0); - final Translog.Operation[] operations = ShardChangesAction.getOperationsBetween(indexShard, indexShard.getGlobalCheckpoint(), + final Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(), 0, 12, 256); assertThat(operations.length, equalTo(12)); assertThat(operations[0].seqNo(), equalTo(0L)); From fa1b52feb6902f6bf489869ca72db26e35432d58 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 11:20:38 +0200 Subject: [PATCH 20/41] iter --- .../elasticsearch/xpack/ccr/CcrSettings.java | 8 +++---- .../xpack/ccr/action/ShardChangesAction.java | 18 +++++++-------- .../xpack/ccr/action/ShardFollowNodeTask.java | 22 +++++++++++-------- .../ccr/action/ShardFollowTasksExecutor.java | 2 +- .../xpack/ccr/ShardChangesIT.java | 2 +- 5 files changed, 28 insertions(+), 24 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java index da1a302c2e770..d5ca346875927 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java @@ -34,11 +34,11 @@ private CcrSettings() { Setting.boolSetting("index.xpack.ccr.following_index", false, Setting.Property.IndexScope); /** - * Index setting for controlling the timeout between the next shard changes request when a shard follow + * Node setting for controlling the timeout between the next shard changes request when a shard follow * task is idle. */ - public static final Setting CCR_IDLE_SHARD_CHANGES_DELAY = Setting.timeSetting( - "index.xpack.ccr.idle_shard_changes_delay", TimeValue.timeValueSeconds(10), Property.NodeScope); + public static final Setting CCR_IDLE_SHARD_RETRY_DELAY = Setting.timeSetting( + "xpack.ccr.idle_shard_retry_delay", TimeValue.timeValueSeconds(10), Property.NodeScope); /** * The settings defined by CCR. @@ -49,7 +49,7 @@ static List> getSettings() { return Arrays.asList( CCR_ENABLED_SETTING, CCR_FOLLOWING_INDEX_SETTING, - CCR_IDLE_SHARD_CHANGES_DELAY); + CCR_IDLE_SHARD_RETRY_DELAY); } } 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 33c055bf1abe2..03f783abcca62 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 @@ -150,15 +150,15 @@ public int hashCode() { public static final class Response extends ActionResponse { private long indexMetadataVersion; - private long leaderCheckpoint; + private long globalCheckpoint; private Translog.Operation[] operations; Response() { } - Response(long indexMetadataVersion, long leaderCheckpoint, final Translog.Operation[] operations) { + Response(long indexMetadataVersion, long globalCheckpoint, final Translog.Operation[] operations) { this.indexMetadataVersion = indexMetadataVersion; - this.leaderCheckpoint = leaderCheckpoint; + this.globalCheckpoint = globalCheckpoint; this.operations = operations; } @@ -166,8 +166,8 @@ public long getIndexMetadataVersion() { return indexMetadataVersion; } - public long getLeaderCheckpoint() { - return leaderCheckpoint; + public long getGlobalCheckpoint() { + return globalCheckpoint; } public Translog.Operation[] getOperations() { @@ -178,7 +178,7 @@ public Translog.Operation[] getOperations() { public void readFrom(final StreamInput in) throws IOException { super.readFrom(in); indexMetadataVersion = in.readVLong(); - leaderCheckpoint = in.readZLong(); + globalCheckpoint = in.readZLong(); operations = in.readArray(Translog.Operation::readOperation, Translog.Operation[]::new); } @@ -186,7 +186,7 @@ public void readFrom(final StreamInput in) throws IOException { public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); out.writeVLong(indexMetadataVersion); - out.writeZLong(leaderCheckpoint); + out.writeZLong(globalCheckpoint); out.writeArray(Translog.Operation::writeOperation, operations); } @@ -196,7 +196,7 @@ public boolean equals(final Object o) { if (o == null || getClass() != o.getClass()) return false; final Response response = (Response) o; return indexMetadataVersion == response.indexMetadataVersion && - leaderCheckpoint == response.leaderCheckpoint && + globalCheckpoint == response.globalCheckpoint && Arrays.equals(operations, response.operations); } @@ -204,7 +204,7 @@ public boolean equals(final Object o) { public int hashCode() { int result = 1; result += Objects.hashCode(indexMetadataVersion); - result += Objects.hashCode(leaderCheckpoint); + result += Objects.hashCode(globalCheckpoint); result += Arrays.hashCode(operations); return result; } 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 edbd822ec04b4..fb3b02acc6f58 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 @@ -29,7 +29,6 @@ import java.util.Comparator; import java.util.Map; import java.util.Objects; -import java.util.OptionalLong; import java.util.PriorityQueue; import java.util.Queue; import java.util.concurrent.atomic.AtomicInteger; @@ -170,16 +169,21 @@ private void sendShardChangesRequest(long from, int maxOperationCount, long maxR private void handleReadResponse(long from, int maxOperationCount, long maxRequiredSeqNo, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { - globalCheckpoint = Math.max(globalCheckpoint, response.getLeaderCheckpoint()); + globalCheckpoint = Math.max(globalCheckpoint, response.getGlobalCheckpoint()); buffer.addAll(Arrays.asList(response.getOperations())); coordinateWrites(); - OptionalLong lastOp = Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::seqNo).max(); - if (lastOp.isPresent() && lastOp.getAsLong() < maxRequiredSeqNo) { - long newFrom = lastOp.getAsLong() + 1; - int newSize = (int) (maxRequiredSeqNo - lastOp.getAsLong()); + Long lastOpSeqNo = null; + if (response.getOperations().length != 0) { + lastOpSeqNo = response.getOperations()[response.getOperations().length - 1].seqNo(); + assert lastOpSeqNo == Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::seqNo).max().getAsLong(); + } + + if (lastOpSeqNo != null && lastOpSeqNo < maxRequiredSeqNo) { + long newFrom = lastOpSeqNo + 1; + int newSize = (int) (maxRequiredSeqNo - lastOpSeqNo); LOGGER.trace("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", - params.getFollowShardId(), lastOp.getAsLong(), maxRequiredSeqNo, newFrom, maxOperationCount); + params.getFollowShardId(), lastOpSeqNo, maxRequiredSeqNo, newFrom, maxOperationCount); sendShardChangesRequest(newFrom, newSize, maxRequiredSeqNo); return; } @@ -188,7 +192,7 @@ private void handleReadResponse(long from, int maxOperationCount, long maxRequir if (response.getOperations().length != 0) { LOGGER.trace("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); Translog.Operation firstOp = response.getOperations()[0]; - lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOp.getAsLong()); + lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOpSeqNo); assert firstOp.seqNo() == from; coordinateReads(); } else { @@ -223,7 +227,7 @@ private synchronized void maybeUpdateMapping(Long minimumRequiredIndexMetadataVe params.getFollowShardId(), currentIndexMetadataVersion, minimumRequiredIndexMetadataVersion); task.run(); } else { - LOGGER.debug("{} updating mapping, index metadata version [{}] is lower than minimum required index metadata version [{}]", + LOGGER.trace("{} updating mapping, index metadata version [{}] is lower than minimum required index metadata version [{}]", params.getFollowShardId(), currentIndexMetadataVersion, minimumRequiredIndexMetadataVersion); updateMapping(imdVersion -> { retryCounter.set(0); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 826921d6093e6..98d9e277f75ec 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -90,7 +90,7 @@ protected AllocatedPersistentTask createTask(long id, String type, String action leaderClient = wrapClient(client, params); } TimeValue idleShardChangesRequestDelay = - settings.getAsTime(CcrSettings.CCR_IDLE_SHARD_CHANGES_DELAY.getKey(), DEFAULT_IDLE_SHARD_CHANGES_DELAY); + settings.getAsTime(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), DEFAULT_IDLE_SHARD_CHANGES_DELAY); Client followerClient = wrapClient(client, params); BiConsumer scheduler = (delay, command) -> threadPool.schedule(delay, Ccr.CCR_THREAD_POOL_NAME, command); 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 f735ea2d1bf6e..688119f6e7ae8 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 @@ -77,7 +77,7 @@ protected Settings nodeSettings(int nodeOrdinal) { newSettings.put(XPackSettings.WATCHER_ENABLED.getKey(), false); newSettings.put(XPackSettings.MACHINE_LEARNING_ENABLED.getKey(), false); newSettings.put(XPackSettings.LOGSTASH_ENABLED.getKey(), false); - newSettings.put(CcrSettings.CCR_IDLE_SHARD_CHANGES_DELAY.getKey(), TimeValue.timeValueSeconds(1)); + newSettings.put(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), TimeValue.timeValueSeconds(1)); return newSettings.build(); } From 932eeaa2a9bb0e1a30f871b486e0bf28a1c2ec99 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 11:31:44 +0200 Subject: [PATCH 21/41] keep strict validation --- .../index/engine/LuceneChangesSnapshot.java | 7 ++++++- .../engine/LuceneChangesSnapshotTests.java | 19 ++++++++++++------- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index 367b0f822a870..0a876468914f8 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -139,7 +139,12 @@ public Translog.Operation next() throws IOException { } private void rangeCheck(Translog.Operation op) { - if (op != null) { + if (op == null) { + if (lastSeenSeqNo < toSeqNo) { + throw new IllegalStateException("Not all operations between from_seqno [" + fromSeqNo + "] " + + "and to_seqno [" + toSeqNo + "] found; prematurely terminated last_seen_seqno [" + lastSeenSeqNo + "]"); + } + } else { final long expectedSeqNo = lastSeenSeqNo + 1; if (op.seqNo() != expectedSeqNo) { throw new IllegalStateException("Not all operations between from_seqno [" + fromSeqNo + "] " + diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index c7c43ead9d703..7de086a3be239 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -40,6 +40,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; public class LuceneChangesSnapshotTests extends EngineTestCase { @@ -62,8 +63,9 @@ public void testBasics() throws Exception { long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); // Empty engine try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, true)) { - List ops = drainAll(snapshot); - assertThat(ops.size(), equalTo(0)); + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat(error.getMessage(), + containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found")); } try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, false)) { assertThat(snapshot, SnapshotMatchers.size(0)); @@ -104,9 +106,10 @@ searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), f try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { searcher = null; - List ops = drainAll(snapshot); - assertThat(ops.size(), equalTo(0)); - } finally { + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat(error.getMessage(), + containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found")); + }finally { IOUtils.close(searcher); } } else { @@ -124,8 +127,10 @@ searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), f try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { searcher = null; - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); - } finally { + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat(error.getMessage(), + containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found")); + }finally { IOUtils.close(searcher); } toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); From aa11dda0faf27e3d689fb1ff969710d478bde344 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 12:01:54 +0200 Subject: [PATCH 22/41] iter2 --- .../xpack/ccr/action/ShardFollowNodeTask.java | 3 +- .../xpack/ccr/action/ShardFollowTask.java | 78 ++++++------------- .../ccr/action/ShardFollowTasksExecutor.java | 6 +- .../xpack/ccr/rest/RestFollowIndexAction.java | 20 ++--- 4 files changed, 38 insertions(+), 69 deletions(-) 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 fb3b02acc6f58..a04c50a90e013 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 @@ -48,7 +48,6 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { public static final int DEFAULT_MAX_CONCURRENT_WRITES = 1; public static final int DEFAULT_MAX_BUFFER_SIZE = 10240; public static final long DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES = Long.MAX_VALUE; - public static final TimeValue DEFAULT_IDLE_SHARD_CHANGES_DELAY = TimeValue.timeValueSeconds(10); private static final int RETRY_LIMIT = 10; private static final TimeValue RETRY_TIMEOUT = TimeValue.timeValueMillis(500); @@ -110,7 +109,7 @@ private synchronized void coordinateReads() { } if (numConcurrentReads == 0) { - // We sneak peek if there is any thing new in the leader primary. + // We sneak peek if there is any thing new in the leader. // If there is we will happily accept numConcurrentReads++; long from = lastRequestedSeqno + 1; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index 96da24779d201..7701f1607238f 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -40,12 +40,12 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { static final ParseField LEADER_SHARD_INDEX_UUID_FIELD = new ParseField("leader_shard_index_uuid"); static final ParseField LEADER_SHARD_SHARDID_FIELD = new ParseField("leader_shard_shard"); static final ParseField HEADERS = new ParseField("headers"); - public static final ParseField MAX_OPERATION_COUNT = new ParseField("max_operation_count"); - public static final ParseField MAX_CONCURRENT_READS = new ParseField("max_concurrent_reads"); - public static final ParseField MAX_OPERATION_SIZE_IN_BYTES = new ParseField("max_operation_size_in_bytes"); + public static final ParseField MAX_BATCH_OPERATION_COUNT = new ParseField("max_batch_operation_count"); + public static final ParseField MAX_CONCURRENT_READS_BATCHES = new ParseField("max_concurrent_read_batches"); + public static final ParseField MAX_BATCH_SIZE_IN_BYTES = new ParseField("max_batch_size_in_bytes"); public static final ParseField MAX_WRITE_SIZE = new ParseField("max_write_size"); - public static final ParseField MAX_CONCURRENT_WRITES = new ParseField("max_concurrent_writes"); - public static final ParseField MAX_BUFFER_SIZE = new ParseField("max_buffer_size"); + public static final ParseField MAX_CONCURRENT_WRITES_BATCHES = new ParseField("max_concurrent_write_batches"); + public static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); @SuppressWarnings("unchecked") private static ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, @@ -61,12 +61,12 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_SHARD_INDEX_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_SHARD_INDEX_UUID_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), LEADER_SHARD_SHARDID_FIELD); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_OPERATION_COUNT); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READS); - PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_OPERATION_SIZE_IN_BYTES); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_BATCH_OPERATION_COUNT); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READS_BATCHES); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_BATCH_SIZE_IN_BYTES); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_SIZE); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITES); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_BUFFER_SIZE); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITES_BATCHES); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_BUFFER_SIZE); PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> p.mapStrings(), HEADERS); } @@ -175,50 +175,22 @@ public static ShardFollowTask fromXContent(XContentParser parser) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - { - if (leaderClusterAlias != null) { - builder.field(LEADER_CLUSTER_ALIAS_FIELD.getPreferredName(), leaderClusterAlias); - } - } - { - builder.field(FOLLOW_SHARD_INDEX_FIELD.getPreferredName(), followShardId.getIndex().getName()); - } - { - builder.field(FOLLOW_SHARD_INDEX_UUID_FIELD.getPreferredName(), followShardId.getIndex().getUUID()); - } - { - builder.field(FOLLOW_SHARD_SHARDID_FIELD.getPreferredName(), followShardId.id()); - } - { - builder.field(LEADER_SHARD_INDEX_FIELD.getPreferredName(), leaderShardId.getIndex().getName()); - } - { - builder.field(LEADER_SHARD_INDEX_UUID_FIELD.getPreferredName(), leaderShardId.getIndex().getUUID()); - } - { - builder.field(LEADER_SHARD_SHARDID_FIELD.getPreferredName(), leaderShardId.id()); - } - { - builder.field(MAX_OPERATION_COUNT.getPreferredName(), maxReadSize); - } - { - builder.field(MAX_CONCURRENT_READS.getPreferredName(), maxConcurrentReads); - } - { - builder.field(MAX_OPERATION_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); - } - { - builder.field(MAX_WRITE_SIZE.getPreferredName(), maxWriteSize); - } - { - builder.field(MAX_CONCURRENT_WRITES.getPreferredName(), maxConcurrentWrites); - } - { - builder.field(MAX_BUFFER_SIZE.getPreferredName(), maxBufferSize); - } - { - builder.field(HEADERS.getPreferredName(), headers); + if (leaderClusterAlias != null) { + builder.field(LEADER_CLUSTER_ALIAS_FIELD.getPreferredName(), leaderClusterAlias); } + builder.field(FOLLOW_SHARD_INDEX_FIELD.getPreferredName(), followShardId.getIndex().getName()); + builder.field(FOLLOW_SHARD_INDEX_UUID_FIELD.getPreferredName(), followShardId.getIndex().getUUID()); + builder.field(FOLLOW_SHARD_SHARDID_FIELD.getPreferredName(), followShardId.id()); + builder.field(LEADER_SHARD_INDEX_FIELD.getPreferredName(), leaderShardId.getIndex().getName()); + builder.field(LEADER_SHARD_INDEX_UUID_FIELD.getPreferredName(), leaderShardId.getIndex().getUUID()); + builder.field(LEADER_SHARD_SHARDID_FIELD.getPreferredName(), leaderShardId.id()); + builder.field(MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxReadSize); + builder.field(MAX_CONCURRENT_READS_BATCHES.getPreferredName(), maxConcurrentReads); + builder.field(MAX_BATCH_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); + builder.field(MAX_WRITE_SIZE.getPreferredName(), maxWriteSize); + builder.field(MAX_CONCURRENT_WRITES_BATCHES.getPreferredName(), maxConcurrentWrites); + builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxBufferSize); + builder.field(HEADERS.getPreferredName(), headers); return builder.endObject(); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 98d9e277f75ec..444b9d869e1b9 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -48,8 +48,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -import static org.elasticsearch.xpack.ccr.action.ShardFollowNodeTask.DEFAULT_IDLE_SHARD_CHANGES_DELAY; - public class ShardFollowTasksExecutor extends PersistentTasksExecutor { private final Client client; @@ -89,8 +87,8 @@ protected AllocatedPersistentTask createTask(long id, String type, String action } else { leaderClient = wrapClient(client, params); } - TimeValue idleShardChangesRequestDelay = - settings.getAsTime(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), DEFAULT_IDLE_SHARD_CHANGES_DELAY); + TimeValue idleShardChangesRequestDelay = settings.getAsTime(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), + CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getDefault(settings)); Client followerClient = wrapClient(client, params); BiConsumer scheduler = (delay, command) -> threadPool.schedule(delay, Ccr.CCR_THREAD_POOL_NAME, command); 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 60a081e198f61..c5ea53417ddba 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 @@ -39,28 +39,28 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient static Request createRequest(RestRequest restRequest) { int maxOperationCount = ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT; - if (restRequest.hasParam(ShardFollowTask.MAX_OPERATION_COUNT.getPreferredName())) { - maxOperationCount = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_OPERATION_COUNT.getPreferredName())); + if (restRequest.hasParam(ShardFollowTask.MAX_BATCH_OPERATION_COUNT.getPreferredName())) { + maxOperationCount = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_BATCH_OPERATION_COUNT.getPreferredName())); } int maxConcurrentReads = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS; - if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName())) { - maxConcurrentReads = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READS.getPreferredName())); + if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_READS_BATCHES.getPreferredName())) { + maxConcurrentReads = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READS_BATCHES.getPreferredName())); } long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; - if (restRequest.hasParam(ShardFollowTask.MAX_OPERATION_SIZE_IN_BYTES.getPreferredName())) { - maxOperationSizeInBytes = Long.valueOf(restRequest.param(ShardFollowTask.MAX_OPERATION_SIZE_IN_BYTES.getPreferredName())); + if (restRequest.hasParam(ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES.getPreferredName())) { + maxOperationSizeInBytes = Long.valueOf(restRequest.param(ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES.getPreferredName())); } int maxWriteSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE; if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())) { maxWriteSize = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())); } int maxConcurrentWrites = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES; - if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_WRITES.getPreferredName())) { - maxConcurrentWrites = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_WRITES.getPreferredName())); + if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_WRITES_BATCHES.getPreferredName())) { + maxConcurrentWrites = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_WRITES_BATCHES.getPreferredName())); } int maxBufferSize = ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE; - if (restRequest.hasParam(ShardFollowTask.MAX_BUFFER_SIZE.getPreferredName())) { - maxBufferSize = Integer.parseInt(restRequest.param(ShardFollowTask.MAX_BUFFER_SIZE.getPreferredName())); + if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName())) { + maxBufferSize = Integer.parseInt(restRequest.param(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName())); } return new Request(restRequest.param("leader_index"), restRequest.param("index"), maxOperationCount, maxConcurrentReads, maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize); From b100b976c47f750c9a876960f60729e6e1a8fd48 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 12:28:12 +0200 Subject: [PATCH 23/41] implemented custom WritePrimaryResult to include global checkpoint in response after it replication has been completed --- .../replication/TransportWriteAction.java | 2 ++ .../ccr/action/ShardFollowTasksExecutor.java | 2 +- .../bulk/BulkShardOperationsResponse.java | 16 +++++++------- .../TransportBulkShardOperationsAction.java | 22 +++++++++++++++++-- 4 files changed, 31 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index ca91a32a17a3a..577426637eceb 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -130,6 +130,7 @@ public static class WritePrimaryResult listener = null; public WritePrimaryResult(ReplicaRequest request, @Nullable Response finalResponse, @@ -137,6 +138,7 @@ public WritePrimaryResult(ReplicaRequest request, @Nullable Response finalRespon IndexShard primary, Logger logger) { super(request, finalResponse, operationFailure); this.location = location; + this.primary = primary; assert location == null || operationFailure == null : "expected either failure to be null or translog location to be null, " + "but found: [" + location + "] translog location and [" + operationFailure + "] failure"; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 444b9d869e1b9..6b69a4f4fac55 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -125,7 +125,7 @@ protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operatio Consumer errorHandler) { final BulkShardOperationsRequest request = new BulkShardOperationsRequest(params.getFollowShardId(), operations); followerClient.execute(BulkShardOperationsAction.INSTANCE, request, - ActionListener.wrap(response -> handler.accept(response.getLocalCheckpoint()), errorHandler)); + ActionListener.wrap(response -> handler.accept(response.getGlobalCheckpoint()), errorHandler)); } @Override diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java index c4d26d96905b1..39cc85ffe71ed 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsResponse.java @@ -14,17 +14,17 @@ public final class BulkShardOperationsResponse extends ReplicationResponse implements WriteResponse { - private long localCheckpoint; + private long globalCheckpoint; - BulkShardOperationsResponse(long localCheckPoint) { - this.localCheckpoint = localCheckPoint; + BulkShardOperationsResponse() { } - BulkShardOperationsResponse() { + public long getGlobalCheckpoint() { + return globalCheckpoint; } - public long getLocalCheckpoint() { - return localCheckpoint; + public void setGlobalCheckpoint(long globalCheckpoint) { + this.globalCheckpoint = globalCheckpoint; } @Override @@ -34,12 +34,12 @@ public void setForcedRefresh(final boolean forcedRefresh) { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - localCheckpoint = in.readZLong(); + globalCheckpoint = in.readZLong(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeZLong(localCheckpoint); + out.writeZLong(globalCheckpoint); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index 9048d121a0e96..dd0dde188239a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -103,8 +103,7 @@ static WritePrimaryResult(replicaRequest, new BulkShardOperationsResponse(localCheckPoint), location, null, primary, logger); + return new CcrWritePrimaryResult(replicaRequest, location, primary, logger); } @Override @@ -132,4 +131,23 @@ protected BulkShardOperationsResponse newResponseInstance() { return new BulkShardOperationsResponse(); } + /** + * Custom write result to include global checkpoint after ops have been replicated. + */ + static class CcrWritePrimaryResult extends WritePrimaryResult { + + CcrWritePrimaryResult(BulkShardOperationsRequest request, Translog.Location location, IndexShard primary, Logger logger) { + super(request, new BulkShardOperationsResponse(), location, null, primary, logger); + } + + @Override + protected void respondIfPossible(Exception ex) { + assert Thread.holdsLock(this); + // maybe invoked multiple times, but that is ok as global checkpoint does not go backwards + finalResponseIfSuccessful.setGlobalCheckpoint(primary.getGlobalCheckpoint()); + super.respondIfPossible(ex); + } + + } + } From 9048913378a2f7e2afbea0121d2b3441f5f7f365 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 14:23:38 +0200 Subject: [PATCH 24/41] made retry timeout configurable --- .../xpack/ccr/action/FollowIndexAction.java | 18 +++++++++++---- .../xpack/ccr/action/ShardFollowNodeTask.java | 9 +++++--- .../xpack/ccr/action/ShardFollowTask.java | 22 ++++++++++++++++--- .../ccr/action/ShardFollowTasksExecutor.java | 2 +- .../xpack/ccr/rest/RestFollowIndexAction.java | 5 ++++- .../xpack/ccr/ShardChangesIT.java | 7 +++--- .../ccr/action/FollowIndexRequestTests.java | 4 +++- .../ccr/action/ShardFollowNodeTaskTests.java | 4 ++-- .../ccr/action/ShardFollowTaskTests.java | 2 ++ 9 files changed, 55 insertions(+), 18 deletions(-) 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 index b375866ac4ebe..daf67303e7321 100644 --- 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 @@ -24,6 +24,7 @@ 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.index.IndexSettings; import org.elasticsearch.index.IndexingSlowLog; import org.elasticsearch.index.SearchSlowLog; @@ -77,9 +78,10 @@ public static class Request extends ActionRequest { private int maxWriteSize; private int maxConcurrentWrites; private int maxBufferSize; + private TimeValue retryTimeout; public Request(String leaderIndex, String followIndex, int maxOperationCount, int maxConcurrentReads, long maxOperationSizeInBytes, - int maxWriteSize, int maxConcurrentWrites, int maxBufferSize) { + int maxWriteSize, int maxConcurrentWrites, int maxBufferSize, TimeValue retryTimeout) { if (maxOperationCount < 1) { throw new IllegalArgumentException("maxOperationCount must be larger than 0"); } @@ -107,13 +109,14 @@ public Request(String leaderIndex, String followIndex, int maxOperationCount, in this.maxWriteSize = maxWriteSize; this.maxConcurrentWrites = maxConcurrentWrites; this.maxBufferSize = maxBufferSize; + this.retryTimeout = retryTimeout; } public Request(String leaderIndex, String followIndex) { this(leaderIndex, followIndex, ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE); + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, null); } Request() { @@ -147,6 +150,7 @@ public void readFrom(StreamInput in) throws IOException { maxWriteSize = in.readVInt(); maxConcurrentWrites = in.readVInt(); maxBufferSize = in.readVInt(); + retryTimeout = in.readOptionalTimeValue(); } @Override @@ -160,6 +164,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(maxWriteSize); out.writeVInt(maxConcurrentWrites); out.writeVInt(maxBufferSize); + out.writeOptionalTimeValue(retryTimeout); } @Override @@ -173,6 +178,7 @@ public boolean equals(Object o) { maxWriteSize == request.maxWriteSize && maxConcurrentWrites == request.maxConcurrentWrites && maxBufferSize == request.maxBufferSize && + Objects.equals(retryTimeout, request.retryTimeout) && Objects.equals(leaderIndex, request.leaderIndex) && Objects.equals(followIndex, request.followIndex); } @@ -180,7 +186,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(leaderIndex, followIndex, maxOperationCount, maxConcurrentReads, maxOperationSizeInBytes, - maxWriteSize, maxConcurrentWrites, maxBufferSize); + maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout); } } @@ -275,11 +281,15 @@ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMe .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; + TimeValue retryTimeout = ShardFollowNodeTask.DEFAULT_RETRY_TIMEOUT; + if (request.retryTimeout != null) { + retryTimeout = request.retryTimeout; + } ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, new ShardId(followIndexMetadata.getIndex(), shardId), new ShardId(leaderIndexMetadata.getIndex(), shardId), request.maxOperationCount, request.maxConcurrentReads, request.maxOperationSizeInBytes, - request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, filteredHeaders); + request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, retryTimeout, filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, new ActionListener>() { @Override 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 a04c50a90e013..eebeebf65ab6e 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 @@ -49,11 +49,12 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { public static final int DEFAULT_MAX_BUFFER_SIZE = 10240; public static final long DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES = Long.MAX_VALUE; private static final int RETRY_LIMIT = 10; - private static final TimeValue RETRY_TIMEOUT = TimeValue.timeValueMillis(500); + public static final TimeValue DEFAULT_RETRY_TIMEOUT = new TimeValue(500); private static final Logger LOGGER = Loggers.getLogger(ShardFollowNodeTask.class); private final ShardFollowTask params; + private final TimeValue retryTimeout; private final TimeValue idleShardChangesRequestDelay; private final BiConsumer scheduler; @@ -68,10 +69,12 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { private final Queue buffer = new PriorityQueue<>(Comparator.comparing(Translog.Operation::seqNo).reversed()); ShardFollowNodeTask(long id, String type, String action, String description, TaskId parentTask, Map headers, - ShardFollowTask params, BiConsumer scheduler, TimeValue idleShardChangesRequestDelay) { + ShardFollowTask params, BiConsumer scheduler, TimeValue idleShardChangesRequestDelay, + TimeValue retryTimeout) { super(id, type, action, description, parentTask, headers); this.params = params; this.scheduler = scheduler; + this.retryTimeout = retryTimeout; this.idleShardChangesRequestDelay = idleShardChangesRequestDelay; } @@ -241,7 +244,7 @@ void handleFailure(Exception e, Runnable task) { if (shouldRetry(e)) { if (isStopped() == false && retryCounter.incrementAndGet() <= RETRY_LIMIT) { LOGGER.debug(new ParameterizedMessage("{} error during follow shard task, retrying...", params.getFollowShardId()), e); - scheduler.accept(RETRY_TIMEOUT, task); + scheduler.accept(retryTimeout, task); } else { markAsFailed(new ElasticsearchException("retrying failed [" + retryCounter.get() + "] times, aborting...", e)); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index 7701f1607238f..43b2ead75ecd3 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -10,7 +10,9 @@ import org.elasticsearch.common.Strings; 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.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; @@ -46,12 +48,13 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { public static final ParseField MAX_WRITE_SIZE = new ParseField("max_write_size"); public static final ParseField MAX_CONCURRENT_WRITES_BATCHES = new ParseField("max_concurrent_write_batches"); public static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); + public static final ParseField RETRY_TIMEOUT = new ParseField("retry_timeout"); @SuppressWarnings("unchecked") private static ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, (a) -> new ShardFollowTask((String) a[0], new ShardId((String) a[1], (String) a[2], (int) a[3]), new ShardId((String) a[4], (String) a[5], (int) a[6]), (int) a[7], (int) a[8], (long) a[9], - (int) a[10], (int) a[11], (int) a[12], (Map) a[13])); + (int) a[10], (int) a[11], (int) a[12], (TimeValue) a[13], (Map) a[14])); static { PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), LEADER_CLUSTER_ALIAS_FIELD); @@ -67,6 +70,9 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_SIZE); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITES_BATCHES); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_BUFFER_SIZE); + PARSER.declareField(ConstructingObjectParser.constructorArg(), + (p, c) -> TimeValue.parseTimeValue(p.text(), RETRY_TIMEOUT.getPreferredName()), + RETRY_TIMEOUT, ObjectParser.ValueType.STRING); PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> p.mapStrings(), HEADERS); } @@ -79,11 +85,12 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { private final int maxWriteSize; private final int maxConcurrentWrites; private final int maxBufferSize; + private final TimeValue retryTimeout; private final Map headers; ShardFollowTask(String leaderClusterAlias, ShardId followShardId, ShardId leaderShardId, int maxReadSize, int maxConcurrentReads, long maxOperationSizeInBytes, int maxWriteSize, int maxConcurrentWrites, - int maxBufferSize, Map headers) { + int maxBufferSize, TimeValue retryTimeout, Map headers) { this.leaderClusterAlias = leaderClusterAlias; this.followShardId = followShardId; this.leaderShardId = leaderShardId; @@ -93,6 +100,7 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { this.maxWriteSize = maxWriteSize; this.maxConcurrentWrites = maxConcurrentWrites; this.maxBufferSize = maxBufferSize; + this.retryTimeout = retryTimeout; this.headers = headers != null ? Collections.unmodifiableMap(headers) : Collections.emptyMap(); } @@ -106,6 +114,7 @@ public ShardFollowTask(StreamInput in) throws IOException { this.maxWriteSize = in.readVInt(); this.maxConcurrentWrites= in.readVInt(); this.maxBufferSize = in.readVInt(); + this.retryTimeout = in.readTimeValue(); this.headers = Collections.unmodifiableMap(in.readMap(StreamInput::readString, StreamInput::readString)); } @@ -145,6 +154,10 @@ public long getMaxOperationSizeInBytes() { return maxOperationSizeInBytes; } + public TimeValue getRetryTimeout() { + return retryTimeout; + } + public Map getHeaders() { return headers; } @@ -165,6 +178,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(maxWriteSize); out.writeVInt(maxConcurrentWrites); out.writeVInt(maxBufferSize); + out.writeTimeValue(retryTimeout); out.writeMap(headers, StreamOutput::writeString, StreamOutput::writeString); } @@ -190,6 +204,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(MAX_WRITE_SIZE.getPreferredName(), maxWriteSize); builder.field(MAX_CONCURRENT_WRITES_BATCHES.getPreferredName(), maxConcurrentWrites); builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxBufferSize); + builder.field(RETRY_TIMEOUT.getPreferredName(), retryTimeout.getStringRep()); builder.field(HEADERS.getPreferredName(), headers); return builder.endObject(); } @@ -208,13 +223,14 @@ public boolean equals(Object o) { maxConcurrentWrites == that.maxConcurrentWrites && maxOperationSizeInBytes == that.maxOperationSizeInBytes && maxBufferSize == that.maxBufferSize && + Objects.equals(retryTimeout, that.retryTimeout) && Objects.equals(headers, that.headers); } @Override public int hashCode() { return Objects.hash(leaderClusterAlias, followShardId, leaderShardId, maxReadSize, maxConcurrentReads, - maxWriteSize, maxConcurrentWrites, maxOperationSizeInBytes, maxBufferSize, headers); + maxWriteSize, maxConcurrentWrites, maxOperationSizeInBytes, maxBufferSize, retryTimeout, headers); } public String toString() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 6b69a4f4fac55..779cadf405930 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -93,7 +93,7 @@ protected AllocatedPersistentTask createTask(long id, String type, String action BiConsumer scheduler = (delay, command) -> threadPool.schedule(delay, Ccr.CCR_THREAD_POOL_NAME, command); return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers, params, - scheduler, idleShardChangesRequestDelay) { + scheduler, idleShardChangesRequestDelay, params.getRetryTimeout()) { @Override protected void updateMapping(LongConsumer handler) { 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 c5ea53417ddba..7b84be6837329 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 @@ -7,6 +7,7 @@ import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -62,7 +63,9 @@ static Request createRequest(RestRequest restRequest) { if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName())) { maxBufferSize = Integer.parseInt(restRequest.param(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName())); } + TimeValue retryTimeout = restRequest.paramAsTime(ShardFollowTask.RETRY_TIMEOUT.getPreferredName(), + ShardFollowNodeTask.DEFAULT_RETRY_TIMEOUT); return new Request(restRequest.param("leader_index"), restRequest.param("index"), maxOperationCount, maxConcurrentReads, - maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize); + maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout); } } 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 688119f6e7ae8..e394f16722746 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 @@ -290,7 +290,8 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) atLeastDocsIndexed("index1", numDocsIndexed / 3); final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", maxReadSize, - randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(32, 2048), randomIntBetween(2, 10), randomIntBetween(1024, 10240)); + randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(32, 2048), randomIntBetween(2, 10), + randomIntBetween(1024, 10240), TimeValue.timeValueMillis(500)); CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); @@ -331,7 +332,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(32, 2048), randomIntBetween(2, 10), - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE); + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, TimeValue.timeValueMillis(500)); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxOperationCount(), @@ -440,7 +441,7 @@ public void testFollowIndex_lowMaxTranslogBytes() throws Exception { } final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", 1024, 1, 1024, - 1024, 1, 10240); + 1024, 1, 10240, TimeValue.timeValueMillis(500)); final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); 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 5576357456e8c..2ff9d362a51cb 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 @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.ccr.action; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.AbstractStreamableTestCase; public class FollowIndexRequestTests extends AbstractStreamableTestCase { @@ -22,6 +23,7 @@ protected FollowIndexAction.Request createTestInstance() { static FollowIndexAction.Request createTestRequest() { return new FollowIndexAction.Request(randomAlphaOfLength(4), randomAlphaOfLength(4), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), randomNonNegativeLong(), randomIntBetween(1, Integer.MAX_VALUE), - randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE)); + randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), + randomBoolean() ? TimeValue.timeValueMillis(500) : null); } } 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 d93de9820f8e8..fb1bf887712e8 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 @@ -130,7 +130,7 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead AtomicBoolean stopped = new AtomicBoolean(false); ShardFollowTask params = new ShardFollowTask(null, new ShardId("follow_index", "", 0), new ShardId("leader_index", "", 0), maxReadSize, maxConcurrentReads, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, - maxWriteSize, maxConcurrentWrites, bufferLimit, Collections.emptyMap()); + maxWriteSize, maxConcurrentWrites, bufferLimit, TimeValue.timeValueMillis(500), Collections.emptyMap()); BiConsumer scheduler = (delay, task) -> { try { @@ -145,7 +145,7 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead AtomicInteger writeCounter = new AtomicInteger(); LocalCheckpointTracker tracker = new LocalCheckpointTracker(followGlobalCheckpoint, followGlobalCheckpoint); return new ShardFollowNodeTask(1L, "type", ShardFollowTask.NAME, "description", null, Collections.emptyMap(), params, scheduler, - TimeValue.timeValueSeconds(1)) { + TimeValue.timeValueSeconds(1), TimeValue.timeValueMillis(500)) { @Override protected void updateMapping(LongConsumer handler) { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java index 5f6370318f7f8..2ce0df3980e52 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.ccr.action; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.AbstractSerializingTestCase; @@ -32,6 +33,7 @@ protected ShardFollowTask createTestInstance() { randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), + TimeValue.parseTimeValue(randomTimeValue(), ""), randomBoolean() ? null : Collections.singletonMap("key", "value")); } From fb8d47eeb1d8d6b5d6640fe451c70f4da0deb934 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 14:33:29 +0200 Subject: [PATCH 25/41] added to test whether we always return at least 1 op when the maxOperationSizeInBytes limit has been met --- .../ccr/action/ShardChangesActionTests.java | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index 8d7c440ab1d38..adbd738725a7b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -26,7 +26,7 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase { - public void testGetOperationsBetween() throws Exception { + public void testGetOperations() throws Exception { final Settings settings = Settings.builder() .put("index.number_of_shards", 1) .put("index.number_of_replicas", 0) @@ -64,7 +64,7 @@ public void testGetOperationsBetween() throws Exception { assertThat(operations.length, equalTo(10)); } - public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { + public void testGetOperationsWhenShardNotStarted() throws Exception { IndexShard indexShard = Mockito.mock(IndexShard.class); ShardRouting shardRouting = TestShardRouting.newShardRouting("index", 0, "_node_id", true, ShardRoutingState.INITIALIZING); @@ -73,7 +73,7 @@ public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { indexShard.getGlobalCheckpoint(), 0, 1, Long.MAX_VALUE)); } - public void testGetOperationsBetweenExceedByteLimit() throws Exception { + public void testGetOperationsExceedByteLimit() throws Exception { final Settings settings = Settings.builder() .put("index.number_of_shards", 1) .put("index.number_of_replicas", 0) @@ -103,4 +103,20 @@ public void testGetOperationsBetweenExceedByteLimit() throws Exception { assertThat(operations[11].seqNo(), equalTo(11L)); } + public void testGetOperationsAlwaysReturnAtLeastOneOp() throws Exception { + final Settings settings = Settings.builder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 0) + .build(); + final IndexService indexService = createIndex("index", settings); + + client().prepareIndex("index", "doc", "0").setSource("{}", XContentType.JSON).get(); + + final IndexShard indexShard = indexService.getShard(0); + final Translog.Operation[] operations = + ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(), 0, 1, 0); + assertThat(operations.length, equalTo(1)); + assertThat(operations[0].seqNo(), equalTo(0L)); + } + } From b0e3d4916c3f45e874a40896fc840ffcdccf6298 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 15:09:20 +0200 Subject: [PATCH 26/41] changed retry delay in tests to 10ms --- .../plugin/ccr/qa/multi-cluster-with-security/build.gradle | 1 + x-pack/plugin/ccr/qa/multi-cluster/build.gradle | 1 + .../test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java | 5 ++--- .../java/org/elasticsearch/xpack/ccr/ShardChangesIT.java | 2 +- .../xpack/ccr/action/ShardFollowNodeTaskTests.java | 2 +- 5 files changed, 6 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle b/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle index cee78c0c91026..d225d7d9a6cc3 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle +++ b/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle @@ -48,6 +48,7 @@ followClusterTestCluster { setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.security.enabled', 'true' setting 'xpack.monitoring.enabled', 'false' + setting 'xpack.ccr.idle_shard_retry_delay', '10ms' extraConfigFile 'roles.yml', 'roles.yml' setupCommand 'setupTestAdmin', 'bin/elasticsearch-users', 'useradd', "test_admin", '-p', 'x-pack-test-password', '-r', "superuser" diff --git a/x-pack/plugin/ccr/qa/multi-cluster/build.gradle b/x-pack/plugin/ccr/qa/multi-cluster/build.gradle index a2848697fe5c1..ca08c340fd86e 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/build.gradle +++ b/x-pack/plugin/ccr/qa/multi-cluster/build.gradle @@ -27,6 +27,7 @@ followClusterTestCluster { numNodes = 1 clusterName = 'follow-cluster' setting 'search.remote.leader_cluster.seeds', "\"${-> leaderClusterTest.nodes.get(0).transportUri()}\"" + setting 'xpack.ccr.idle_shard_retry_delay', '10ms' } followClusterTestRunner { diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 05ba629c9d9d9..5a52970e7f21d 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; @@ -65,7 +64,7 @@ public void testFollowIndex() throws Exception { logger.info("Running against follow cluster"); final String followIndexName = "test_index2"; createAndFollowIndex("leader_cluster:" + leaderIndexName, followIndexName); - assertBusy(() -> verifyDocuments(followIndexName, numDocs), 20, TimeUnit.SECONDS); + assertBusy(() -> verifyDocuments(followIndexName, numDocs)); // unfollow and then follow and then index a few docs in leader index: unfollowIndex(followIndexName); followIndex("leader_cluster:" + leaderIndexName, followIndexName); @@ -75,7 +74,7 @@ public void testFollowIndex() throws Exception { index(leaderClient, leaderIndexName, Integer.toString(id + 1), "field", id + 1, "filtered_field", "true"); index(leaderClient, leaderIndexName, Integer.toString(id + 2), "field", id + 2, "filtered_field", "true"); } - assertBusy(() -> verifyDocuments(followIndexName, numDocs + 3), 20, TimeUnit.SECONDS); + assertBusy(() -> verifyDocuments(followIndexName, numDocs + 3)); } } 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 e394f16722746..c14ca3d32883f 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 @@ -77,7 +77,7 @@ protected Settings nodeSettings(int nodeOrdinal) { newSettings.put(XPackSettings.WATCHER_ENABLED.getKey(), false); newSettings.put(XPackSettings.MACHINE_LEARNING_ENABLED.getKey(), false); newSettings.put(XPackSettings.LOGSTASH_ENABLED.getKey(), false); - newSettings.put(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), TimeValue.timeValueSeconds(1)); + newSettings.put(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), TimeValue.timeValueMillis(10)); return newSettings.build(); } 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 fb1bf887712e8..3ccd2bc2efded 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 @@ -145,7 +145,7 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead AtomicInteger writeCounter = new AtomicInteger(); LocalCheckpointTracker tracker = new LocalCheckpointTracker(followGlobalCheckpoint, followGlobalCheckpoint); return new ShardFollowNodeTask(1L, "type", ShardFollowTask.NAME, "description", null, Collections.emptyMap(), params, scheduler, - TimeValue.timeValueSeconds(1), TimeValue.timeValueMillis(500)) { + TimeValue.timeValueMillis(10), TimeValue.timeValueMillis(500)) { @Override protected void updateMapping(LongConsumer handler) { From b6f0320e7ccc47ca9b5d0293913fbb102935efdd Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 16:58:36 +0200 Subject: [PATCH 27/41] made idelShardRetryDelay a parameter in the follow api --- .../multi-cluster-with-security/build.gradle | 1 - .../xpack/ccr/FollowIndexSecurityIT.java | 8 +++--- .../plugin/ccr/qa/multi-cluster/build.gradle | 1 - .../xpack/ccr/FollowIndexIT.java | 2 ++ .../elasticsearch/xpack/ccr/CcrSettings.java | 11 +------- .../xpack/ccr/action/FollowIndexAction.java | 27 +++++++++++-------- .../xpack/ccr/action/ShardFollowNodeTask.java | 1 + .../xpack/ccr/action/ShardFollowTask.java | 20 +++++++++++--- .../ccr/action/ShardFollowTasksExecutor.java | 5 +--- .../xpack/ccr/rest/RestFollowIndexAction.java | 4 ++- .../xpack/ccr/ShardChangesIT.java | 27 ++++++++++++------- .../ccr/action/FollowIndexActionTests.java | 3 ++- .../ccr/action/FollowIndexRequestTests.java | 2 +- .../ccr/action/ShardFollowNodeTaskTests.java | 3 ++- .../ccr/action/ShardFollowTaskTests.java | 1 + 15 files changed, 69 insertions(+), 47 deletions(-) diff --git a/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle b/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle index d225d7d9a6cc3..cee78c0c91026 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle +++ b/x-pack/plugin/ccr/qa/multi-cluster-with-security/build.gradle @@ -48,7 +48,6 @@ followClusterTestCluster { setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.security.enabled', 'true' setting 'xpack.monitoring.enabled', 'false' - setting 'xpack.ccr.idle_shard_retry_delay', '10ms' extraConfigFile 'roles.yml', 'roles.yml' setupCommand 'setupTestAdmin', 'bin/elasticsearch-users', 'useradd', "test_admin", '-p', 'x-pack-test-password', '-r', "superuser" diff --git a/x-pack/plugin/ccr/qa/multi-cluster-with-security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java b/x-pack/plugin/ccr/qa/multi-cluster-with-security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java index d8c0c2f0c27df..bfd32f385d20c 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster-with-security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster-with-security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java @@ -84,7 +84,7 @@ public void testFollowIndex() throws Exception { assertThat(tasks.size(), equalTo(0)); assertThat(countCcrNodeTasks(), equalTo(0)); }); - + followIndex("leader_cluster:" + allowedIndex, allowedIndex); assertThat(countCcrNodeTasks(), equalTo(1)); assertOK(client().performRequest(new Request("POST", "/" + allowedIndex + "/_xpack/ccr/_unfollow"))); @@ -95,14 +95,14 @@ public void testFollowIndex() throws Exception { assertThat(tasks.size(), equalTo(0)); assertThat(countCcrNodeTasks(), equalTo(0)); }); - + createAndFollowIndex("leader_cluster:" + unallowedIndex, unallowedIndex); // Verify that nothing has been replicated and no node tasks are running // These node tasks should have been failed due to the fact that the user // has no sufficient priviledges. assertBusy(() -> assertThat(countCcrNodeTasks(), equalTo(0))); verifyDocuments(adminClient(), unallowedIndex, 0); - + followIndex("leader_cluster:" + unallowedIndex, unallowedIndex); assertBusy(() -> assertThat(countCcrNodeTasks(), equalTo(0))); verifyDocuments(adminClient(), unallowedIndex, 0); @@ -146,12 +146,14 @@ private static void refresh(String index) throws IOException { private static void followIndex(String leaderIndex, String followIndex) throws IOException { final Request request = new Request("POST", "/" + followIndex + "/_xpack/ccr/_follow"); request.addParameter("leader_index", leaderIndex); + request.addParameter("idle_shard_retry_delay", "10ms"); assertOK(client().performRequest(request)); } private static void createAndFollowIndex(String leaderIndex, String followIndex) throws IOException { final Request request = new Request("POST", "/" + followIndex + "/_xpack/ccr/_create_and_follow"); request.addParameter("leader_index", leaderIndex); + request.addParameter("idle_shard_retry_delay", "10ms"); assertOK(client().performRequest(request)); } diff --git a/x-pack/plugin/ccr/qa/multi-cluster/build.gradle b/x-pack/plugin/ccr/qa/multi-cluster/build.gradle index ca08c340fd86e..a2848697fe5c1 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/build.gradle +++ b/x-pack/plugin/ccr/qa/multi-cluster/build.gradle @@ -27,7 +27,6 @@ followClusterTestCluster { numNodes = 1 clusterName = 'follow-cluster' setting 'search.remote.leader_cluster.seeds', "\"${-> leaderClusterTest.nodes.get(0).transportUri()}\"" - setting 'xpack.ccr.idle_shard_retry_delay', '10ms' } followClusterTestRunner { diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 5a52970e7f21d..8e637f7dbc183 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -96,12 +96,14 @@ private static void refresh(String index) throws IOException { private static void followIndex(String leaderIndex, String followIndex) throws IOException { final Request request = new Request("POST", "/" + followIndex + "/_xpack/ccr/_follow"); request.addParameter("leader_index", leaderIndex); + request.addParameter("idle_shard_retry_delay", "10ms"); assertOK(client().performRequest(request)); } private static void createAndFollowIndex(String leaderIndex, String followIndex) throws IOException { final Request request = new Request("POST", "/" + followIndex + "/_xpack/ccr/_create_and_follow"); request.addParameter("leader_index", leaderIndex); + request.addParameter("idle_shard_retry_delay", "10ms"); assertOK(client().performRequest(request)); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java index d5ca346875927..6960766bad0cb 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrSettings.java @@ -7,7 +7,6 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.unit.TimeValue; import java.util.Arrays; import java.util.List; @@ -33,13 +32,6 @@ private CcrSettings() { public static final Setting CCR_FOLLOWING_INDEX_SETTING = Setting.boolSetting("index.xpack.ccr.following_index", false, Setting.Property.IndexScope); - /** - * Node setting for controlling the timeout between the next shard changes request when a shard follow - * task is idle. - */ - public static final Setting CCR_IDLE_SHARD_RETRY_DELAY = Setting.timeSetting( - "xpack.ccr.idle_shard_retry_delay", TimeValue.timeValueSeconds(10), Property.NodeScope); - /** * The settings defined by CCR. * @@ -48,8 +40,7 @@ private CcrSettings() { static List> getSettings() { return Arrays.asList( CCR_ENABLED_SETTING, - CCR_FOLLOWING_INDEX_SETTING, - CCR_IDLE_SHARD_RETRY_DELAY); + CCR_FOLLOWING_INDEX_SETTING); } } 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 index daf67303e7321..5d88ca9555641 100644 --- 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 @@ -79,9 +79,11 @@ public static class Request extends ActionRequest { private int maxConcurrentWrites; private int maxBufferSize; private TimeValue retryTimeout; + private TimeValue idleShardRetryDelay; - public Request(String leaderIndex, String followIndex, int maxOperationCount, int maxConcurrentReads, long maxOperationSizeInBytes, - int maxWriteSize, int maxConcurrentWrites, int maxBufferSize, TimeValue retryTimeout) { + public Request(String leaderIndex, String followIndex, int maxOperationCount, int maxConcurrentReads, + long maxOperationSizeInBytes, int maxWriteSize, int maxConcurrentWrites, int maxBufferSize, + TimeValue retryTimeout, TimeValue idleShardRetryDelay) { if (maxOperationCount < 1) { throw new IllegalArgumentException("maxOperationCount must be larger than 0"); } @@ -110,13 +112,7 @@ public Request(String leaderIndex, String followIndex, int maxOperationCount, in this.maxConcurrentWrites = maxConcurrentWrites; this.maxBufferSize = maxBufferSize; this.retryTimeout = retryTimeout; - } - - public Request(String leaderIndex, String followIndex) { - this(leaderIndex, followIndex, ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT, - ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, - ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, null); + this.idleShardRetryDelay = idleShardRetryDelay; } Request() { @@ -151,6 +147,7 @@ public void readFrom(StreamInput in) throws IOException { maxConcurrentWrites = in.readVInt(); maxBufferSize = in.readVInt(); retryTimeout = in.readOptionalTimeValue(); + idleShardRetryDelay = in.readOptionalTimeValue(); } @Override @@ -165,6 +162,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(maxConcurrentWrites); out.writeVInt(maxBufferSize); out.writeOptionalTimeValue(retryTimeout); + out.writeOptionalTimeValue(idleShardRetryDelay); } @Override @@ -179,6 +177,7 @@ public boolean equals(Object o) { maxConcurrentWrites == request.maxConcurrentWrites && maxBufferSize == request.maxBufferSize && Objects.equals(retryTimeout, request.retryTimeout) && + Objects.equals(idleShardRetryDelay, request.idleShardRetryDelay) && Objects.equals(leaderIndex, request.leaderIndex) && Objects.equals(followIndex, request.followIndex); } @@ -186,7 +185,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(leaderIndex, followIndex, maxOperationCount, maxConcurrentReads, maxOperationSizeInBytes, - maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout); + maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout, idleShardRetryDelay); } } @@ -285,11 +284,17 @@ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMe if (request.retryTimeout != null) { retryTimeout = request.retryTimeout; } + TimeValue idleShardRetryDelay = ShardFollowNodeTask.DEFAULT_IDLE_SHARD_RETRY_DELAY; + if (request.idleShardRetryDelay != null) { + idleShardRetryDelay = request.idleShardRetryDelay; + } + ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, new ShardId(followIndexMetadata.getIndex(), shardId), new ShardId(leaderIndexMetadata.getIndex(), shardId), request.maxOperationCount, request.maxConcurrentReads, request.maxOperationSizeInBytes, - request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, retryTimeout, filteredHeaders); + request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, retryTimeout, + idleShardRetryDelay, filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, new ActionListener>() { @Override 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 eebeebf65ab6e..4892211ff8b8c 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 @@ -50,6 +50,7 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { public static final long DEFAULT_MAX_OPERATIONS_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); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index 43b2ead75ecd3..af98f01941404 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -49,12 +49,13 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { public static final ParseField MAX_CONCURRENT_WRITES_BATCHES = new ParseField("max_concurrent_write_batches"); public static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); public static final ParseField RETRY_TIMEOUT = new ParseField("retry_timeout"); + public static final ParseField IDLE_SHARD_RETRY_DELAY = new ParseField("idle_shard_retry_delay"); @SuppressWarnings("unchecked") private static ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, (a) -> new ShardFollowTask((String) a[0], new ShardId((String) a[1], (String) a[2], (int) a[3]), new ShardId((String) a[4], (String) a[5], (int) a[6]), (int) a[7], (int) a[8], (long) a[9], - (int) a[10], (int) a[11], (int) a[12], (TimeValue) a[13], (Map) a[14])); + (int) a[10], (int) a[11], (int) a[12], (TimeValue) a[13], (TimeValue) a[14], (Map) a[15])); static { PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), LEADER_CLUSTER_ALIAS_FIELD); @@ -73,6 +74,9 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { PARSER.declareField(ConstructingObjectParser.constructorArg(), (p, c) -> TimeValue.parseTimeValue(p.text(), RETRY_TIMEOUT.getPreferredName()), RETRY_TIMEOUT, ObjectParser.ValueType.STRING); + PARSER.declareField(ConstructingObjectParser.constructorArg(), + (p, c) -> TimeValue.parseTimeValue(p.text(), IDLE_SHARD_RETRY_DELAY.getPreferredName()), + IDLE_SHARD_RETRY_DELAY, ObjectParser.ValueType.STRING); PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> p.mapStrings(), HEADERS); } @@ -86,11 +90,12 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { private final int maxConcurrentWrites; private final int maxBufferSize; private final TimeValue retryTimeout; + private final TimeValue idleShardRetryDelay; private final Map headers; ShardFollowTask(String leaderClusterAlias, ShardId followShardId, ShardId leaderShardId, int maxReadSize, int maxConcurrentReads, long maxOperationSizeInBytes, int maxWriteSize, int maxConcurrentWrites, - int maxBufferSize, TimeValue retryTimeout, Map headers) { + int maxBufferSize, TimeValue retryTimeout, TimeValue idleShardRetryDelay, Map headers) { this.leaderClusterAlias = leaderClusterAlias; this.followShardId = followShardId; this.leaderShardId = leaderShardId; @@ -101,6 +106,7 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { this.maxConcurrentWrites = maxConcurrentWrites; this.maxBufferSize = maxBufferSize; this.retryTimeout = retryTimeout; + this.idleShardRetryDelay = idleShardRetryDelay; this.headers = headers != null ? Collections.unmodifiableMap(headers) : Collections.emptyMap(); } @@ -115,6 +121,7 @@ public ShardFollowTask(StreamInput in) throws IOException { this.maxConcurrentWrites= in.readVInt(); this.maxBufferSize = in.readVInt(); this.retryTimeout = in.readTimeValue(); + this.idleShardRetryDelay = in.readTimeValue(); this.headers = Collections.unmodifiableMap(in.readMap(StreamInput::readString, StreamInput::readString)); } @@ -158,6 +165,10 @@ public TimeValue getRetryTimeout() { return retryTimeout; } + public TimeValue getIdleShardRetryDelay() { + return idleShardRetryDelay; + } + public Map getHeaders() { return headers; } @@ -179,6 +190,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(maxConcurrentWrites); out.writeVInt(maxBufferSize); out.writeTimeValue(retryTimeout); + out.writeTimeValue(idleShardRetryDelay); out.writeMap(headers, StreamOutput::writeString, StreamOutput::writeString); } @@ -205,6 +217,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(MAX_CONCURRENT_WRITES_BATCHES.getPreferredName(), maxConcurrentWrites); builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxBufferSize); builder.field(RETRY_TIMEOUT.getPreferredName(), retryTimeout.getStringRep()); + builder.field(IDLE_SHARD_RETRY_DELAY.getPreferredName(), idleShardRetryDelay.getStringRep()); builder.field(HEADERS.getPreferredName(), headers); return builder.endObject(); } @@ -224,13 +237,14 @@ public boolean equals(Object o) { maxOperationSizeInBytes == that.maxOperationSizeInBytes && maxBufferSize == that.maxBufferSize && Objects.equals(retryTimeout, that.retryTimeout) && + Objects.equals(idleShardRetryDelay, that.idleShardRetryDelay) && Objects.equals(headers, that.headers); } @Override public int hashCode() { return Objects.hash(leaderClusterAlias, followShardId, leaderShardId, maxReadSize, maxConcurrentReads, - maxWriteSize, maxConcurrentWrites, maxOperationSizeInBytes, maxBufferSize, retryTimeout, headers); + maxWriteSize, maxConcurrentWrites, maxOperationSizeInBytes, maxBufferSize, retryTimeout, idleShardRetryDelay, headers); } public String toString() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 779cadf405930..6b3717ce7428c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -35,7 +35,6 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.Ccr; -import org.elasticsearch.xpack.ccr.CcrSettings; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; @@ -87,13 +86,11 @@ protected AllocatedPersistentTask createTask(long id, String type, String action } else { leaderClient = wrapClient(client, params); } - TimeValue idleShardChangesRequestDelay = settings.getAsTime(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), - CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getDefault(settings)); Client followerClient = wrapClient(client, params); BiConsumer scheduler = (delay, command) -> threadPool.schedule(delay, Ccr.CCR_THREAD_POOL_NAME, command); return new ShardFollowNodeTask(id, type, action, getDescription(taskInProgress), parentTaskId, headers, params, - scheduler, idleShardChangesRequestDelay, params.getRetryTimeout()) { + scheduler, params.getIdleShardRetryDelay(), params.getRetryTimeout()) { @Override protected void updateMapping(LongConsumer handler) { 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 7b84be6837329..ab1974f69e9b5 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 @@ -65,7 +65,9 @@ static Request createRequest(RestRequest restRequest) { } TimeValue retryTimeout = restRequest.paramAsTime(ShardFollowTask.RETRY_TIMEOUT.getPreferredName(), ShardFollowNodeTask.DEFAULT_RETRY_TIMEOUT); + TimeValue idleShardRetryTimeout = restRequest.paramAsTime(ShardFollowTask.IDLE_SHARD_RETRY_DELAY.getPreferredName(), + ShardFollowNodeTask.DEFAULT_IDLE_SHARD_RETRY_DELAY); return new Request(restRequest.param("leader_index"), restRequest.param("index"), maxOperationCount, maxConcurrentReads, - maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout); + maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout, idleShardRetryTimeout); } } 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 c14ca3d32883f..155f10557a654 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 @@ -77,7 +77,6 @@ protected Settings nodeSettings(int nodeOrdinal) { newSettings.put(XPackSettings.WATCHER_ENABLED.getKey(), false); newSettings.put(XPackSettings.MACHINE_LEARNING_ENABLED.getKey(), false); newSettings.put(XPackSettings.LOGSTASH_ENABLED.getKey(), false); - newSettings.put(CcrSettings.CCR_IDLE_SHARD_RETRY_DELAY.getKey(), TimeValue.timeValueMillis(10)); return newSettings.build(); } @@ -165,7 +164,7 @@ public void testFollowIndex() throws Exception { assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); ensureYellow("index1"); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2"); + final FollowIndexAction.Request followRequest = createFollowRequest("index1", "index2"); final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); @@ -223,7 +222,7 @@ public void testSyncMappings() throws Exception { assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); ensureYellow("index1"); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2"); + final FollowIndexAction.Request followRequest = createFollowRequest("index1", "index2"); final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); @@ -291,7 +290,7 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", maxReadSize, randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(32, 2048), randomIntBetween(2, 10), - randomIntBetween(1024, 10240), TimeValue.timeValueMillis(500)); + randomIntBetween(1024, 10240), TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); @@ -332,7 +331,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(32, 2048), randomIntBetween(2, 10), - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, TimeValue.timeValueMillis(500)); + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxOperationCount(), @@ -375,7 +374,7 @@ public void testFollowIndexWithNestedField() throws Exception { ensureGreen("index1", "index2"); - final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2"); + final FollowIndexAction.Request followRequest = createFollowRequest("index1", "index2"); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); final int numDocs = randomIntBetween(2, 64); @@ -418,13 +417,13 @@ public void testFollowNonExistentIndex() throws Exception { assertAcked(client().admin().indices().prepareCreate("test-leader").get()); assertAcked(client().admin().indices().prepareCreate("test-follower").get()); // Leader index does not exist. - FollowIndexAction.Request followRequest1 = new FollowIndexAction.Request("non-existent-leader", "test-follower"); + FollowIndexAction.Request followRequest1 = createFollowRequest("non-existent-leader", "test-follower"); expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest1).actionGet()); // Follower index does not exist. - FollowIndexAction.Request followRequest2 = new FollowIndexAction.Request("non-test-leader", "non-existent-follower"); + FollowIndexAction.Request followRequest2 = createFollowRequest("non-test-leader", "non-existent-follower"); expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest2).actionGet()); // Both indices do not exist. - FollowIndexAction.Request followRequest3 = new FollowIndexAction.Request("non-existent-leader", "non-existent-follower"); + FollowIndexAction.Request followRequest3 = createFollowRequest("non-existent-leader", "non-existent-follower"); expectThrows(IllegalArgumentException.class, () -> client().execute(FollowIndexAction.INSTANCE, followRequest3).actionGet()); } @@ -441,7 +440,7 @@ public void testFollowIndex_lowMaxTranslogBytes() throws Exception { } final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", 1024, 1, 1024, - 1024, 1, 10240, TimeValue.timeValueMillis(500)); + 1024, 1, 10240, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); @@ -642,4 +641,12 @@ private void assertSameDocCount(String index1, String index2) throws Exception { assertThat(response2.getHits().getTotalHits(), equalTo(response1.getHits().getTotalHits())); }); } + + public static FollowIndexAction.Request createFollowRequest(String leaderIndex, String followIndex) { + return new FollowIndexAction.Request(leaderIndex, followIndex, ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT, + ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, + ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, + ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, TimeValue.timeValueMillis(10), + TimeValue.timeValueMillis(10)); + } } 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/FollowIndexActionTests.java index f0f26b921ac76..31a6eb31c81b2 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/FollowIndexActionTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.index.MapperTestUtils; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.ccr.ShardChangesIT; import java.io.IOException; @@ -22,7 +23,7 @@ public class FollowIndexActionTests extends ESTestCase { public void testValidation() throws IOException { - FollowIndexAction.Request request = new FollowIndexAction.Request("index1", "index2"); + 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)); 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 2ff9d362a51cb..462f80461df43 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 @@ -24,6 +24,6 @@ static FollowIndexAction.Request createTestRequest() { return new FollowIndexAction.Request(randomAlphaOfLength(4), randomAlphaOfLength(4), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), randomNonNegativeLong(), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), - randomBoolean() ? TimeValue.timeValueMillis(500) : null); + randomBoolean() ? TimeValue.timeValueMillis(500) : null, randomBoolean() ? TimeValue.timeValueMillis(500) : null); } } 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 3ccd2bc2efded..c54366801849c 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 @@ -130,7 +130,8 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead AtomicBoolean stopped = new AtomicBoolean(false); ShardFollowTask params = new ShardFollowTask(null, new ShardId("follow_index", "", 0), new ShardId("leader_index", "", 0), maxReadSize, maxConcurrentReads, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, - maxWriteSize, maxConcurrentWrites, bufferLimit, TimeValue.timeValueMillis(500), Collections.emptyMap()); + maxWriteSize, maxConcurrentWrites, bufferLimit, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10), + Collections.emptyMap()); BiConsumer scheduler = (delay, task) -> { try { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java index 2ce0df3980e52..dd4d2bb4ed834 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java @@ -34,6 +34,7 @@ protected ShardFollowTask createTestInstance() { randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), TimeValue.parseTimeValue(randomTimeValue(), ""), + TimeValue.parseTimeValue(randomTimeValue(), ""), randomBoolean() ? null : Collections.singletonMap("key", "value")); } From d17a4b339e37aedddfff42f6a2db505ee4590e06 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 5 Jul 2018 17:18:23 +0200 Subject: [PATCH 28/41] added comment --- .../elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 4892211ff8b8c..fed439afb6bbd 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 @@ -190,6 +190,12 @@ private void handleReadResponse(long from, int maxOperationCount, long maxRequir sendShardChangesRequest(newFrom, newSize, maxRequiredSeqNo); return; } + // If lastOpSeqNo == null then from must be equal to maxRequiredSeqNo otherwise something went wrong and + // we are not getting ops while we really should. + // There is one exception why can't fail because of this and that is that we may read from a leader shard + // copy with a global checkpoint equal to from and no ops are than being returned. If we can detect this + // here or make sure that the shard changes api does not do this (add a flag requireOps and let it fail + // so that it can read from another shard copy) then we can add a hard check here. numConcurrentReads--; if (response.getOperations().length != 0) { From 53fe1c24dba98481176363b996670c3f4293792a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 08:10:22 +0200 Subject: [PATCH 29/41] made parameter names consistent and collapsed maxReadSize and maxWriteSize into maxBatchOperationCount --- .../xpack/ccr/action/FollowIndexAction.java | 80 ++++++------- .../xpack/ccr/action/ShardChangesAction.java | 2 +- .../xpack/ccr/action/ShardFollowNodeTask.java | 21 ++-- .../xpack/ccr/action/ShardFollowTask.java | 112 ++++++++---------- .../ccr/action/ShardFollowTasksExecutor.java | 2 +- .../xpack/ccr/rest/RestFollowIndexAction.java | 32 +++-- .../xpack/ccr/ShardChangesIT.java | 21 ++-- .../ccr/action/FollowIndexRequestTests.java | 4 +- .../ccr/action/ShardFollowNodeTaskTests.java | 28 ++--- .../ccr/action/ShardFollowTaskTests.java | 1 - 10 files changed, 137 insertions(+), 166 deletions(-) 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 index 5d88ca9555641..af9847a8e7b17 100644 --- 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 @@ -72,45 +72,40 @@ public static class Request extends ActionRequest { private String leaderIndex; private String followIndex; - private int maxOperationCount; - private int maxConcurrentReads; + private int maxBatchOperationCount; + private int maxConcurrentReadBatches; private long maxOperationSizeInBytes; - private int maxWriteSize; - private int maxConcurrentWrites; - private int maxBufferSize; + private int maxConcurrentWriteBatches; + private int maxWriteBufferSize; private TimeValue retryTimeout; private TimeValue idleShardRetryDelay; - public Request(String leaderIndex, String followIndex, int maxOperationCount, int maxConcurrentReads, - long maxOperationSizeInBytes, int maxWriteSize, int maxConcurrentWrites, int maxBufferSize, + public Request(String leaderIndex, String followIndex, int maxBatchOperationCount, int maxConcurrentReadBatches, + long maxOperationSizeInBytes, int maxConcurrentWriteBatches, int maxWriteBufferSize, TimeValue retryTimeout, TimeValue idleShardRetryDelay) { - if (maxOperationCount < 1) { - throw new IllegalArgumentException("maxOperationCount must be larger than 0"); + if (maxBatchOperationCount < 1) { + throw new IllegalArgumentException("maxBatchOperationCount must be larger than 0"); } - if (maxConcurrentReads < 1) { + 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 (maxWriteSize < 1) { - throw new IllegalArgumentException("maxWriteSize must be larger than 0"); + if (maxConcurrentWriteBatches < 1) { + throw new IllegalArgumentException("maxConcurrentWriteBatches must be larger than 0"); } - if (maxConcurrentWrites < 1) { - throw new IllegalArgumentException("maxConcurrentWrites must be larger than 0"); - } - if (maxBufferSize < 1) { - throw new IllegalArgumentException("maxBufferSize must be larger than 0"); + if (maxWriteBufferSize < 1) { + throw new IllegalArgumentException("maxWriteBufferSize must be larger than 0"); } this.leaderIndex = Objects.requireNonNull(leaderIndex); this.followIndex = Objects.requireNonNull(followIndex); - this.maxOperationCount = maxOperationCount; - this.maxConcurrentReads = maxConcurrentReads; + this.maxBatchOperationCount = maxBatchOperationCount; + this.maxConcurrentReadBatches = maxConcurrentReadBatches; this.maxOperationSizeInBytes = maxOperationSizeInBytes; - this.maxWriteSize = maxWriteSize; - this.maxConcurrentWrites = maxConcurrentWrites; - this.maxBufferSize = maxBufferSize; + this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; + this.maxWriteBufferSize = maxWriteBufferSize; this.retryTimeout = retryTimeout; this.idleShardRetryDelay = idleShardRetryDelay; } @@ -126,8 +121,8 @@ public String getFollowIndex() { return followIndex; } - public int getMaxOperationCount() { - return maxOperationCount; + public int getMaxBatchOperationCount() { + return maxBatchOperationCount; } @Override @@ -140,12 +135,11 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); leaderIndex = in.readString(); followIndex = in.readString(); - maxOperationCount = in.readVInt(); - maxConcurrentReads = in.readVInt(); + maxBatchOperationCount = in.readVInt(); + maxConcurrentReadBatches = in.readVInt(); maxOperationSizeInBytes = in.readVLong(); - maxWriteSize = in.readVInt(); - maxConcurrentWrites = in.readVInt(); - maxBufferSize = in.readVInt(); + maxConcurrentWriteBatches = in.readVInt(); + maxWriteBufferSize = in.readVInt(); retryTimeout = in.readOptionalTimeValue(); idleShardRetryDelay = in.readOptionalTimeValue(); } @@ -155,12 +149,11 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(leaderIndex); out.writeString(followIndex); - out.writeVInt(maxOperationCount); - out.writeVInt(maxConcurrentReads); + out.writeVInt(maxBatchOperationCount); + out.writeVInt(maxConcurrentReadBatches); out.writeVLong(maxOperationSizeInBytes); - out.writeVInt(maxWriteSize); - out.writeVInt(maxConcurrentWrites); - out.writeVInt(maxBufferSize); + out.writeVInt(maxConcurrentWriteBatches); + out.writeVInt(maxWriteBufferSize); out.writeOptionalTimeValue(retryTimeout); out.writeOptionalTimeValue(idleShardRetryDelay); } @@ -170,12 +163,11 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; - return maxOperationCount == request.maxOperationCount && - maxConcurrentReads == request.maxConcurrentReads && + return maxBatchOperationCount == request.maxBatchOperationCount && + maxConcurrentReadBatches == request.maxConcurrentReadBatches && maxOperationSizeInBytes == request.maxOperationSizeInBytes && - maxWriteSize == request.maxWriteSize && - maxConcurrentWrites == request.maxConcurrentWrites && - maxBufferSize == request.maxBufferSize && + maxConcurrentWriteBatches == request.maxConcurrentWriteBatches && + maxWriteBufferSize == request.maxWriteBufferSize && Objects.equals(retryTimeout, request.retryTimeout) && Objects.equals(idleShardRetryDelay, request.idleShardRetryDelay) && Objects.equals(leaderIndex, request.leaderIndex) && @@ -184,8 +176,8 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(leaderIndex, followIndex, maxOperationCount, maxConcurrentReads, maxOperationSizeInBytes, - maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout, idleShardRetryDelay); + return Objects.hash(leaderIndex, followIndex, maxBatchOperationCount, maxConcurrentReadBatches, maxOperationSizeInBytes, + maxConcurrentWriteBatches, maxWriteBufferSize, retryTimeout, idleShardRetryDelay); } } @@ -292,9 +284,9 @@ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMe ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, new ShardId(followIndexMetadata.getIndex(), shardId), new ShardId(leaderIndexMetadata.getIndex(), shardId), - request.maxOperationCount, request.maxConcurrentReads, request.maxOperationSizeInBytes, - request.maxWriteSize, request.maxConcurrentWrites, request.maxBufferSize, retryTimeout, - idleShardRetryDelay, filteredHeaders); + request.maxBatchOperationCount, request.maxConcurrentReadBatches, request.maxOperationSizeInBytes, + request.maxConcurrentWriteBatches, request.maxWriteBufferSize, retryTimeout, idleShardRetryDelay, + filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, new ActionListener>() { @Override 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 03f783abcca62..dd67f09bd7246 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 @@ -56,7 +56,7 @@ public static class Request extends SingleShardRequest { private long fromSeqNo; private int maxOperationCount; private ShardId shardId; - private long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; + private long maxOperationSizeInBytes = ShardFollowNodeTask.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 fed439afb6bbd..633c4776858ae 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 @@ -42,12 +42,11 @@ */ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { - public static final int DEFAULT_MAX_OPERATION_COUNT = 1024; - public static final int DEFAULT_MAX_WRITE_SIZE = 1024; - public static final int DEFAULT_MAX_CONCURRENT_READS = 1; - public static final int DEFAULT_MAX_CONCURRENT_WRITES = 1; - public static final int DEFAULT_MAX_BUFFER_SIZE = 10240; - public static final long DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES = Long.MAX_VALUE; + 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); @@ -102,7 +101,7 @@ private synchronized void coordinateReads() { LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, globalCheckpoint={}", params.getFollowShardId(), lastRequestedSeqno, globalCheckpoint); - final int maxReadSize = params.getMaxReadSize(); + final int maxReadSize = params.getMaxBatchOperationCount(); while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { numConcurrentReads++; long from = lastRequestedSeqno + 1; @@ -124,12 +123,12 @@ private synchronized void coordinateReads() { private boolean hasReadBudget() { assert Thread.holdsLock(this); - if (numConcurrentReads >= params.getMaxConcurrentReads()) { + if (numConcurrentReads >= params.getMaxConcurrentReadBatches()) { LOGGER.trace("{} no new reads, maximum number of concurrent reads have been reached [{}]", params.getFollowShardId(), numConcurrentReads); return false; } - if (buffer.size() > params.getMaxBufferSize()) { + if (buffer.size() > params.getMaxWriteBufferSize()) { LOGGER.trace("{} no new reads, buffer limit has been reached [{}]", params.getFollowShardId(), buffer.size()); return false; } @@ -138,7 +137,7 @@ private boolean hasReadBudget() { private synchronized void coordinateWrites() { while (hasWriteBudget() && buffer.isEmpty() == false) { - Translog.Operation[] ops = new Translog.Operation[Math.min(params.getMaxWriteSize(), buffer.size())]; + Translog.Operation[] ops = new Translog.Operation[Math.min(params.getMaxBatchOperationCount(), buffer.size())]; for (int i = 0; i < ops.length; i++) { ops[i] = buffer.remove(); } @@ -151,7 +150,7 @@ private synchronized void coordinateWrites() { private boolean hasWriteBudget() { assert Thread.holdsLock(this); - if (numConcurrentWrites >= params.getMaxConcurrentWrites()) { + if (numConcurrentWrites >= params.getMaxConcurrentWriteBatches()) { LOGGER.trace("{} maximum number of concurrent writes have been reached [{}]", params.getFollowShardId(), numConcurrentWrites); return false; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index af98f01941404..114ec722f2b96 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -43,10 +43,9 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { static final ParseField LEADER_SHARD_SHARDID_FIELD = new ParseField("leader_shard_shard"); static final ParseField HEADERS = new ParseField("headers"); public static final ParseField MAX_BATCH_OPERATION_COUNT = new ParseField("max_batch_operation_count"); - public static final ParseField MAX_CONCURRENT_READS_BATCHES = new ParseField("max_concurrent_read_batches"); + public static final ParseField MAX_CONCURRENT_READ_BATCHES = new ParseField("max_concurrent_read_batches"); public static final ParseField MAX_BATCH_SIZE_IN_BYTES = new ParseField("max_batch_size_in_bytes"); - public static final ParseField MAX_WRITE_SIZE = new ParseField("max_write_size"); - public static final ParseField MAX_CONCURRENT_WRITES_BATCHES = new ParseField("max_concurrent_write_batches"); + public static final ParseField MAX_CONCURRENT_WRITE_BATCHES = new ParseField("max_concurrent_write_batches"); public static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); public static final ParseField RETRY_TIMEOUT = new ParseField("retry_timeout"); public static final ParseField IDLE_SHARD_RETRY_DELAY = new ParseField("idle_shard_retry_delay"); @@ -55,7 +54,7 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { private static ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, (a) -> new ShardFollowTask((String) a[0], new ShardId((String) a[1], (String) a[2], (int) a[3]), new ShardId((String) a[4], (String) a[5], (int) a[6]), (int) a[7], (int) a[8], (long) a[9], - (int) a[10], (int) a[11], (int) a[12], (TimeValue) a[13], (TimeValue) a[14], (Map) a[15])); + (int) a[10], (int) a[11], (TimeValue) a[12], (TimeValue) a[13], (Map) a[14])); static { PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), LEADER_CLUSTER_ALIAS_FIELD); @@ -66,10 +65,9 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { PARSER.declareString(ConstructingObjectParser.constructorArg(), LEADER_SHARD_INDEX_UUID_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), LEADER_SHARD_SHARDID_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_BATCH_OPERATION_COUNT); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READS_BATCHES); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READ_BATCHES); PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAX_BATCH_SIZE_IN_BYTES); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_SIZE); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITES_BATCHES); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITE_BATCHES); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_BUFFER_SIZE); PARSER.declareField(ConstructingObjectParser.constructorArg(), (p, c) -> TimeValue.parseTimeValue(p.text(), RETRY_TIMEOUT.getPreferredName()), @@ -83,28 +81,26 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { private final String leaderClusterAlias; private final ShardId followShardId; private final ShardId leaderShardId; - private final int maxReadSize; - private final int maxConcurrentReads; - private final long maxOperationSizeInBytes; - private final int maxWriteSize; - private final int maxConcurrentWrites; - private final int maxBufferSize; + private final int maxBatchOperationCount; + private final int maxConcurrentReadBatches; + private final long maxBatchSizeInBytes; + private final int maxConcurrentWriteBatches; + private final int maxWriteBufferSize; private final TimeValue retryTimeout; private final TimeValue idleShardRetryDelay; private final Map headers; - ShardFollowTask(String leaderClusterAlias, ShardId followShardId, ShardId leaderShardId, int maxReadSize, - int maxConcurrentReads, long maxOperationSizeInBytes, int maxWriteSize, int maxConcurrentWrites, - int maxBufferSize, TimeValue retryTimeout, TimeValue idleShardRetryDelay, Map headers) { + ShardFollowTask(String leaderClusterAlias, ShardId followShardId, ShardId leaderShardId, int maxBatchOperationCount, + int maxConcurrentReadBatches, long maxBatchSizeInBytes, int maxConcurrentWriteBatches, + int maxWriteBufferSize, TimeValue retryTimeout, TimeValue idleShardRetryDelay, Map headers) { this.leaderClusterAlias = leaderClusterAlias; this.followShardId = followShardId; this.leaderShardId = leaderShardId; - this.maxReadSize = maxReadSize; - this.maxConcurrentReads = maxConcurrentReads; - this.maxOperationSizeInBytes = maxOperationSizeInBytes; - this.maxWriteSize = maxWriteSize; - this.maxConcurrentWrites = maxConcurrentWrites; - this.maxBufferSize = maxBufferSize; + this.maxBatchOperationCount = maxBatchOperationCount; + this.maxConcurrentReadBatches = maxConcurrentReadBatches; + this.maxBatchSizeInBytes = maxBatchSizeInBytes; + this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; + this.maxWriteBufferSize = maxWriteBufferSize; this.retryTimeout = retryTimeout; this.idleShardRetryDelay = idleShardRetryDelay; this.headers = headers != null ? Collections.unmodifiableMap(headers) : Collections.emptyMap(); @@ -114,12 +110,11 @@ public ShardFollowTask(StreamInput in) throws IOException { this.leaderClusterAlias = in.readOptionalString(); this.followShardId = ShardId.readShardId(in); this.leaderShardId = ShardId.readShardId(in); - this.maxReadSize = in.readVInt(); - this.maxConcurrentReads = in.readVInt(); - this.maxOperationSizeInBytes = in.readVLong(); - this.maxWriteSize = in.readVInt(); - this.maxConcurrentWrites= in.readVInt(); - this.maxBufferSize = in.readVInt(); + this.maxBatchOperationCount = in.readVInt(); + this.maxConcurrentReadBatches = in.readVInt(); + this.maxBatchSizeInBytes = in.readVLong(); + this.maxConcurrentWriteBatches = in.readVInt(); + this.maxWriteBufferSize = in.readVInt(); this.retryTimeout = in.readTimeValue(); this.idleShardRetryDelay = in.readTimeValue(); this.headers = Collections.unmodifiableMap(in.readMap(StreamInput::readString, StreamInput::readString)); @@ -137,28 +132,24 @@ public ShardId getLeaderShardId() { return leaderShardId; } - public int getMaxReadSize() { - return maxReadSize; + public int getMaxBatchOperationCount() { + return maxBatchOperationCount; } - public int getMaxWriteSize() { - return maxWriteSize; + public int getMaxConcurrentReadBatches() { + return maxConcurrentReadBatches; } - public int getMaxConcurrentReads() { - return maxConcurrentReads; + public int getMaxConcurrentWriteBatches() { + return maxConcurrentWriteBatches; } - public int getMaxConcurrentWrites() { - return maxConcurrentWrites; + public int getMaxWriteBufferSize() { + return maxWriteBufferSize; } - public int getMaxBufferSize() { - return maxBufferSize; - } - - public long getMaxOperationSizeInBytes() { - return maxOperationSizeInBytes; + public long getMaxBatchSizeInBytes() { + return maxBatchSizeInBytes; } public TimeValue getRetryTimeout() { @@ -183,12 +174,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(leaderClusterAlias); followShardId.writeTo(out); leaderShardId.writeTo(out); - out.writeVLong(maxReadSize); - out.writeVInt(maxConcurrentReads); - out.writeVLong(maxOperationSizeInBytes); - out.writeVInt(maxWriteSize); - out.writeVInt(maxConcurrentWrites); - out.writeVInt(maxBufferSize); + out.writeVLong(maxBatchOperationCount); + out.writeVInt(maxConcurrentReadBatches); + out.writeVLong(maxBatchSizeInBytes); + out.writeVInt(maxConcurrentWriteBatches); + out.writeVInt(maxWriteBufferSize); out.writeTimeValue(retryTimeout); out.writeTimeValue(idleShardRetryDelay); out.writeMap(headers, StreamOutput::writeString, StreamOutput::writeString); @@ -210,12 +200,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(LEADER_SHARD_INDEX_FIELD.getPreferredName(), leaderShardId.getIndex().getName()); builder.field(LEADER_SHARD_INDEX_UUID_FIELD.getPreferredName(), leaderShardId.getIndex().getUUID()); builder.field(LEADER_SHARD_SHARDID_FIELD.getPreferredName(), leaderShardId.id()); - builder.field(MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxReadSize); - builder.field(MAX_CONCURRENT_READS_BATCHES.getPreferredName(), maxConcurrentReads); - builder.field(MAX_BATCH_SIZE_IN_BYTES.getPreferredName(), maxOperationSizeInBytes); - builder.field(MAX_WRITE_SIZE.getPreferredName(), maxWriteSize); - builder.field(MAX_CONCURRENT_WRITES_BATCHES.getPreferredName(), maxConcurrentWrites); - builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxBufferSize); + builder.field(MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); + builder.field(MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); + builder.field(MAX_BATCH_SIZE_IN_BYTES.getPreferredName(), maxBatchSizeInBytes); + builder.field(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize); builder.field(RETRY_TIMEOUT.getPreferredName(), retryTimeout.getStringRep()); builder.field(IDLE_SHARD_RETRY_DELAY.getPreferredName(), idleShardRetryDelay.getStringRep()); builder.field(HEADERS.getPreferredName(), headers); @@ -230,12 +219,11 @@ public boolean equals(Object o) { return Objects.equals(leaderClusterAlias, that.leaderClusterAlias) && Objects.equals(followShardId, that.followShardId) && Objects.equals(leaderShardId, that.leaderShardId) && - maxReadSize == that.maxReadSize && - maxConcurrentReads == that.maxConcurrentReads && - maxWriteSize == that.maxWriteSize && - maxConcurrentWrites == that.maxConcurrentWrites && - maxOperationSizeInBytes == that.maxOperationSizeInBytes && - maxBufferSize == that.maxBufferSize && + maxBatchOperationCount == that.maxBatchOperationCount && + maxConcurrentReadBatches == that.maxConcurrentReadBatches && + maxConcurrentWriteBatches == that.maxConcurrentWriteBatches && + maxBatchSizeInBytes == that.maxBatchSizeInBytes && + maxWriteBufferSize == that.maxWriteBufferSize && Objects.equals(retryTimeout, that.retryTimeout) && Objects.equals(idleShardRetryDelay, that.idleShardRetryDelay) && Objects.equals(headers, that.headers); @@ -243,8 +231,8 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(leaderClusterAlias, followShardId, leaderShardId, maxReadSize, maxConcurrentReads, - maxWriteSize, maxConcurrentWrites, maxOperationSizeInBytes, maxBufferSize, retryTimeout, idleShardRetryDelay, headers); + return Objects.hash(leaderClusterAlias, followShardId, leaderShardId, maxBatchOperationCount, maxConcurrentReadBatches, + maxConcurrentWriteBatches, maxBatchSizeInBytes, maxWriteBufferSize, retryTimeout, idleShardRetryDelay, headers); } public String toString() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 6b3717ce7428c..bcecb55718ed4 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -131,7 +131,7 @@ protected void innerSendShardChangesRequest(long from, int maxOperationCount, Co ShardChangesAction.Request request = new ShardChangesAction.Request(params.getLeaderShardId()); request.setFromSeqNo(from); request.setMaxOperationCount(maxOperationCount); - request.setMaxOperationSizeInBytes(params.getMaxOperationSizeInBytes()); + request.setMaxOperationSizeInBytes(params.getMaxBatchSizeInBytes()); leaderClient.execute(ShardChangesAction.INSTANCE, request, ActionListener.wrap(handler::accept, errorHandler)); } }; 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 ab1974f69e9b5..6804ce8ecb72c 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 @@ -39,35 +39,31 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient } static Request createRequest(RestRequest restRequest) { - int maxOperationCount = ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT; + int maxBatchOperationCount = ShardFollowNodeTask.DEFAULT_MAX_BATCH_OPERATION_COUNT; if (restRequest.hasParam(ShardFollowTask.MAX_BATCH_OPERATION_COUNT.getPreferredName())) { - maxOperationCount = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_BATCH_OPERATION_COUNT.getPreferredName())); + maxBatchOperationCount = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_BATCH_OPERATION_COUNT.getPreferredName())); } - int maxConcurrentReads = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS; - if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_READS_BATCHES.getPreferredName())) { - maxConcurrentReads = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READS_BATCHES.getPreferredName())); + int maxConcurrentReadBatches = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READ_BATCHES; + if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_READ_BATCHES.getPreferredName())) { + maxConcurrentReadBatches = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_READ_BATCHES.getPreferredName())); } - long maxOperationSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES; + long maxBatchSizeInBytes = ShardFollowNodeTask.DEFAULT_MAX_BATCH_SIZE_IN_BYTES; if (restRequest.hasParam(ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES.getPreferredName())) { - maxOperationSizeInBytes = Long.valueOf(restRequest.param(ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES.getPreferredName())); + maxBatchSizeInBytes = Long.valueOf(restRequest.param(ShardFollowTask.MAX_BATCH_SIZE_IN_BYTES.getPreferredName())); } - int maxWriteSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE; - if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())) { - maxWriteSize = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_WRITE_SIZE.getPreferredName())); + int maxConcurrentWriteBatches = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITE_BATCHES; + if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_WRITE_BATCHES.getPreferredName())) { + maxConcurrentWriteBatches = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_WRITE_BATCHES.getPreferredName())); } - int maxConcurrentWrites = ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES; - if (restRequest.hasParam(ShardFollowTask.MAX_CONCURRENT_WRITES_BATCHES.getPreferredName())) { - maxConcurrentWrites = Integer.valueOf(restRequest.param(ShardFollowTask.MAX_CONCURRENT_WRITES_BATCHES.getPreferredName())); - } - int maxBufferSize = ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE; + int maxWriteBufferSize = ShardFollowNodeTask.DEFAULT_MAX_WRITE_BUFFER_SIZE; if (restRequest.hasParam(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName())) { - maxBufferSize = Integer.parseInt(restRequest.param(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName())); + maxWriteBufferSize = Integer.parseInt(restRequest.param(ShardFollowTask.MAX_WRITE_BUFFER_SIZE.getPreferredName())); } TimeValue retryTimeout = restRequest.paramAsTime(ShardFollowTask.RETRY_TIMEOUT.getPreferredName(), ShardFollowNodeTask.DEFAULT_RETRY_TIMEOUT); TimeValue idleShardRetryTimeout = restRequest.paramAsTime(ShardFollowTask.IDLE_SHARD_RETRY_DELAY.getPreferredName(), ShardFollowNodeTask.DEFAULT_IDLE_SHARD_RETRY_DELAY); - return new Request(restRequest.param("leader_index"), restRequest.param("index"), maxOperationCount, maxConcurrentReads, - maxOperationSizeInBytes, maxWriteSize, maxConcurrentWrites, maxBufferSize, retryTimeout, idleShardRetryTimeout); + return new Request(restRequest.param("leader_index"), restRequest.param("index"), maxBatchOperationCount, maxConcurrentReadBatches, + maxBatchSizeInBytes, maxConcurrentWriteBatches, maxWriteBufferSize, retryTimeout, idleShardRetryTimeout); } } 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 155f10557a654..63620c9b69b86 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 @@ -289,7 +289,7 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) atLeastDocsIndexed("index1", numDocsIndexed / 3); final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", maxReadSize, - randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(32, 2048), randomIntBetween(2, 10), + randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(2, 10), randomIntBetween(1024, 10240), TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); @@ -330,12 +330,12 @@ public void testFollowIndexAndCloseNode() throws Exception { thread.start(); final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", randomIntBetween(32, 2048), - randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(32, 2048), randomIntBetween(2, 10), - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); + randomIntBetween(2, 10), Long.MAX_VALUE, randomIntBetween(2, 10), + ShardFollowNodeTask.DEFAULT_MAX_WRITE_BUFFER_SIZE, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); client().execute(FollowIndexAction.INSTANCE, followRequest).get(); - long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxOperationCount(), - followRequest.getMaxOperationCount() * 10)); + long maxNumDocsReplicated = Math.min(3000, randomLongBetween(followRequest.getMaxBatchOperationCount(), + followRequest.getMaxBatchOperationCount() * 10)); long minNumDocsReplicated = maxNumDocsReplicated / 3L; logger.info("waiting for at least [{}] documents to be indexed and then stop a random data node", minNumDocsReplicated); awaitBusy(() -> { @@ -440,7 +440,7 @@ public void testFollowIndex_lowMaxTranslogBytes() throws Exception { } final FollowIndexAction.Request followRequest = new FollowIndexAction.Request("index1", "index2", 1024, 1, 1024, - 1024, 1, 10240, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); + 1, 10240, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10)); final CreateAndFollowIndexAction.Request createAndFollowRequest = new CreateAndFollowIndexAction.Request(followRequest); client().execute(CreateAndFollowIndexAction.INSTANCE, createAndFollowRequest).get(); @@ -643,10 +643,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_OPERATION_COUNT, - ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, - ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, TimeValue.timeValueMillis(10), - TimeValue.timeValueMillis(10)); + 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, + TimeValue.timeValueMillis(10), TimeValue.timeValueMillis(10)); } } 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 462f80461df43..33abc62dce69d 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 @@ -23,7 +23,7 @@ protected FollowIndexAction.Request createTestInstance() { static FollowIndexAction.Request createTestRequest() { return new FollowIndexAction.Request(randomAlphaOfLength(4), randomAlphaOfLength(4), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), randomNonNegativeLong(), randomIntBetween(1, Integer.MAX_VALUE), - randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), - randomBoolean() ? TimeValue.timeValueMillis(500) : null, randomBoolean() ? TimeValue.timeValueMillis(500) : null); + randomIntBetween(1, Integer.MAX_VALUE), randomBoolean() ? TimeValue.timeValueMillis(500) : null, + randomBoolean() ? TimeValue.timeValueMillis(500) : null); } } 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 c54366801849c..7dc4845d29442 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 @@ -50,9 +50,9 @@ public class ShardFollowNodeTaskTests extends ESTestCase { public void testDefaults() throws Exception { long followGlobalCheckpoint = randomIntBetween(-1, 2048); - task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READS, - ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITES, 10000, - ShardFollowNodeTask.DEFAULT_MAX_BUFFER_SIZE, followGlobalCheckpoint); + task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_BATCH_OPERATION_COUNT, + ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_READ_BATCHES, ShardFollowNodeTask.DEFAULT_MAX_CONCURRENT_WRITE_BATCHES, + 10000, ShardFollowNodeTask.DEFAULT_MAX_WRITE_BUFFER_SIZE, followGlobalCheckpoint); task.start(followGlobalCheckpoint); assertBusy(() -> { @@ -63,8 +63,7 @@ public void testDefaults() throws Exception { public void testHitBufferLimit() throws Exception { // Setting buffer limit to 100, so that we are sure the limit will be met - task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_OPERATION_COUNT, 3, - ShardFollowNodeTask.DEFAULT_MAX_WRITE_SIZE, 1, 10000, 100, -1); + task = createShardFollowTask(ShardFollowNodeTask.DEFAULT_MAX_BATCH_OPERATION_COUNT, 3, 1, 10000, 100, -1); task.start(-1); assertBusy(() -> { @@ -75,7 +74,7 @@ public void testHitBufferLimit() throws Exception { @TestLogging("org.elasticsearch.xpack.ccr.action:TRACE") public void testConcurrentReadsAndWrites() throws Exception { long followGlobalCheckpoint = randomIntBetween(-1, 2048); - task = createShardFollowTask(randomIntBetween(32, 2048), randomIntBetween(2, 10), randomIntBetween(32, 2048), + task = createShardFollowTask(randomIntBetween(32, 2048), randomIntBetween(2, 10), randomIntBetween(2, 10), 50000, 10240, followGlobalCheckpoint); task.start(followGlobalCheckpoint); @@ -85,7 +84,7 @@ public void testConcurrentReadsAndWrites() throws Exception { } public void testMappingUpdate() throws Exception { - task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024, -1); + task = createShardFollowTask(1024, 1, 1, 10000, 1024, -1); task.start(-1); assertBusy(() -> { @@ -99,7 +98,7 @@ public void testMappingUpdate() throws Exception { } public void testOccasionalApiFailure() throws Exception { - task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024, -1); + task = createShardFollowTask(1024, 1, 1, 10000, 1024, -1); task.start(-1); randomlyFailWithRetryableError.set(true); assertBusy(() -> { @@ -109,7 +108,7 @@ public void testOccasionalApiFailure() throws Exception { } public void testNotAllExpectedOpsReturned() throws Exception { - task = createShardFollowTask(1024, 1, 1024, 1, 10000, 1024, -1); + task = createShardFollowTask(1024, 1, 1, 10000, 1024, -1); task.start(-1); randomlyTruncateRequests.set(true); assertBusy(() -> { @@ -118,9 +117,8 @@ public void testNotAllExpectedOpsReturned() throws Exception { assertThat(truncatedRequests.get(), greaterThan(0)); } - ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentReads, int maxWriteSize, - int maxConcurrentWrites, int leaderGlobalCheckpoint, int bufferLimit, - long followGlobalCheckpoint) { + ShardFollowNodeTask createShardFollowTask(int maxBatchOperationCount, int maxConcurrentReadBatches, int maxConcurrentWriteBathces, + int leaderGlobalCheckpoint, int bufferWriteLimit, long followGlobalCheckpoint) { imdVersion = new AtomicLong(1L); mappingUpdateCounter = new AtomicInteger(0); randomlyTruncateRequests = new AtomicBoolean(false); @@ -129,9 +127,9 @@ ShardFollowNodeTask createShardFollowTask(int maxReadSize, int maxConcurrentRead failedRequests = new AtomicInteger(0); AtomicBoolean stopped = new AtomicBoolean(false); ShardFollowTask params = new ShardFollowTask(null, new ShardId("follow_index", "", 0), - new ShardId("leader_index", "", 0), maxReadSize, maxConcurrentReads, ShardFollowNodeTask.DEFAULT_MAX_OPERATIONS_SIZE_IN_BYTES, - maxWriteSize, maxConcurrentWrites, bufferLimit, TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10), - Collections.emptyMap()); + new ShardId("leader_index", "", 0), maxBatchOperationCount, maxConcurrentReadBatches, + ShardFollowNodeTask.DEFAULT_MAX_BATCH_SIZE_IN_BYTES, maxConcurrentWriteBathces, bufferWriteLimit, + TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(10), Collections.emptyMap()); BiConsumer scheduler = (delay, task) -> { try { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java index dd4d2bb4ed834..300794a6c00cf 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskTests.java @@ -32,7 +32,6 @@ protected ShardFollowTask createTestInstance() { randomNonNegativeLong(), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), - randomIntBetween(1, Integer.MAX_VALUE), TimeValue.parseTimeValue(randomTimeValue(), ""), TimeValue.parseTimeValue(randomTimeValue(), ""), randomBoolean() ? null : Collections.singletonMap("key", "value")); From 813498ae3557d5f41a9200809faad9a999ce4fb0 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 09:12:35 +0200 Subject: [PATCH 30/41] also apply maxBatchSizeInBytes on write side --- .../elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 633c4776858ae..cc5a7d7a6aed6 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 @@ -137,9 +137,15 @@ private boolean hasReadBudget() { private synchronized void coordinateWrites() { while (hasWriteBudget() && buffer.isEmpty() == false) { + long sumEstimatedSize = 0L; Translog.Operation[] ops = new Translog.Operation[Math.min(params.getMaxBatchOperationCount(), buffer.size())]; for (int i = 0; i < ops.length; i++) { ops[i] = buffer.remove(); + sumEstimatedSize += ops[i].estimateSize(); + if (sumEstimatedSize > params.getMaxBatchSizeInBytes()) { + ops = Arrays.copyOf(ops, i + 1); + break; + } } numConcurrentWrites++; LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops[0].seqNo(), From cb03c9abd4a27350b0d92ff0ed0d13f09502f91a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 11:50:02 +0200 Subject: [PATCH 31/41] iter --- .../xpack/ccr/action/FollowIndexAction.java | 16 ++++------------ .../xpack/ccr/action/ShardFollowNodeTask.java | 14 +++++++------- .../ccr/action/FollowIndexRequestTests.java | 3 +-- 3 files changed, 12 insertions(+), 21 deletions(-) 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 index af9847a8e7b17..9d4ea57b4a4f2 100644 --- 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 @@ -106,8 +106,8 @@ public Request(String leaderIndex, String followIndex, int maxBatchOperationCoun this.maxOperationSizeInBytes = maxOperationSizeInBytes; this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; this.maxWriteBufferSize = maxWriteBufferSize; - this.retryTimeout = retryTimeout; - this.idleShardRetryDelay = idleShardRetryDelay; + this.retryTimeout = Objects.requireNonNull(retryTimeout); + this.idleShardRetryDelay = Objects.requireNonNull(idleShardRetryDelay); } Request() { @@ -272,21 +272,13 @@ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMe .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; - TimeValue retryTimeout = ShardFollowNodeTask.DEFAULT_RETRY_TIMEOUT; - if (request.retryTimeout != null) { - retryTimeout = request.retryTimeout; - } - TimeValue idleShardRetryDelay = ShardFollowNodeTask.DEFAULT_IDLE_SHARD_RETRY_DELAY; - if (request.idleShardRetryDelay != null) { - idleShardRetryDelay = request.idleShardRetryDelay; - } 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, retryTimeout, idleShardRetryDelay, - filteredHeaders); + request.maxConcurrentWriteBatches, request.maxWriteBufferSize, request.retryTimeout, + request.idleShardRetryDelay, filteredHeaders); persistentTasksService.sendStartRequest(taskId, ShardFollowTask.NAME, shardFollowTask, new ActionListener>() { @Override 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 cc5a7d7a6aed6..1d3d5ee471634 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 @@ -78,17 +78,17 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { this.idleShardChangesRequestDelay = idleShardChangesRequestDelay; } - void start(long followGlobalCheckpoint) { - this.lastRequestedSeqno = followGlobalCheckpoint; - this.processedGlobalCheckpoint = followGlobalCheckpoint; - this.globalCheckpoint = followGlobalCheckpoint; + void start(long followerGlobalCheckpoint) { + this.lastRequestedSeqno = followerGlobalCheckpoint; + this.processedGlobalCheckpoint = followerGlobalCheckpoint; + this.globalCheckpoint = followerGlobalCheckpoint; // Forcefully updates follower mapping, this gets us the leader imd version and // makes sure that leader and follower mapping are identical. updateMapping(imdVersion -> { currentIndexMetadataVersion = imdVersion; LOGGER.info("{} Started to follow leader shard {}, followGlobalCheckPoint={}, indexMetaDataVersion={}", - params.getFollowShardId(), params.getLeaderShardId(), followGlobalCheckpoint, imdVersion); + params.getFollowShardId(), params.getLeaderShardId(), followerGlobalCheckpoint, imdVersion); coordinateReads(); }); } @@ -105,8 +105,8 @@ private synchronized void coordinateReads() { while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { numConcurrentReads++; long from = lastRequestedSeqno + 1; - LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, from, maxReadSize); long maxRequiredSeqno = Math.min(globalCheckpoint, from + maxReadSize); + LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, maxRequiredSeqno, maxReadSize); sendShardChangesRequest(from, maxReadSize, maxRequiredSeqno); lastRequestedSeqno = maxRequiredSeqno; } @@ -117,7 +117,7 @@ private synchronized void coordinateReads() { numConcurrentReads++; long from = lastRequestedSeqno + 1; LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, maxReadSize, from); + sendShardChangesRequest(from, maxReadSize, lastRequestedSeqno); } } 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 33abc62dce69d..ac5bc9b74626e 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 @@ -23,7 +23,6 @@ protected FollowIndexAction.Request createTestInstance() { static FollowIndexAction.Request createTestRequest() { return new FollowIndexAction.Request(randomAlphaOfLength(4), randomAlphaOfLength(4), randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), randomNonNegativeLong(), randomIntBetween(1, Integer.MAX_VALUE), - randomIntBetween(1, Integer.MAX_VALUE), randomBoolean() ? TimeValue.timeValueMillis(500) : null, - randomBoolean() ? TimeValue.timeValueMillis(500) : null); + randomIntBetween(1, Integer.MAX_VALUE), TimeValue.timeValueMillis(500), TimeValue.timeValueMillis(500)); } } From 7682b8c9fa281936100aa599e2297b4463757bed Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 12:14:22 +0200 Subject: [PATCH 32/41] iter2 --- .../xpack/ccr/action/ShardFollowNodeTask.java | 68 +++++++++---------- .../ccr/action/ShardFollowNodeTaskTests.java | 12 ++-- 2 files changed, 39 insertions(+), 41 deletions(-) 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 1d3d5ee471634..c207ede5ebdae 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 @@ -169,51 +169,51 @@ private void sendShardChangesRequest(long from, int maxOperationCount, long maxR innerSendShardChangesRequest(from, maxOperationCount, response -> { retryCounter.set(0); - handleReadResponse(from, maxOperationCount, maxRequiredSeqNo, response); + handleReadResponse(from, maxRequiredSeqNo, response); }, e -> handleFailure(e, () -> sendShardChangesRequest(from, maxOperationCount, maxRequiredSeqNo))); } - private void handleReadResponse(long from, int maxOperationCount, long maxRequiredSeqNo, ShardChangesAction.Response response) { + private void handleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { globalCheckpoint = Math.max(globalCheckpoint, response.getGlobalCheckpoint()); - buffer.addAll(Arrays.asList(response.getOperations())); - coordinateWrites(); - - Long lastOpSeqNo = null; - if (response.getOperations().length != 0) { - lastOpSeqNo = response.getOperations()[response.getOperations().length - 1].seqNo(); - assert lastOpSeqNo == Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::seqNo).max().getAsLong(); + final long newMinRequiredSeqNo; + if (response.getOperations().length == 0) { + newMinRequiredSeqNo = from; + } else { + assert response.getOperations()[0].seqNo() == from : + "first operation is not what we asked for. From is [" + from + "], got " + response.getOperations()[0]; + buffer.addAll(Arrays.asList(response.getOperations())); + final long maxSeqNo = response.getOperations()[response.getOperations().length - 1].seqNo(); + assert maxSeqNo== + Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::seqNo).max().getAsLong(); + newMinRequiredSeqNo = maxSeqNo + 1; + // update last requested seq no as we may have gotten more than we asked for and we don't want to ask it again. + lastRequestedSeqno = Math.max(lastRequestedSeqno, maxSeqNo); + assert lastRequestedSeqno <= globalCheckpoint: + "lastRequestedSeqno [" + lastRequestedSeqno + "] is larger than the global checkpoint [" + globalCheckpoint + "]"; + coordinateWrites(); } - if (lastOpSeqNo != null && lastOpSeqNo < maxRequiredSeqNo) { - long newFrom = lastOpSeqNo + 1; - int newSize = (int) (maxRequiredSeqNo - lastOpSeqNo); - LOGGER.trace("{} received [{}] as last op while [{}] was expected, continue to read [{}/{}]...", - params.getFollowShardId(), lastOpSeqNo, maxRequiredSeqNo, newFrom, maxOperationCount); - sendShardChangesRequest(newFrom, newSize, maxRequiredSeqNo); - return; - } - // If lastOpSeqNo == null then from must be equal to maxRequiredSeqNo otherwise something went wrong and - // we are not getting ops while we really should. - // There is one exception why can't fail because of this and that is that we may read from a leader shard - // copy with a global checkpoint equal to from and no ops are than being returned. If we can detect this - // here or make sure that the shard changes api does not do this (add a flag requireOps and let it fail - // so that it can read from another shard copy) then we can add a hard check here. - - numConcurrentReads--; - if (response.getOperations().length != 0) { - LOGGER.trace("{} post updating lastRequestedSeqno to [{}]", params.getFollowShardId(), lastRequestedSeqno); - Translog.Operation firstOp = response.getOperations()[0]; - lastRequestedSeqno = Math.max(lastRequestedSeqno, lastOpSeqNo); - assert firstOp.seqNo() == from; - coordinateReads(); + if (newMinRequiredSeqNo < maxRequiredSeqNo) { + int newSize = (int) (maxRequiredSeqNo - newMinRequiredSeqNo) + 1; + LOGGER.trace("{} received [{}] ops, still missing [{}/{}], continuing to read...", + params.getFollowShardId(), response.getOperations().length, newMinRequiredSeqNo, maxRequiredSeqNo); + sendShardChangesRequest(newMinRequiredSeqNo, newSize, maxRequiredSeqNo); } else { - LOGGER.trace("{} received no ops, scheduling to coordinate reads", params.getFollowShardId()); - scheduler.accept(idleShardChangesRequestDelay, this::coordinateReads); + // read is completed, decrement + numConcurrentReads--; + if (response.getOperations().length == 0 && globalCheckpoint == lastRequestedSeqno) { + // we got nothing and we have no reason to believe asking again well get us more, treat shard as idle and delay + // future requests + LOGGER.trace("{} received no ops and no known ops to fetch, scheduling to coordinate reads", + params.getFollowShardId()); + scheduler.accept(idleShardChangesRequestDelay, this::coordinateReads); + } else { + coordinateReads(); + } } - assert numConcurrentReads >= 0; } }); } 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 7dc4845d29442..68b73f0d6a5fb 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 @@ -193,13 +193,11 @@ protected void innerSendShardChangesRequest(long from, int maxOperationCount, Co maxOperationCount = maxOperationCount / 2; } List ops = new ArrayList<>(); - for (int i = 0; i < maxOperationCount; i++) { - long seqNo = from + i; - if (seqNo <= leaderGlobalCheckpoint) { - String id = UUIDs.randomBase64UUID(); - byte[] source = "{}".getBytes(StandardCharsets.UTF_8); - ops.add(new Translog.Index("doc", id, seqNo, 0, source)); - } + long maxSeqNo = Math.min(from + maxOperationCount, leaderGlobalCheckpoint); + for (long seqNo = from; seqNo <= maxSeqNo; seqNo++) { + String id = UUIDs.randomBase64UUID(); + byte[] source = "{}".getBytes(StandardCharsets.UTF_8); + ops.add(new Translog.Index("doc", id, seqNo, 0, source)); } response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, ops.toArray(new Translog.Operation[0])); From 80c895dac80f29b0f81eebdc89ebd19a59a00d5a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 12:22:09 +0200 Subject: [PATCH 33/41] easier to understand CcrWritePrimaryResult logic --- .../bulk/TransportBulkShardOperationsAction.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index dd0dde188239a..f5dc28168875c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.ccr.action.bulk; import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; @@ -141,11 +142,11 @@ static class CcrWritePrimaryResult extends WritePrimaryResult listener) { + // Return a fresh global checkpoint after the operations have been replicated for the shard follow task: + BulkShardOperationsResponse response = finalResponseIfSuccessful; + response.setGlobalCheckpoint(primary.getGlobalCheckpoint()); + listener.onResponse(response); } } From e6c5422e11169d48a500bd0a4aa654ee75be54bb Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 13:52:44 +0200 Subject: [PATCH 34/41] added some more stats --- .../xpack/ccr/action/ShardFollowNodeTask.java | 44 ++++++++++++++++--- .../ShardFollowNodeTaskStatusTests.java | 4 +- 2 files changed, 39 insertions(+), 9 deletions(-) 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 c207ede5ebdae..b334c8aa0d615 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 @@ -292,42 +292,61 @@ protected boolean isStopped() { @Override public Status getStatus() { - return new Status(processedGlobalCheckpoint, numConcurrentReads, numConcurrentWrites); + return new Status(globalCheckpoint, lastRequestedSeqno, processedGlobalCheckpoint, numConcurrentReads, numConcurrentWrites); } public static class Status implements Task.Status { public static final String NAME = "shard-follow-node-task-status"; - static final ParseField PROCESSED_GLOBAL_CHECKPOINT_FIELD = new ParseField("processed_global_checkpoint"); + static final ParseField GLOBAL_CHECKPOINT_FIELD = new ParseField("leader_global_checkpoint"); + static final ParseField PROCESSED_GLOBAL_CHECKPOINT_FIELD = new ParseField("follower_global_checkpoint"); + static final ParseField LAST_REQUESTED_SEQNO_FIELD = new ParseField("last_requested_seqno"); 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 ConstructingObjectParser PARSER = - new ConstructingObjectParser<>(NAME, args -> new Status((long) args[0], (int) args[1], (int) args[2])); + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, + args -> new Status((long) args[0], (long) args[1], (long) args[2], (int) args[3], (int) args[4])); static { + PARSER.declareLong(ConstructingObjectParser.constructorArg(), GLOBAL_CHECKPOINT_FIELD); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), LAST_REQUESTED_SEQNO_FIELD); PARSER.declareLong(ConstructingObjectParser.constructorArg(), PROCESSED_GLOBAL_CHECKPOINT_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_READS_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_WRITES_FIELD); } + private final long globalCheckpoint; + private final long lastRequestedSeqno; private final long processedGlobalCheckpoint; private final int numberOfConcurrentReads; private final int numberOfConcurrentWrites; - Status(long processedGlobalCheckpoint, int numberOfConcurrentReads, int numberOfConcurrentWrites) { + Status(long globalCheckpoint, long lastRequestedSeqno, long processedGlobalCheckpoint, int numberOfConcurrentReads, + int numberOfConcurrentWrites) { + this.globalCheckpoint = globalCheckpoint; + this.lastRequestedSeqno = lastRequestedSeqno; this.processedGlobalCheckpoint = processedGlobalCheckpoint; this.numberOfConcurrentReads = numberOfConcurrentReads; this.numberOfConcurrentWrites = numberOfConcurrentWrites; } public Status(StreamInput in) throws IOException { + this.globalCheckpoint = in.readZLong(); + this.lastRequestedSeqno = in.readZLong(); this.processedGlobalCheckpoint = in.readZLong(); this.numberOfConcurrentReads = in.readVInt(); this.numberOfConcurrentWrites = in.readVInt(); } + public long getGlobalCheckpoint() { + return globalCheckpoint; + } + + public long getLastRequestedSeqno() { + return lastRequestedSeqno; + } + public long getProcessedGlobalCheckpoint() { return processedGlobalCheckpoint; } @@ -347,6 +366,8 @@ public String getWriteableName() { @Override public void writeTo(StreamOutput out) throws IOException { + out.writeZLong(globalCheckpoint); + out.writeZLong(lastRequestedSeqno); out.writeZLong(processedGlobalCheckpoint); out.writeVInt(numberOfConcurrentReads); out.writeVInt(numberOfConcurrentWrites); @@ -355,9 +376,15 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); + { + builder.field(GLOBAL_CHECKPOINT_FIELD.getPreferredName(), globalCheckpoint); + } { builder.field(PROCESSED_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), processedGlobalCheckpoint); } + { + builder.field(LAST_REQUESTED_SEQNO_FIELD.getPreferredName(), lastRequestedSeqno); + } { builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads); } @@ -377,14 +404,17 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Status status = (Status) o; - return processedGlobalCheckpoint == status.processedGlobalCheckpoint && + return globalCheckpoint == status.globalCheckpoint && + lastRequestedSeqno == status.lastRequestedSeqno && + processedGlobalCheckpoint == status.processedGlobalCheckpoint && numberOfConcurrentReads == status.numberOfConcurrentReads && numberOfConcurrentWrites == status.numberOfConcurrentWrites; } @Override public int hashCode() { - return Objects.hash(processedGlobalCheckpoint, numberOfConcurrentReads, numberOfConcurrentWrites); + return Objects.hash(globalCheckpoint, lastRequestedSeqno, processedGlobalCheckpoint, numberOfConcurrentReads, + numberOfConcurrentWrites); } public String toString() { 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 a1debf6aef917..b550679ebee36 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 @@ -20,8 +20,8 @@ protected ShardFollowNodeTask.Status doParseInstance(XContentParser parser) thro @Override protected ShardFollowNodeTask.Status createTestInstance() { - return new ShardFollowNodeTask.Status(randomNonNegativeLong(), randomIntBetween(0, Integer.MAX_VALUE), - randomIntBetween(0, Integer.MAX_VALUE)); + return new ShardFollowNodeTask.Status(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), + randomIntBetween(0, Integer.MAX_VALUE), randomIntBetween(0, Integer.MAX_VALUE)); } @Override From efee1aff2a86c876e6c1fb1f671a93b0ce757efe Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 14:16:49 +0200 Subject: [PATCH 35/41] changed ops in bulk shard ops request from array to list --- .../xpack/ccr/action/ShardFollowNodeTask.java | 21 +++++++++++-------- .../ccr/action/ShardFollowTasksExecutor.java | 3 ++- .../bulk/BulkShardOperationsRequest.java | 17 ++++++++------- .../TransportBulkShardOperationsAction.java | 13 ++++++------ .../ccr/action/ShardFollowNodeTaskTests.java | 2 +- .../action/bulk/BulkShardOperationsTests.java | 17 ++++++++------- 6 files changed, 42 insertions(+), 31 deletions(-) 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 b334c8aa0d615..088cd6cd738aa 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 @@ -25,8 +25,10 @@ import org.elasticsearch.tasks.TaskId; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.PriorityQueue; @@ -138,18 +140,19 @@ private boolean hasReadBudget() { private synchronized void coordinateWrites() { while (hasWriteBudget() && buffer.isEmpty() == false) { long sumEstimatedSize = 0L; - Translog.Operation[] ops = new Translog.Operation[Math.min(params.getMaxBatchOperationCount(), buffer.size())]; - for (int i = 0; i < ops.length; i++) { - ops[i] = buffer.remove(); - sumEstimatedSize += ops[i].estimateSize(); + int length = Math.min(params.getMaxBatchOperationCount(), buffer.size()); + List ops = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + Translog.Operation op = buffer.remove(); + ops.add(op); + sumEstimatedSize += op.estimateSize(); if (sumEstimatedSize > params.getMaxBatchSizeInBytes()) { - ops = Arrays.copyOf(ops, i + 1); break; } } numConcurrentWrites++; - LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops[0].seqNo(), - ops[ops.length - 1].seqNo(), ops.length); + LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops.get(0).seqNo(), + ops.get(ops.size() - 1).seqNo(), ops.size()); sendBulkShardOperationsRequest(ops); } } @@ -218,7 +221,7 @@ private void handleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAc }); } - private void sendBulkShardOperationsRequest(Translog.Operation[] operations) { + private void sendBulkShardOperationsRequest(List operations) { innerSendBulkShardOperationsRequest(operations, followerLocalCheckpoint -> { retryCounter.set(0); @@ -275,7 +278,7 @@ private boolean shouldRetry(Exception e) { // These methods are protected for testing purposes: protected abstract void updateMapping(LongConsumer handler); - protected abstract void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, LongConsumer handler, + protected abstract void innerSendBulkShardOperationsRequest(List operations, LongConsumer handler, Consumer errorHandler); protected abstract void innerSendShardChangesRequest(long from, int maxOperationCount, Consumer handler, diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index bcecb55718ed4..969e9e8bf9020 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -39,6 +39,7 @@ import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.function.BiConsumer; @@ -118,7 +119,7 @@ protected void updateMapping(LongConsumer handler) { } @Override - protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, LongConsumer handler, + protected void innerSendBulkShardOperationsRequest(List operations, LongConsumer handler, Consumer errorHandler) { final BulkShardOperationsRequest request = new BulkShardOperationsRequest(params.getFollowShardId(), operations); followerClient.execute(BulkShardOperationsAction.INSTANCE, request, diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsRequest.java index ef9d27ef919db..c28789fb580a8 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsRequest.java @@ -12,41 +12,44 @@ import org.elasticsearch.index.translog.Translog; import java.io.IOException; +import java.util.List; public final class BulkShardOperationsRequest extends ReplicatedWriteRequest { - private Translog.Operation[] operations; + private List operations; public BulkShardOperationsRequest() { - } - public BulkShardOperationsRequest(final ShardId shardId, final Translog.Operation[] operations) { + public BulkShardOperationsRequest(final ShardId shardId, final List operations) { super(shardId); setRefreshPolicy(RefreshPolicy.NONE); this.operations = operations; } - public Translog.Operation[] getOperations() { + public List getOperations() { return operations; } @Override public void readFrom(final StreamInput in) throws IOException { super.readFrom(in); - operations = in.readArray(Translog.Operation::readOperation, Translog.Operation[]::new); + operations = in.readList(Translog.Operation::readOperation); } @Override public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); - out.writeArray(Translog.Operation::writeOperation, operations); + out.writeVInt(operations.size()); + for (Translog.Operation operation : operations) { + Translog.Operation.writeOperation(out, operation); + } } @Override public String toString() { return "BulkShardOperationsRequest{" + - "operations=" + operations.length+ + "operations=" + operations.size()+ ", shardId=" + shardId + ", timeout=" + timeout + ", index='" + index + '\'' + diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index f5dc28168875c..0770c8713562e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -25,7 +25,8 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; public class TransportBulkShardOperationsAction extends TransportWriteAction { @@ -63,10 +64,10 @@ protected WritePrimaryResult shardOperationOnPrimary( final ShardId shardId, - final Translog.Operation[] sourceOperations, + final List sourceOperations, final IndexShard primary, final Logger logger) throws IOException { - final Translog.Operation[] targetOperations = Arrays.stream(sourceOperations).map(operation -> { + final List targetOperations = sourceOperations.stream().map(operation -> { final Translog.Operation operationWithPrimaryTerm; switch (operation.opType()) { case INDEX: @@ -101,7 +102,7 @@ static WritePrimaryResult shardOperationOnReplica } private static Translog.Location applyTranslogOperations( - final Translog.Operation[] operations, final IndexShard shard, final Engine.Operation.Origin origin) throws IOException { + final List operations, final IndexShard shard, final Engine.Operation.Origin origin) throws IOException { Translog.Location location = null; for (final Translog.Operation operation : operations) { final Engine.Result result = shard.applyTranslogOperation(operation, origin); @@ -123,7 +124,7 @@ private static Translog.Location applyTranslogOperations( assert result.getResultType() == Engine.Result.Type.SUCCESS; location = locationToSync(location, result.getTranslogLocation()); } - assert operations.length == 0 || location != null; + assert operations.size() == 0 || location != null; return location; } 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 68b73f0d6a5fb..61fd0c4ca23da 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 @@ -153,7 +153,7 @@ protected void updateMapping(LongConsumer handler) { } @Override - protected void innerSendBulkShardOperationsRequest(Translog.Operation[] operations, LongConsumer handler, + protected void innerSendBulkShardOperationsRequest(List operations, LongConsumer handler, Consumer errorHandler) { if (randomlyFailWithRetryableError.get() && readCounter.incrementAndGet() % 5 == 0) { failedRequests.incrementAndGet(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java index e837da2b65deb..2b669e9154afb 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java @@ -19,6 +19,8 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; import static org.hamcrest.Matchers.equalTo; @@ -34,22 +36,23 @@ public void testPrimaryTermFromFollower() throws IOException { // we use this primary on the operations yet we expect the applied operations to have the primary term of the follower final long primaryTerm = randomLongBetween(1, Integer.MAX_VALUE); - final Translog.Operation[] operations = new Translog.Operation[randomIntBetween(0, 127)]; - for (int i = 0; i < operations.length; i++) { + int numOps = randomIntBetween(0, 127); + final List operations = new ArrayList<>(randomIntBetween(0, 127)); + for (int i = 0; i < numOps; i++) { final String id = Integer.toString(i); final long seqNo = i; final Translog.Operation.Type type = randomValueOtherThan(Translog.Operation.Type.CREATE, () -> randomFrom(Translog.Operation.Type.values())); switch (type) { case INDEX: - operations[i] = new Translog.Index("_doc", id, seqNo, primaryTerm, 0, VersionType.INTERNAL, SOURCE, null, -1); + operations.add(new Translog.Index("_doc", id, seqNo, primaryTerm, 0, VersionType.INTERNAL, SOURCE, null, -1)); break; case DELETE: - operations[i] = - new Translog.Delete("_doc", id, new Term("_id", Uid.encodeId(id)), seqNo, primaryTerm, 0, VersionType.INTERNAL); + operations.add( + new Translog.Delete("_doc", id, new Term("_id", Uid.encodeId(id)), seqNo, primaryTerm, 0, VersionType.INTERNAL)); break; case NO_OP: - operations[i] = new Translog.NoOp(seqNo, primaryTerm, "test"); + operations.add(new Translog.NoOp(seqNo, primaryTerm, "test")); break; default: throw new IllegalStateException("unexpected operation type [" + type + "]"); @@ -60,7 +63,7 @@ public void testPrimaryTermFromFollower() throws IOException { TransportBulkShardOperationsAction.shardOperationOnPrimary(followerPrimary.shardId(), operations, followerPrimary, logger); try (Translog.Snapshot snapshot = followerPrimary.newTranslogSnapshotFromMinSeqNo(0)) { - assertThat(snapshot.totalOperations(), equalTo(operations.length)); + assertThat(snapshot.totalOperations(), equalTo(operations.size())); Translog.Operation operation; while ((operation = snapshot.next()) != null) { assertThat(operation.primaryTerm(), equalTo(followerPrimary.getPrimaryTerm())); From 2367e065d8c612a3bfe68e5c605383e54d73a66f Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 14:17:40 +0200 Subject: [PATCH 36/41] one set of {} is good enough --- .../xpack/ccr/action/ShardFollowNodeTask.java | 8 -------- 1 file changed, 8 deletions(-) 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 088cd6cd738aa..283fc982d9fe6 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 @@ -381,17 +381,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); { builder.field(GLOBAL_CHECKPOINT_FIELD.getPreferredName(), globalCheckpoint); - } - { builder.field(PROCESSED_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), processedGlobalCheckpoint); - } - { builder.field(LAST_REQUESTED_SEQNO_FIELD.getPreferredName(), lastRequestedSeqno); - } - { builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads); - } - { builder.field(NUMBER_OF_CONCURRENT_WRITES_FIELD.getPreferredName(), numberOfConcurrentWrites); } builder.endObject(); From d9086fef9fee8a8a1947d30809eafc0842b9b7c9 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 9 Jul 2018 15:34:07 +0200 Subject: [PATCH 37/41] only peek read when there is budget --- .../xpack/ccr/action/ShardFollowNodeTask.java | 9 ++++++++- .../ccr/action/ShardFollowNodeTaskTests.java | 15 +++++++++------ 2 files changed, 17 insertions(+), 7 deletions(-) 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 283fc982d9fe6..952983942eb61 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 @@ -113,7 +113,8 @@ private synchronized void coordinateReads() { lastRequestedSeqno = maxRequiredSeqno; } - if (numConcurrentReads == 0) { + if (numConcurrentReads == 0 && hasReadBudget()) { + assert lastRequestedSeqno == globalCheckpoint; // We sneak peek if there is any thing new in the leader. // If there is we will happily accept numConcurrentReads++; @@ -154,6 +155,12 @@ private synchronized void coordinateWrites() { LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops.get(0).seqNo(), ops.get(ops.size() - 1).seqNo(), ops.size()); sendBulkShardOperationsRequest(ops); + + // In case that buffer is higher than max write buffer size then reads may all have been stopped, + // this if check makes sure that we start a read when there is budget in case no reads are being performed. + if (numConcurrentReads == 0 && hasReadBudget()) { + coordinateReads(); + } } } 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 61fd0c4ca23da..9e6dca7b61718 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 @@ -37,6 +37,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase { private ShardFollowNodeTask task; + private AtomicLong leaderGlobalCheckPoint; private AtomicLong imdVersion; private AtomicInteger mappingUpdateCounter; @@ -84,13 +85,14 @@ public void testConcurrentReadsAndWrites() throws Exception { } public void testMappingUpdate() throws Exception { - task = createShardFollowTask(1024, 1, 1, 10000, 1024, -1); + task = createShardFollowTask(1024, 1, 1, 1000, 1024, -1); task.start(-1); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), greaterThanOrEqualTo(1000L)); }); imdVersion.set(2L); + leaderGlobalCheckPoint.set(10000L); assertBusy(() -> { assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); }); @@ -118,7 +120,8 @@ public void testNotAllExpectedOpsReturned() throws Exception { } ShardFollowNodeTask createShardFollowTask(int maxBatchOperationCount, int maxConcurrentReadBatches, int maxConcurrentWriteBathces, - int leaderGlobalCheckpoint, int bufferWriteLimit, long followGlobalCheckpoint) { + int globalCheckpoint, int bufferWriteLimit, long followGlobalCheckpoint) { + leaderGlobalCheckPoint = new AtomicLong(globalCheckpoint); imdVersion = new AtomicLong(1L); mappingUpdateCounter = new AtomicInteger(0); randomlyTruncateRequests = new AtomicBoolean(false); @@ -185,21 +188,21 @@ protected void innerSendShardChangesRequest(long from, int maxOperationCount, Co } ShardChangesAction.Response response; - if (from > leaderGlobalCheckpoint) { - response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, new Translog.Operation[0]); + if (from > leaderGlobalCheckPoint.get()) { + response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckPoint.get(), new Translog.Operation[0]); } else { if (randomlyTruncateRequests.get() && maxOperationCount > 10 && truncatedRequests.get() < 5) { truncatedRequests.incrementAndGet(); maxOperationCount = maxOperationCount / 2; } List ops = new ArrayList<>(); - long maxSeqNo = Math.min(from + maxOperationCount, leaderGlobalCheckpoint); + long maxSeqNo = Math.min(from + maxOperationCount, leaderGlobalCheckPoint.get()); for (long seqNo = from; seqNo <= maxSeqNo; seqNo++) { String id = UUIDs.randomBase64UUID(); byte[] source = "{}".getBytes(StandardCharsets.UTF_8); ops.add(new Translog.Index("doc", id, seqNo, 0, source)); } - response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckpoint, + response = new ShardChangesAction.Response(imdVersion.get(), leaderGlobalCheckPoint.get(), ops.toArray(new Translog.Operation[0])); } // Emulate network thread and avoid SO: From bf251afbf3f20843ccbd3e07e8b684fb88bb2090 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 10 Jul 2018 10:51:20 +0200 Subject: [PATCH 38/41] variable rename --- .../xpack/ccr/action/ShardFollowNodeTask.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 952983942eb61..087d926397551 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 @@ -103,13 +103,13 @@ private synchronized void coordinateReads() { LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, globalCheckpoint={}", params.getFollowShardId(), lastRequestedSeqno, globalCheckpoint); - final int maxReadSize = params.getMaxBatchOperationCount(); + final int maxBatchOperationCount = params.getMaxBatchOperationCount(); while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { numConcurrentReads++; long from = lastRequestedSeqno + 1; - long maxRequiredSeqno = Math.min(globalCheckpoint, from + maxReadSize); - LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, maxRequiredSeqno, maxReadSize); - sendShardChangesRequest(from, maxReadSize, maxRequiredSeqno); + long maxRequiredSeqno = Math.min(globalCheckpoint, from + maxBatchOperationCount); + LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, maxRequiredSeqno, maxBatchOperationCount); + sendShardChangesRequest(from, maxBatchOperationCount, maxRequiredSeqno); lastRequestedSeqno = maxRequiredSeqno; } @@ -120,7 +120,7 @@ private synchronized void coordinateReads() { numConcurrentReads++; long from = lastRequestedSeqno + 1; LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, maxReadSize, lastRequestedSeqno); + sendShardChangesRequest(from, maxBatchOperationCount, lastRequestedSeqno); } } From 775de27a2ce55626755533e925229f89fe69b09a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 10 Jul 2018 12:47:43 +0200 Subject: [PATCH 39/41] call coordinateReads() from handleWriteResponse() --- .../xpack/ccr/action/ShardFollowNodeTask.java | 12 ++++++------ .../xpack/ccr/action/ShardFollowNodeTaskTests.java | 2 -- 2 files changed, 6 insertions(+), 8 deletions(-) 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 087d926397551..c1c24cb12eb14 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 @@ -155,12 +155,6 @@ private synchronized void coordinateWrites() { LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops.get(0).seqNo(), ops.get(ops.size() - 1).seqNo(), ops.size()); sendBulkShardOperationsRequest(ops); - - // In case that buffer is higher than max write buffer size then reads may all have been stopped, - // this if check makes sure that we start a read when there is budget in case no reads are being performed. - if (numConcurrentReads == 0 && hasReadBudget()) { - coordinateReads(); - } } } @@ -243,6 +237,12 @@ private synchronized void handleWriteResponse(long followerLocalCheckpoint) { numConcurrentWrites--; assert numConcurrentWrites >= 0; coordinateWrites(); + + // In case that buffer has more ops than is allowed then reads may all have been stopped, + // this if check makes sure that we start a read when there is budget in case no reads are being performed. + if (numConcurrentReads == 0) { + coordinateReads(); + } } private synchronized void maybeUpdateMapping(Long minimumRequiredIndexMetadataVersion, Runnable task) { 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 9e6dca7b61718..e8d2e703517fd 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 @@ -32,7 +32,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.nullValue; -@TestLogging("org.elasticsearch.xpack.ccr.action:TRACE") public class ShardFollowNodeTaskTests extends ESTestCase { private ShardFollowNodeTask task; @@ -72,7 +71,6 @@ public void testHitBufferLimit() throws Exception { }); } - @TestLogging("org.elasticsearch.xpack.ccr.action:TRACE") public void testConcurrentReadsAndWrites() throws Exception { long followGlobalCheckpoint = randomIntBetween(-1, 2048); task = createShardFollowTask(randomIntBetween(32, 2048), randomIntBetween(2, 10), From cf630470737973d50c3aac826475cc2ed457b84e Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 10 Jul 2018 12:56:58 +0200 Subject: [PATCH 40/41] renamed fields and variables --- .../xpack/ccr/action/ShardFollowNodeTask.java | 76 +++++++++---------- .../xpack/ccr/ShardChangesIT.java | 2 +- .../ccr/action/ShardFollowNodeTaskTests.java | 15 ++-- 3 files changed, 46 insertions(+), 47 deletions(-) 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 c1c24cb12eb14..5684a51a115d3 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 @@ -61,11 +61,11 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { private final BiConsumer scheduler; private volatile long lastRequestedSeqno; - private volatile long globalCheckpoint; + private volatile long leaderGlobalCheckpoint; private volatile int numConcurrentReads = 0; private volatile int numConcurrentWrites = 0; - private volatile long processedGlobalCheckpoint = 0; + private volatile long followerGlobalCheckpoint = 0; private volatile long currentIndexMetadataVersion = 0; private final AtomicInteger retryCounter = new AtomicInteger(0); private final Queue buffer = new PriorityQueue<>(Comparator.comparing(Translog.Operation::seqNo).reversed()); @@ -82,8 +82,8 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { void start(long followerGlobalCheckpoint) { this.lastRequestedSeqno = followerGlobalCheckpoint; - this.processedGlobalCheckpoint = followerGlobalCheckpoint; - this.globalCheckpoint = followerGlobalCheckpoint; + this.followerGlobalCheckpoint = followerGlobalCheckpoint; + this.leaderGlobalCheckpoint = followerGlobalCheckpoint; // Forcefully updates follower mapping, this gets us the leader imd version and // makes sure that leader and follower mapping are identical. @@ -101,20 +101,20 @@ private synchronized void coordinateReads() { return; } - LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, globalCheckpoint={}", - params.getFollowShardId(), lastRequestedSeqno, globalCheckpoint); + LOGGER.trace("{} coordinate reads, lastRequestedSeqno={}, leaderGlobalCheckpoint={}", + params.getFollowShardId(), lastRequestedSeqno, leaderGlobalCheckpoint); final int maxBatchOperationCount = params.getMaxBatchOperationCount(); - while (hasReadBudget() && lastRequestedSeqno < globalCheckpoint) { + while (hasReadBudget() && lastRequestedSeqno < leaderGlobalCheckpoint) { numConcurrentReads++; long from = lastRequestedSeqno + 1; - long maxRequiredSeqno = Math.min(globalCheckpoint, from + maxBatchOperationCount); + long maxRequiredSeqno = Math.min(leaderGlobalCheckpoint, from + maxBatchOperationCount); LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, maxRequiredSeqno, maxBatchOperationCount); sendShardChangesRequest(from, maxBatchOperationCount, maxRequiredSeqno); lastRequestedSeqno = maxRequiredSeqno; } if (numConcurrentReads == 0 && hasReadBudget()) { - assert lastRequestedSeqno == globalCheckpoint; + assert lastRequestedSeqno == leaderGlobalCheckpoint; // We sneak peek if there is any thing new in the leader. // If there is we will happily accept numConcurrentReads++; @@ -181,7 +181,7 @@ private void sendShardChangesRequest(long from, int maxOperationCount, long maxR private void handleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAction.Response response) { maybeUpdateMapping(response.getIndexMetadataVersion(), () -> { synchronized (ShardFollowNodeTask.this) { - globalCheckpoint = Math.max(globalCheckpoint, response.getGlobalCheckpoint()); + leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getGlobalCheckpoint()); final long newMinRequiredSeqNo; if (response.getOperations().length == 0) { newMinRequiredSeqNo = from; @@ -195,8 +195,8 @@ private void handleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAc newMinRequiredSeqNo = maxSeqNo + 1; // update last requested seq no as we may have gotten more than we asked for and we don't want to ask it again. lastRequestedSeqno = Math.max(lastRequestedSeqno, maxSeqNo); - assert lastRequestedSeqno <= globalCheckpoint: - "lastRequestedSeqno [" + lastRequestedSeqno + "] is larger than the global checkpoint [" + globalCheckpoint + "]"; + assert lastRequestedSeqno <= leaderGlobalCheckpoint : "lastRequestedSeqno [" + lastRequestedSeqno + + "] is larger than the global checkpoint [" + leaderGlobalCheckpoint + "]"; coordinateWrites(); } @@ -208,7 +208,7 @@ private void handleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAc } else { // read is completed, decrement numConcurrentReads--; - if (response.getOperations().length == 0 && globalCheckpoint == lastRequestedSeqno) { + if (response.getOperations().length == 0 && leaderGlobalCheckpoint == lastRequestedSeqno) { // we got nothing and we have no reason to believe asking again well get us more, treat shard as idle and delay // future requests LOGGER.trace("{} received no ops and no known ops to fetch, scheduling to coordinate reads", @@ -233,7 +233,7 @@ private void sendBulkShardOperationsRequest(List operations) } private synchronized void handleWriteResponse(long followerLocalCheckpoint) { - processedGlobalCheckpoint = Math.max(processedGlobalCheckpoint, followerLocalCheckpoint); + this.followerGlobalCheckpoint = Math.max(this.followerGlobalCheckpoint, followerLocalCheckpoint); numConcurrentWrites--; assert numConcurrentWrites >= 0; coordinateWrites(); @@ -302,15 +302,15 @@ protected boolean isStopped() { @Override public Status getStatus() { - return new Status(globalCheckpoint, lastRequestedSeqno, processedGlobalCheckpoint, numConcurrentReads, numConcurrentWrites); + return new Status(leaderGlobalCheckpoint, lastRequestedSeqno, followerGlobalCheckpoint, numConcurrentReads, numConcurrentWrites); } public static class Status implements Task.Status { public static final String NAME = "shard-follow-node-task-status"; - static final ParseField GLOBAL_CHECKPOINT_FIELD = new ParseField("leader_global_checkpoint"); - static final ParseField PROCESSED_GLOBAL_CHECKPOINT_FIELD = new ParseField("follower_global_checkpoint"); + static final ParseField LEADER_GLOBAL_CHECKPOINT_FIELD = new ParseField("leader_global_checkpoint"); + static final ParseField FOLLOWER_GLOBAL_CHECKPOINT_FIELD = new ParseField("follower_global_checkpoint"); static final ParseField LAST_REQUESTED_SEQNO_FIELD = new ParseField("last_requested_seqno"); 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"); @@ -319,46 +319,46 @@ public static class Status implements Task.Status { args -> new Status((long) args[0], (long) args[1], (long) args[2], (int) args[3], (int) args[4])); static { - PARSER.declareLong(ConstructingObjectParser.constructorArg(), GLOBAL_CHECKPOINT_FIELD); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), LEADER_GLOBAL_CHECKPOINT_FIELD); PARSER.declareLong(ConstructingObjectParser.constructorArg(), LAST_REQUESTED_SEQNO_FIELD); - PARSER.declareLong(ConstructingObjectParser.constructorArg(), PROCESSED_GLOBAL_CHECKPOINT_FIELD); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_GLOBAL_CHECKPOINT_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_READS_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_WRITES_FIELD); } - private final long globalCheckpoint; + private final long leaderGlobalCheckpoint; private final long lastRequestedSeqno; - private final long processedGlobalCheckpoint; + private final long followerGlobalCheckpoint; private final int numberOfConcurrentReads; private final int numberOfConcurrentWrites; - Status(long globalCheckpoint, long lastRequestedSeqno, long processedGlobalCheckpoint, int numberOfConcurrentReads, - int numberOfConcurrentWrites) { - this.globalCheckpoint = globalCheckpoint; + Status(long leaderGlobalCheckpoint, long lastRequestedSeqno, long followerGlobalCheckpoint, + int numberOfConcurrentReads, int numberOfConcurrentWrites) { + this.leaderGlobalCheckpoint = leaderGlobalCheckpoint; this.lastRequestedSeqno = lastRequestedSeqno; - this.processedGlobalCheckpoint = processedGlobalCheckpoint; + this.followerGlobalCheckpoint = followerGlobalCheckpoint; this.numberOfConcurrentReads = numberOfConcurrentReads; this.numberOfConcurrentWrites = numberOfConcurrentWrites; } public Status(StreamInput in) throws IOException { - this.globalCheckpoint = in.readZLong(); + this.leaderGlobalCheckpoint = in.readZLong(); this.lastRequestedSeqno = in.readZLong(); - this.processedGlobalCheckpoint = in.readZLong(); + this.followerGlobalCheckpoint = in.readZLong(); this.numberOfConcurrentReads = in.readVInt(); this.numberOfConcurrentWrites = in.readVInt(); } - public long getGlobalCheckpoint() { - return globalCheckpoint; + public long getLeaderGlobalCheckpoint() { + return leaderGlobalCheckpoint; } public long getLastRequestedSeqno() { return lastRequestedSeqno; } - public long getProcessedGlobalCheckpoint() { - return processedGlobalCheckpoint; + public long getFollowerGlobalCheckpoint() { + return followerGlobalCheckpoint; } public int getNumberOfConcurrentReads() { @@ -376,9 +376,9 @@ public String getWriteableName() { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeZLong(globalCheckpoint); + out.writeZLong(leaderGlobalCheckpoint); out.writeZLong(lastRequestedSeqno); - out.writeZLong(processedGlobalCheckpoint); + out.writeZLong(followerGlobalCheckpoint); out.writeVInt(numberOfConcurrentReads); out.writeVInt(numberOfConcurrentWrites); } @@ -387,8 +387,8 @@ public void writeTo(StreamOutput out) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); { - builder.field(GLOBAL_CHECKPOINT_FIELD.getPreferredName(), globalCheckpoint); - builder.field(PROCESSED_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), processedGlobalCheckpoint); + builder.field(LEADER_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), leaderGlobalCheckpoint); + builder.field(FOLLOWER_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), followerGlobalCheckpoint); builder.field(LAST_REQUESTED_SEQNO_FIELD.getPreferredName(), lastRequestedSeqno); builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads); builder.field(NUMBER_OF_CONCURRENT_WRITES_FIELD.getPreferredName(), numberOfConcurrentWrites); @@ -406,16 +406,16 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Status status = (Status) o; - return globalCheckpoint == status.globalCheckpoint && + return leaderGlobalCheckpoint == status.leaderGlobalCheckpoint && lastRequestedSeqno == status.lastRequestedSeqno && - processedGlobalCheckpoint == status.processedGlobalCheckpoint && + followerGlobalCheckpoint == status.followerGlobalCheckpoint && numberOfConcurrentReads == status.numberOfConcurrentReads && numberOfConcurrentWrites == status.numberOfConcurrentWrites; } @Override public int hashCode() { - return Objects.hash(globalCheckpoint, lastRequestedSeqno, processedGlobalCheckpoint, numberOfConcurrentReads, + return Objects.hash(leaderGlobalCheckpoint, lastRequestedSeqno, followerGlobalCheckpoint, numberOfConcurrentReads, numberOfConcurrentWrites); } 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 63620c9b69b86..287a46f8eb026 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 @@ -490,7 +490,7 @@ private CheckedRunnable assertTask(final int numberOfPrimaryShards, f ShardFollowNodeTask.Status status = (ShardFollowNodeTask.Status) taskInfo.getStatus(); assertThat(status, notNullValue()); assertThat("incorrect global checkpoint " + shardFollowTaskParams, - status.getProcessedGlobalCheckpoint(), + status.getFollowerGlobalCheckpoint(), equalTo(numDocsPerShard.get(shardFollowTaskParams.getLeaderShardId()))); } }; 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 e8d2e703517fd..30ae633433634 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 @@ -12,7 +12,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.After; import java.nio.charset.StandardCharsets; @@ -56,7 +55,7 @@ public void testDefaults() throws Exception { task.start(followGlobalCheckpoint); assertBusy(() -> { - assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + assertThat(task.getStatus().getFollowerGlobalCheckpoint(), equalTo(10000L)); }); assertThat(mappingUpdateCounter.get(), equalTo(1)); } @@ -67,7 +66,7 @@ public void testHitBufferLimit() throws Exception { task.start(-1); assertBusy(() -> { - assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + assertThat(task.getStatus().getFollowerGlobalCheckpoint(), equalTo(10000L)); }); } @@ -78,7 +77,7 @@ public void testConcurrentReadsAndWrites() throws Exception { task.start(followGlobalCheckpoint); assertBusy(() -> { - assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(50000L)); + assertThat(task.getStatus().getFollowerGlobalCheckpoint(), equalTo(50000L)); }); } @@ -87,12 +86,12 @@ public void testMappingUpdate() throws Exception { task.start(-1); assertBusy(() -> { - assertThat(task.getStatus().getProcessedGlobalCheckpoint(), greaterThanOrEqualTo(1000L)); + assertThat(task.getStatus().getFollowerGlobalCheckpoint(), greaterThanOrEqualTo(1000L)); }); imdVersion.set(2L); leaderGlobalCheckPoint.set(10000L); assertBusy(() -> { - assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + assertThat(task.getStatus().getFollowerGlobalCheckpoint(), equalTo(10000L)); }); assertThat(mappingUpdateCounter.get(), equalTo(2)); } @@ -102,7 +101,7 @@ public void testOccasionalApiFailure() throws Exception { task.start(-1); randomlyFailWithRetryableError.set(true); assertBusy(() -> { - assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + assertThat(task.getStatus().getFollowerGlobalCheckpoint(), equalTo(10000L)); }); assertThat(failedRequests.get(), greaterThan(0)); } @@ -112,7 +111,7 @@ public void testNotAllExpectedOpsReturned() throws Exception { task.start(-1); randomlyTruncateRequests.set(true); assertBusy(() -> { - assertThat(task.getStatus().getProcessedGlobalCheckpoint(), equalTo(10000L)); + assertThat(task.getStatus().getFollowerGlobalCheckpoint(), equalTo(10000L)); }); assertThat(truncatedRequests.get(), greaterThan(0)); } From cb09fd72a3a34337590eea4ba944cda20a2081b2 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 10 Jul 2018 13:06:43 +0200 Subject: [PATCH 41/41] the if check is not needed, calling coordinateReads() and coordinateWrites() is always safe --- .../elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 5684a51a115d3..026bb58d9db2a 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 @@ -239,10 +239,8 @@ private synchronized void handleWriteResponse(long followerLocalCheckpoint) { coordinateWrites(); // In case that buffer has more ops than is allowed then reads may all have been stopped, - // this if check makes sure that we start a read when there is budget in case no reads are being performed. - if (numConcurrentReads == 0) { - coordinateReads(); - } + // this invocation makes sure that we start a read when there is budget in case no reads are being performed. + coordinateReads(); } private synchronized void maybeUpdateMapping(Long minimumRequiredIndexMetadataVersion, Runnable task) {