From 98182728b62749d57a495474aaaed5dd8d52aa0c Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 25 Oct 2018 04:36:15 -0400 Subject: [PATCH] CCR: Rename follow-task parameters and stats (#34836) * CCR: Rename follow parameters and stats This commit renames the follow-task parameters and its stats. Below are the changes: ## Params - remote_cluster (unchanged) - leader_index (unchanged) - max_read_request_operation_count -> max_read_request_operation_count - max_batch_size -> max_read_request_size - max_write_request_operation_count (new) - max_write_request_size (new) - max_concurrent_read_batches -> max_outstanding_read_requests - max_concurrent_write_batches -> max_outstanding_write_requests - max_write_buffer_size (unchanged) - max_write_buffer_count (unchanged) - max_retry_delay (unchanged) - poll_timeout -> read_poll_timeout ## Stats - remote_cluster (unchanged) - leader_index (unchanged) - follower_index (unchanged) - shard_id (unchanged) - leader_global_checkpoint (unchanged) - leader_max_seq_no (unchanged) - follower_global_checkpoint (unchanged) - follower_max_seq_no (unchanged) - last_requested_seq_no (unchanged) - number_of_concurrent_reads -> outstanding_read_requests - number_of_concurrent_writes -> outstanding_write_requests - buffer_size_in_bytes -> write_buffer_size_in_bytes (new) - number_of_queued_writes -> write_buffer_operation_count - mapping_version -> follower_mapping_version - total_fetch_time_millis -> total_read_time_millis - total_fetch_remote_time_millis -> total_read_remote_exec_time_millis - number_of_successful_fetches -> successful_read_requests - number_of_failed_fetches -> failed_read_requests - operation_received -> operations_read - total_transferred_bytes -> bytes_read - total_index_time_millis -> total_write_time_millis [?] - number_of_successful_bulk_operations -> successful_write_requests - number_of_failed_bulk_operations -> failed_write_requests - number_of_operations_indexed -> operations_written - fetch_exception -> read_exceptions - time_since_last_read_millis -> time_since_last_read_millis * add test for max_write_request_(operation_count|size) --- .../rest-api-spec/test/ccr/auto_follow.yml | 6 +- .../rest-api-spec/test/ccr/follow_stats.yml | 30 +- .../xpack/ccr/ESCCRRestTestCase.java | 8 +- .../ccr/action/AutoFollowCoordinator.java | 12 +- .../xpack/ccr/action/ShardChangesAction.java | 4 +- .../xpack/ccr/action/ShardFollowNodeTask.java | 134 ++--- .../xpack/ccr/action/ShardFollowTask.java | 161 +++--- .../ccr/action/ShardFollowTasksExecutor.java | 4 +- .../TransportPutAutoFollowPatternAction.java | 8 +- .../action/TransportResumeFollowAction.java | 72 ++- .../elasticsearch/xpack/CcrIntegTestCase.java | 4 +- .../xpack/CcrSingleNodeTestCase.java | 6 +- .../elasticsearch/xpack/ccr/AutoFollowIT.java | 34 +- .../xpack/ccr/AutoFollowMetadataTests.java | 2 + .../elasticsearch/xpack/ccr/CcrLicenseIT.java | 2 +- .../xpack/ccr/IndexFollowingIT.java | 41 +- .../action/AutoFollowCoordinatorTests.java | 26 +- .../GetAutoFollowPatternResponseTests.java | 2 + .../PutAutoFollowPatternRequestTests.java | 6 +- .../ResumeFollowActionRequestTests.java | 10 +- .../ShardFollowNodeTaskRandomTests.java | 14 +- .../ShardFollowNodeTaskStatusTests.java | 36 +- .../ccr/action/ShardFollowNodeTaskTests.java | 388 +++++++++------ .../ShardFollowTaskReplicationTests.java | 4 +- .../ccr/action/ShardFollowTaskTests.java | 2 + ...ortDeleteAutoFollowPatternActionTests.java | 6 +- ...nsportGetAutoFollowPatternActionTests.java | 4 +- ...nsportPutAutoFollowPatternActionTests.java | 2 +- .../action/TransportUnfollowActionTests.java | 4 +- .../ccr/FollowStatsMonitoringDocTests.java | 92 ++-- .../xpack/core/ccr/AutoFollowMetadata.java | 154 +++--- .../core/ccr/ShardFollowNodeTaskStatus.java | 464 +++++++++--------- .../action/PutAutoFollowPatternAction.java | 126 +++-- .../core/ccr/action/PutFollowAction.java | 36 +- .../core/ccr/action/ResumeFollowAction.java | 195 +++++--- .../src/main/resources/monitoring-es.json | 34 +- 36 files changed, 1215 insertions(+), 918 deletions(-) diff --git a/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml index e0f058080cb70..4d4026f46a472 100644 --- a/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml +++ b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/auto_follow.yml @@ -25,7 +25,7 @@ body: remote_cluster: local leader_index_patterns: ['logs-*'] - max_concurrent_read_batches: 2 + max_outstanding_read_requests: 2 - is_true: acknowledged - do: @@ -33,13 +33,13 @@ name: my_pattern - match: { my_pattern.remote_cluster: 'local' } - match: { my_pattern.leader_index_patterns: ['logs-*'] } - - match: { my_pattern.max_concurrent_read_batches: 2 } + - match: { my_pattern.max_outstanding_read_requests: 2 } - do: ccr.get_auto_follow_pattern: {} - match: { my_pattern.remote_cluster: 'local' } - match: { my_pattern.leader_index_patterns: ['logs-*'] } - - match: { my_pattern.max_concurrent_read_batches: 2 } + - match: { my_pattern.max_outstanding_read_requests: 2 } - do: ccr.delete_auto_follow_pattern: diff --git a/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_stats.yml b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_stats.yml index 29bb68369d8cc..97c538b60bc4e 100644 --- a/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_stats.yml +++ b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_stats.yml @@ -56,21 +56,21 @@ - gte: { indices.0.shards.0.follower_global_checkpoint: -1 } - gte: { indices.0.shards.0.follower_max_seq_no: -1 } - gte: { indices.0.shards.0.last_requested_seq_no: -1 } - - gte: { indices.0.shards.0.number_of_concurrent_reads: 0 } - - match: { indices.0.shards.0.number_of_concurrent_writes: 0 } - - match: { indices.0.shards.0.number_of_queued_writes: 0 } - - gte: { indices.0.shards.0.mapping_version: 0 } - - gte: { indices.0.shards.0.total_fetch_time_millis: 0 } - - gte: { indices.0.shards.0.number_of_successful_fetches: 0 } - - gte: { indices.0.shards.0.number_of_failed_fetches: 0 } - - match: { indices.0.shards.0.operations_received: 0 } - - match: { indices.0.shards.0.total_transferred_bytes: 0 } - - match: { indices.0.shards.0.total_index_time_millis: 0 } - - match: { indices.0.shards.0.number_of_successful_bulk_operations: 0 } - - match: { indices.0.shards.0.number_of_failed_bulk_operations: 0 } - - match: { indices.0.shards.0.number_of_operations_indexed: 0 } - - length: { indices.0.shards.0.fetch_exceptions: 0 } - - gte: { indices.0.shards.0.time_since_last_fetch_millis: -1 } + - gte: { indices.0.shards.0.outstanding_read_requests: 0 } + - match: { indices.0.shards.0.outstanding_write_requests: 0 } + - match: { indices.0.shards.0.write_buffer_operation_count: 0 } + - gte: { indices.0.shards.0.follower_mapping_version: 0 } + - gte: { indices.0.shards.0.total_read_time_millis: 0 } + - gte: { indices.0.shards.0.successful_read_requests: 0 } + - gte: { indices.0.shards.0.failed_read_requests: 0 } + - match: { indices.0.shards.0.operations_read: 0 } + - match: { indices.0.shards.0.bytes_read: 0 } + - match: { indices.0.shards.0.total_write_time_millis: 0 } + - match: { indices.0.shards.0.successful_write_requests: 0 } + - match: { indices.0.shards.0.failed_write_requests: 0 } + - match: { indices.0.shards.0.operations_written: 0 } + - length: { indices.0.shards.0.read_exceptions: 0 } + - gte: { indices.0.shards.0.time_since_last_read_millis: -1 } - do: ccr.pause_follow: diff --git a/x-pack/plugin/ccr/qa/src/main/java/org/elasticsearch/xpack/ccr/ESCCRRestTestCase.java b/x-pack/plugin/ccr/qa/src/main/java/org/elasticsearch/xpack/ccr/ESCCRRestTestCase.java index 3b8951b343d4f..14780702fc4a4 100644 --- a/x-pack/plugin/ccr/qa/src/main/java/org/elasticsearch/xpack/ccr/ESCCRRestTestCase.java +++ b/x-pack/plugin/ccr/qa/src/main/java/org/elasticsearch/xpack/ccr/ESCCRRestTestCase.java @@ -59,7 +59,7 @@ protected static void refresh(String index) throws IOException { protected static void resumeFollow(String followIndex) throws IOException { final Request request = new Request("POST", "/" + followIndex + "/_ccr/resume_follow"); - request.setJsonEntity("{\"poll_timeout\": \"10ms\"}"); + request.setJsonEntity("{\"read_poll_timeout\": \"10ms\"}"); assertOK(client().performRequest(request)); } @@ -74,7 +74,7 @@ protected static void followIndex(String leaderCluster, String leaderIndex, Stri protected static void followIndex(RestClient client, String leaderCluster, String leaderIndex, String followIndex) throws IOException { final Request request = new Request("PUT", "/" + followIndex + "/_ccr/follow"); request.setJsonEntity("{\"remote_cluster\": \"" + leaderCluster + "\", \"leader_index\": \"" + leaderIndex + - "\", \"poll_timeout\": \"10ms\"}"); + "\", \"read_poll_timeout\": \"10ms\"}"); assertOK(client.performRequest(request)); } @@ -136,10 +136,10 @@ protected static void verifyCcrMonitoring(final String expectedLeaderIndex, fina assertThat(followerIndex, equalTo(expectedFollowerIndex)); int foundNumberOfOperationsReceived = - (int) XContentMapValues.extractValue("_source.ccr_stats.operations_received", hit); + (int) XContentMapValues.extractValue("_source.ccr_stats.operations_read", hit); numberOfOperationsReceived = Math.max(numberOfOperationsReceived, foundNumberOfOperationsReceived); int foundNumberOfOperationsIndexed = - (int) XContentMapValues.extractValue("_source.ccr_stats.number_of_operations_indexed", hit); + (int) XContentMapValues.extractValue("_source.ccr_stats.operations_written", hit); numberOfOperationsIndexed = Math.max(numberOfOperationsIndexed, foundNumberOfOperationsIndexed); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index b5ba39ae7e2f5..b32ed829cf42c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -324,14 +324,16 @@ private void followLeaderIndex(String autoFollowPattenName, ResumeFollowAction.Request followRequest = new ResumeFollowAction.Request(); followRequest.setFollowerIndex(followIndexName); - followRequest.setMaxBatchOperationCount(pattern.getMaxBatchOperationCount()); - followRequest.setMaxConcurrentReadBatches(pattern.getMaxConcurrentReadBatches()); - followRequest.setMaxBatchSize(pattern.getMaxBatchSize()); - followRequest.setMaxConcurrentWriteBatches(pattern.getMaxConcurrentWriteBatches()); + followRequest.setMaxReadRequestOperationCount(pattern.getMaxReadRequestOperationCount()); + followRequest.setMaxReadRequestSize(pattern.getMaxReadRequestSize()); + followRequest.setMaxOutstandingReadRequests(pattern.getMaxOutstandingReadRequests()); + followRequest.setMaxWriteRequestOperationCount(pattern.getMaxWriteRequestOperationCount()); + followRequest.setMaxWriteRequestSize(pattern.getMaxWriteRequestSize()); + followRequest.setMaxOutstandingWriteRequests(pattern.getMaxOutstandingWriteRequests()); followRequest.setMaxWriteBufferCount(pattern.getMaxWriteBufferCount()); followRequest.setMaxWriteBufferSize(pattern.getMaxWriteBufferSize()); followRequest.setMaxRetryDelay(pattern.getMaxRetryDelay()); - followRequest.setPollTimeout(pattern.getPollTimeout()); + followRequest.setReadPollTimeout(pattern.getPollTimeout()); PutFollowAction.Request request = new PutFollowAction.Request(); request.setRemoteCluster(remoteCluster); 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 611d31978692c..cf54a236a0451 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 @@ -65,8 +65,8 @@ public static class Request extends SingleShardRequest { private int maxOperationCount; private ShardId shardId; private String expectedHistoryUUID; - private TimeValue pollTimeout = TransportResumeFollowAction.DEFAULT_POLL_TIMEOUT; - private ByteSizeValue maxBatchSize = TransportResumeFollowAction.DEFAULT_MAX_BATCH_SIZE; + private TimeValue pollTimeout = TransportResumeFollowAction.DEFAULT_READ_POLL_TIMEOUT; + private ByteSizeValue maxBatchSize = TransportResumeFollowAction.DEFAULT_MAX_READ_REQUEST_SIZE; private long relativeStartNanos; 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 9788195c7e565..8c302344ad86d 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 @@ -70,19 +70,19 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { private long lastRequestedSeqNo; private long followerGlobalCheckpoint = 0; private long followerMaxSeqNo = 0; - private int numConcurrentReads = 0; - private int numConcurrentWrites = 0; + private int numOutstandingReads = 0; + private int numOutstandingWrites = 0; private long currentMappingVersion = 0; - private long totalFetchTookTimeMillis = 0; - private long totalFetchTimeMillis = 0; - private long numberOfSuccessfulFetches = 0; - private long numberOfFailedFetches = 0; - private long operationsReceived = 0; - private long totalTransferredBytes = 0; - private long totalIndexTimeMillis = 0; - private long numberOfSuccessfulBulkOperations = 0; - private long numberOfFailedBulkOperations = 0; - private long numberOfOperationsIndexed = 0; + private long totalReadRemoteExecTimeMillis = 0; + private long totalReadTimeMillis = 0; + private long successfulReadRequests = 0; + private long failedReadRequests = 0; + private long operationsRead = 0; + private long bytesRead = 0; + private long totalWriteTimeMillis = 0; + private long successfulWriteRequests = 0; + private long failedWriteRequests = 0; + private long operationWritten = 0; private long lastFetchTime = -1; private final Queue buffer = new PriorityQueue<>(Comparator.comparing(Translog.Operation::seqNo)); private long bufferSizeInBytes = 0; @@ -104,7 +104,7 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask { this.fetchExceptions = new LinkedHashMap>() { @Override protected boolean removeEldestEntry(final Map.Entry> eldest) { - return size() > params.getMaxConcurrentReadBatches(); + return size() > params.getMaxOutstandingReadRequests(); } }; } @@ -129,12 +129,12 @@ void start( } // updates follower mapping, this gets us the leader mapping version and makes sure that leader and follower mapping are identical - updateMapping(mappingVersion -> { + updateMapping(followerMappingVersion -> { synchronized (ShardFollowNodeTask.this) { - currentMappingVersion = mappingVersion; + currentMappingVersion = followerMappingVersion; } - LOGGER.info("{} Started to follow leader shard {}, followGlobalCheckPoint={}, mappingVersion={}", - params.getFollowShardId(), params.getLeaderShardId(), followerGlobalCheckpoint, mappingVersion); + LOGGER.info("{} Started to follow leader shard {}, followGlobalCheckPoint={}, followerMappingVersion={}", + params.getFollowShardId(), params.getLeaderShardId(), followerGlobalCheckpoint, followerMappingVersion); coordinateReads(); }); } @@ -147,41 +147,41 @@ synchronized void coordinateReads() { LOGGER.trace("{} coordinate reads, lastRequestedSeqNo={}, leaderGlobalCheckpoint={}", params.getFollowShardId(), lastRequestedSeqNo, leaderGlobalCheckpoint); - final int maxBatchOperationCount = params.getMaxBatchOperationCount(); + final int maxReadRequestOperationCount = params.getMaxReadRequestOperationCount(); while (hasReadBudget() && lastRequestedSeqNo < leaderGlobalCheckpoint) { final long from = lastRequestedSeqNo + 1; - final long maxRequiredSeqNo = Math.min(leaderGlobalCheckpoint, from + maxBatchOperationCount - 1); - final int requestBatchCount; - if (numConcurrentReads == 0) { + final long maxRequiredSeqNo = Math.min(leaderGlobalCheckpoint, from + maxReadRequestOperationCount - 1); + final int requestOpCount; + if (numOutstandingReads == 0) { // This is the only request, we can optimistically fetch more documents if possible but not enforce max_required_seqno. - requestBatchCount = maxBatchOperationCount; + requestOpCount = maxReadRequestOperationCount; } else { - requestBatchCount = Math.toIntExact(maxRequiredSeqNo - from + 1); + requestOpCount = Math.toIntExact(maxRequiredSeqNo - from + 1); } - assert 0 < requestBatchCount && requestBatchCount <= maxBatchOperationCount : "request_batch_count=" + requestBatchCount; + assert 0 < requestOpCount && requestOpCount <= maxReadRequestOperationCount : "read_request_operation_count=" + requestOpCount; LOGGER.trace("{}[{} ongoing reads] read from_seqno={} max_required_seqno={} batch_count={}", - params.getFollowShardId(), numConcurrentReads, from, maxRequiredSeqNo, requestBatchCount); - numConcurrentReads++; - sendShardChangesRequest(from, requestBatchCount, maxRequiredSeqNo); + params.getFollowShardId(), numOutstandingReads, from, maxRequiredSeqNo, requestOpCount); + numOutstandingReads++; + sendShardChangesRequest(from, requestOpCount, maxRequiredSeqNo); lastRequestedSeqNo = maxRequiredSeqNo; } - if (numConcurrentReads == 0 && hasReadBudget()) { + if (numOutstandingReads == 0 && hasReadBudget()) { assert lastRequestedSeqNo == leaderGlobalCheckpoint; // We sneak peek if there is any thing new in the leader. // If there is we will happily accept - numConcurrentReads++; + numOutstandingReads++; long from = lastRequestedSeqNo + 1; - LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numConcurrentReads, from); - sendShardChangesRequest(from, maxBatchOperationCount, lastRequestedSeqNo); + LOGGER.trace("{}[{}] peek read [{}]", params.getFollowShardId(), numOutstandingReads, from); + sendShardChangesRequest(from, maxReadRequestOperationCount, lastRequestedSeqNo); } } private boolean hasReadBudget() { assert Thread.holdsLock(this); - if (numConcurrentReads >= params.getMaxConcurrentReadBatches()) { + if (numOutstandingReads >= params.getMaxOutstandingReadRequests()) { LOGGER.trace("{} no new reads, maximum number of concurrent reads have been reached [{}]", - params.getFollowShardId(), numConcurrentReads); + params.getFollowShardId(), numOutstandingReads); return false; } if (bufferSizeInBytes >= params.getMaxWriteBufferSize().getBytes()) { @@ -203,19 +203,19 @@ private synchronized void coordinateWrites() { while (hasWriteBudget() && buffer.isEmpty() == false) { long sumEstimatedSize = 0L; - int length = Math.min(params.getMaxBatchOperationCount(), buffer.size()); + int length = Math.min(params.getMaxWriteRequestOperationCount(), 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.getMaxBatchSize().getBytes()) { + if (sumEstimatedSize > params.getMaxWriteRequestSize().getBytes()) { break; } } bufferSizeInBytes -= sumEstimatedSize; - numConcurrentWrites++; - LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numConcurrentWrites, ops.get(0).seqNo(), + numOutstandingWrites++; + LOGGER.trace("{}[{}] write [{}/{}] [{}]", params.getFollowShardId(), numOutstandingWrites, ops.get(0).seqNo(), ops.get(ops.size() - 1).seqNo(), ops.size()); sendBulkShardOperationsRequest(ops, leaderMaxSeqNoOfUpdatesOrDeletes, new AtomicInteger(0)); } @@ -223,9 +223,9 @@ private synchronized void coordinateWrites() { private boolean hasWriteBudget() { assert Thread.holdsLock(this); - if (numConcurrentWrites >= params.getMaxConcurrentWriteBatches()) { + if (numOutstandingWrites >= params.getMaxOutstandingWriteRequests()) { LOGGER.trace("{} maximum number of concurrent writes have been reached [{}]", - params.getFollowShardId(), numConcurrentWrites); + params.getFollowShardId(), numOutstandingWrites); return false; } return true; @@ -247,11 +247,11 @@ private void sendShardChangesRequest(long from, int maxOperationCount, long maxR fetchExceptions.remove(from); if (response.getOperations().length > 0) { // do not count polls against fetch stats - totalFetchTookTimeMillis += response.getTookInMillis(); - totalFetchTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); - numberOfSuccessfulFetches++; - operationsReceived += response.getOperations().length; - totalTransferredBytes += + totalReadRemoteExecTimeMillis += response.getTookInMillis(); + totalReadTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); + successfulReadRequests++; + operationsRead += response.getOperations().length; + bytesRead += Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::estimateSize).sum(); } } @@ -259,8 +259,8 @@ private void sendShardChangesRequest(long from, int maxOperationCount, long maxR }, e -> { synchronized (ShardFollowNodeTask.this) { - totalFetchTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); - numberOfFailedFetches++; + totalReadTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); + failedReadRequests++; fetchExceptions.put(from, Tuple.tuple(retryCounter, ExceptionsHelper.convertToElastic(e))); } handleFailure(e, retryCounter, () -> sendShardChangesRequest(from, maxOperationCount, maxRequiredSeqNo, retryCounter)); @@ -310,7 +310,7 @@ synchronized void innerHandleReadResponse(long from, long maxRequiredSeqNo, Shar sendShardChangesRequest(newFromSeqNo, newSize, maxRequiredSeqNo); } else { // read is completed, decrement - numConcurrentReads--; + numOutstandingReads--; coordinateReads(); } } @@ -322,16 +322,16 @@ private void sendBulkShardOperationsRequest(List operations, innerSendBulkShardOperationsRequest(followerHistoryUUID, operations, leaderMaxSeqNoOfUpdatesOrDeletes, response -> { synchronized (ShardFollowNodeTask.this) { - totalIndexTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); - numberOfSuccessfulBulkOperations++; - numberOfOperationsIndexed += operations.size(); + totalWriteTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); + successfulWriteRequests++; + operationWritten += operations.size(); } handleWriteResponse(response); }, e -> { synchronized (ShardFollowNodeTask.this) { - totalIndexTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); - numberOfFailedBulkOperations++; + totalWriteTimeMillis += TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTime); + failedWriteRequests++; } handleFailure(e, retryCounter, () -> sendBulkShardOperationsRequest(operations, leaderMaxSeqNoOfUpdatesOrDeletes, retryCounter)); @@ -342,8 +342,8 @@ private void sendBulkShardOperationsRequest(List operations, private synchronized void handleWriteResponse(final BulkShardOperationsResponse response) { this.followerGlobalCheckpoint = Math.max(this.followerGlobalCheckpoint, response.getGlobalCheckpoint()); this.followerMaxSeqNo = Math.max(this.followerMaxSeqNo, response.getMaxSeqNo()); - numConcurrentWrites--; - assert numConcurrentWrites >= 0; + numOutstandingWrites--; + assert numOutstandingWrites >= 0; coordinateWrites(); // In case that buffer has more ops than is allowed then reads may all have been stopped, @@ -380,7 +380,7 @@ private void handleFailure(Exception e, AtomicInteger retryCounter, Runnable tas int currentRetry = retryCounter.incrementAndGet(); LOGGER.debug(new ParameterizedMessage("{} error during follow shard task, retrying [{}]", params.getFollowShardId(), currentRetry), e); - long delay = computeDelay(currentRetry, params.getPollTimeout().getMillis()); + long delay = computeDelay(currentRetry, params.getReadPollTimeout().getMillis()); scheduler.accept(TimeValue.timeValueMillis(delay), task); } else { fatalException = ExceptionsHelper.convertToElastic(e); @@ -463,21 +463,21 @@ public synchronized ShardFollowNodeTaskStatus getStatus() { followerGlobalCheckpoint, followerMaxSeqNo, lastRequestedSeqNo, - numConcurrentReads, - numConcurrentWrites, + numOutstandingReads, + numOutstandingWrites, buffer.size(), bufferSizeInBytes, currentMappingVersion, - totalFetchTimeMillis, - totalFetchTookTimeMillis, - numberOfSuccessfulFetches, - numberOfFailedFetches, - operationsReceived, - totalTransferredBytes, - totalIndexTimeMillis, - numberOfSuccessfulBulkOperations, - numberOfFailedBulkOperations, - numberOfOperationsIndexed, + totalReadTimeMillis, + totalReadRemoteExecTimeMillis, + successfulReadRequests, + failedReadRequests, + operationsRead, + bytesRead, + totalWriteTimeMillis, + successfulWriteRequests, + failedWriteRequests, + operationWritten, new TreeMap<>( fetchExceptions .entrySet() 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 13e3da77491c7..f22fe0d2238ac 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 @@ -44,20 +44,23 @@ 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_BATCH_OPERATION_COUNT = new ParseField("max_batch_operation_count"); - public static final ParseField MAX_CONCURRENT_READ_BATCHES = new ParseField("max_concurrent_read_batches"); - public static final ParseField MAX_BATCH_SIZE = new ParseField("max_batch_size"); - public static final ParseField MAX_CONCURRENT_WRITE_BATCHES = new ParseField("max_concurrent_write_batches"); + public static final ParseField MAX_READ_REQUEST_OPERATION_COUNT = new ParseField("max_read_request_operation_count"); + public static final ParseField MAX_READ_REQUEST_SIZE = new ParseField("max_read_request_size"); + public static final ParseField MAX_OUTSTANDING_READ_REQUESTS = new ParseField("max_outstanding_read_requests"); + public static final ParseField MAX_WRITE_REQUEST_OPERATION_COUNT = new ParseField("max_write_request_operation_count"); + public static final ParseField MAX_WRITE_REQUEST_SIZE = new ParseField("max_write_request_size"); + public static final ParseField MAX_OUTSTANDING_WRITE_REQUESTS = new ParseField("max_outstanding_write_requests"); public static final ParseField MAX_WRITE_BUFFER_COUNT = new ParseField("max_write_buffer_count"); public static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); public static final ParseField MAX_RETRY_DELAY = new ParseField("max_retry_delay"); - public static final ParseField POLL_TIMEOUT = new ParseField("poll_timeout"); + public static final ParseField READ_POLL_TIMEOUT = new ParseField("read_poll_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], (ByteSizeValue) a[9], - (int) a[10], (int) a[11], (ByteSizeValue) a[12], (TimeValue) a[13], (TimeValue) a[14], (Map) a[15])); + (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], (ByteSizeValue) a[8], (int) a[9], (int) a[10], (ByteSizeValue) a[11], (int) a[12], + (int) a[13], (ByteSizeValue) a[14], (TimeValue) a[15], (TimeValue) a[16], (Map) a[17])); static { PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), REMOTE_CLUSTER_FIELD); @@ -67,14 +70,18 @@ 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_BATCH_OPERATION_COUNT); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_READ_BATCHES); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_READ_REQUEST_OPERATION_COUNT); PARSER.declareField( ConstructingObjectParser.constructorArg(), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_BATCH_SIZE.getPreferredName()), - MAX_BATCH_SIZE, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_READ_REQUEST_SIZE.getPreferredName()), MAX_READ_REQUEST_SIZE, ObjectParser.ValueType.STRING); - PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_CONCURRENT_WRITE_BATCHES); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_OUTSTANDING_READ_REQUESTS); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_REQUEST_OPERATION_COUNT); + PARSER.declareField( + ConstructingObjectParser.constructorArg(), + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_BUFFER_SIZE.getPreferredName()), MAX_WRITE_REQUEST_SIZE, + ObjectParser.ValueType.STRING); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_OUTSTANDING_WRITE_REQUESTS); PARSER.declareInt(ConstructingObjectParser.constructorArg(), MAX_WRITE_BUFFER_COUNT); PARSER.declareField( ConstructingObjectParser.constructorArg(), @@ -85,48 +92,54 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams { (p, c) -> TimeValue.parseTimeValue(p.text(), MAX_RETRY_DELAY.getPreferredName()), MAX_RETRY_DELAY, ObjectParser.ValueType.STRING); PARSER.declareField(ConstructingObjectParser.constructorArg(), - (p, c) -> TimeValue.parseTimeValue(p.text(), POLL_TIMEOUT.getPreferredName()), - POLL_TIMEOUT, ObjectParser.ValueType.STRING); + (p, c) -> TimeValue.parseTimeValue(p.text(), READ_POLL_TIMEOUT.getPreferredName()), + READ_POLL_TIMEOUT, ObjectParser.ValueType.STRING); PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> p.mapStrings(), HEADERS); } private final String remoteCluster; private final ShardId followShardId; private final ShardId leaderShardId; - private final int maxBatchOperationCount; - private final int maxConcurrentReadBatches; - private final ByteSizeValue maxBatchSize; - private final int maxConcurrentWriteBatches; + private final int maxReadRequestOperationCount; + private final ByteSizeValue maxReadRequestSize; + private final int maxOutstandingReadRequests; + private final int maxWriteRequestOperationCount; + private final ByteSizeValue maxWriteRequestSize; + private final int maxOutstandingWriteRequests; private final int maxWriteBufferCount; private final ByteSizeValue maxWriteBufferSize; private final TimeValue maxRetryDelay; - private final TimeValue pollTimeout; + private final TimeValue readPollTimeout; private final Map headers; ShardFollowTask( final String remoteCluster, final ShardId followShardId, final ShardId leaderShardId, - final int maxBatchOperationCount, - final int maxConcurrentReadBatches, - final ByteSizeValue maxBatchSize, - final int maxConcurrentWriteBatches, + final int maxReadRequestOperationCount, + final ByteSizeValue maxReadRequestSize, + final int maxOutstandingReadRequests, + final int maxWriteRequestOperationCount, + final ByteSizeValue maxWriteRequestSize, + final int maxOutstandingWriteRequests, final int maxWriteBufferCount, final ByteSizeValue maxWriteBufferSize, final TimeValue maxRetryDelay, - final TimeValue pollTimeout, + final TimeValue readPollTimeout, final Map headers) { this.remoteCluster = remoteCluster; this.followShardId = followShardId; this.leaderShardId = leaderShardId; - this.maxBatchOperationCount = maxBatchOperationCount; - this.maxConcurrentReadBatches = maxConcurrentReadBatches; - this.maxBatchSize = maxBatchSize; - this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; + this.maxReadRequestOperationCount = maxReadRequestOperationCount; + this.maxReadRequestSize = maxReadRequestSize; + this.maxOutstandingReadRequests = maxOutstandingReadRequests; + this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; + this.maxWriteRequestSize = maxWriteRequestSize; + this.maxOutstandingWriteRequests = maxOutstandingWriteRequests; this.maxWriteBufferCount = maxWriteBufferCount; this.maxWriteBufferSize = maxWriteBufferSize; this.maxRetryDelay = maxRetryDelay; - this.pollTimeout = pollTimeout; + this.readPollTimeout = readPollTimeout; this.headers = headers != null ? Collections.unmodifiableMap(headers) : Collections.emptyMap(); } @@ -134,14 +147,16 @@ public ShardFollowTask(StreamInput in) throws IOException { this.remoteCluster = in.readString(); this.followShardId = ShardId.readShardId(in); this.leaderShardId = ShardId.readShardId(in); - this.maxBatchOperationCount = in.readVInt(); - this.maxConcurrentReadBatches = in.readVInt(); - this.maxBatchSize = new ByteSizeValue(in); - this.maxConcurrentWriteBatches = in.readVInt(); + this.maxReadRequestOperationCount = in.readVInt(); + this.maxReadRequestSize = new ByteSizeValue(in); + this.maxOutstandingReadRequests = in.readVInt(); + this.maxWriteRequestOperationCount = in.readVInt(); + this.maxWriteRequestSize = new ByteSizeValue(in); + this.maxOutstandingWriteRequests = in.readVInt(); this.maxWriteBufferCount = in.readVInt(); this.maxWriteBufferSize = new ByteSizeValue(in); this.maxRetryDelay = in.readTimeValue(); - this.pollTimeout = in.readTimeValue(); + this.readPollTimeout = in.readTimeValue(); this.headers = Collections.unmodifiableMap(in.readMap(StreamInput::readString, StreamInput::readString)); } @@ -157,16 +172,24 @@ public ShardId getLeaderShardId() { return leaderShardId; } - public int getMaxBatchOperationCount() { - return maxBatchOperationCount; + public int getMaxReadRequestOperationCount() { + return maxReadRequestOperationCount; + } + + public int getMaxOutstandingReadRequests() { + return maxOutstandingReadRequests; + } + + public int getMaxWriteRequestOperationCount() { + return maxWriteRequestOperationCount; } - public int getMaxConcurrentReadBatches() { - return maxConcurrentReadBatches; + public ByteSizeValue getMaxWriteRequestSize() { + return maxWriteRequestSize; } - public int getMaxConcurrentWriteBatches() { - return maxConcurrentWriteBatches; + public int getMaxOutstandingWriteRequests() { + return maxOutstandingWriteRequests; } public int getMaxWriteBufferCount() { @@ -177,16 +200,16 @@ public ByteSizeValue getMaxWriteBufferSize() { return maxWriteBufferSize; } - public ByteSizeValue getMaxBatchSize() { - return maxBatchSize; + public ByteSizeValue getMaxReadRequestSize() { + return maxReadRequestSize; } public TimeValue getMaxRetryDelay() { return maxRetryDelay; } - public TimeValue getPollTimeout() { - return pollTimeout; + public TimeValue getReadPollTimeout() { + return readPollTimeout; } public String getTaskId() { @@ -207,14 +230,16 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(remoteCluster); followShardId.writeTo(out); leaderShardId.writeTo(out); - out.writeVLong(maxBatchOperationCount); - out.writeVInt(maxConcurrentReadBatches); - maxBatchSize.writeTo(out); - out.writeVInt(maxConcurrentWriteBatches); + out.writeVLong(maxReadRequestOperationCount); + maxReadRequestSize.writeTo(out); + out.writeVInt(maxOutstandingReadRequests); + out.writeVLong(maxWriteRequestOperationCount); + maxWriteRequestSize.writeTo(out); + out.writeVInt(maxOutstandingWriteRequests); out.writeVInt(maxWriteBufferCount); maxWriteBufferSize.writeTo(out); out.writeTimeValue(maxRetryDelay); - out.writeTimeValue(pollTimeout); + out.writeTimeValue(readPollTimeout); out.writeMap(headers, StreamOutput::writeString, StreamOutput::writeString); } @@ -232,14 +257,16 @@ 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(), maxBatchOperationCount); - builder.field(MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); - builder.field(MAX_BATCH_SIZE.getPreferredName(), maxBatchSize.getStringRep()); - builder.field(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + builder.field(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), maxReadRequestOperationCount); + builder.field(MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep()); + builder.field(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxOutstandingReadRequests); + builder.field(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount); + builder.field(MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep()); + builder.field(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), maxOutstandingWriteRequests); builder.field(MAX_WRITE_BUFFER_COUNT.getPreferredName(), maxWriteBufferCount); builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize.getStringRep()); builder.field(MAX_RETRY_DELAY.getPreferredName(), maxRetryDelay.getStringRep()); - builder.field(POLL_TIMEOUT.getPreferredName(), pollTimeout.getStringRep()); + builder.field(READ_POLL_TIMEOUT.getPreferredName(), readPollTimeout.getStringRep()); builder.field(HEADERS.getPreferredName(), headers); return builder.endObject(); } @@ -252,14 +279,16 @@ public boolean equals(Object o) { return Objects.equals(remoteCluster, that.remoteCluster) && Objects.equals(followShardId, that.followShardId) && Objects.equals(leaderShardId, that.leaderShardId) && - maxBatchOperationCount == that.maxBatchOperationCount && - maxConcurrentReadBatches == that.maxConcurrentReadBatches && - maxConcurrentWriteBatches == that.maxConcurrentWriteBatches && - maxBatchSize.equals(that.maxBatchSize) && + maxReadRequestOperationCount == that.maxReadRequestOperationCount && + maxReadRequestSize.equals(that.maxReadRequestSize) && + maxOutstandingReadRequests == that.maxOutstandingReadRequests && + maxWriteRequestOperationCount == that.maxWriteRequestOperationCount && + maxWriteRequestSize.equals(that.maxWriteRequestSize) && + maxOutstandingWriteRequests == that.maxOutstandingWriteRequests && maxWriteBufferCount == that.maxWriteBufferCount && maxWriteBufferSize.equals(that.maxWriteBufferSize) && Objects.equals(maxRetryDelay, that.maxRetryDelay) && - Objects.equals(pollTimeout, that.pollTimeout) && + Objects.equals(readPollTimeout, that.readPollTimeout) && Objects.equals(headers, that.headers); } @@ -269,14 +298,16 @@ public int hashCode() { remoteCluster, followShardId, leaderShardId, - maxBatchOperationCount, - maxConcurrentReadBatches, - maxConcurrentWriteBatches, - maxBatchSize, + maxReadRequestOperationCount, + maxReadRequestSize, + maxOutstandingReadRequests, + maxWriteRequestOperationCount, + maxWriteRequestSize, + maxOutstandingWriteRequests, maxWriteBufferCount, maxWriteBufferSize, maxRetryDelay, - pollTimeout, + readPollTimeout, headers ); } 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 128c2a846d8b9..5a82b45cf8c38 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 @@ -153,8 +153,8 @@ protected void innerSendShardChangesRequest(long from, int maxOperationCount, Co new ShardChangesAction.Request(params.getLeaderShardId(), recordedLeaderShardHistoryUUID); request.setFromSeqNo(from); request.setMaxOperationCount(maxOperationCount); - request.setMaxBatchSize(params.getMaxBatchSize()); - request.setPollTimeout(params.getPollTimeout()); + request.setMaxBatchSize(params.getMaxReadRequestSize()); + request.setPollTimeout(params.getReadPollTimeout()); leaderClient.execute(ShardChangesAction.INSTANCE, request, ActionListener.wrap(handler::accept, errorHandler)); } }; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java index 79f1ed7a2ee5e..8416c414990df 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternAction.java @@ -160,14 +160,16 @@ static ClusterState innerPut(PutAutoFollowPatternAction.Request request, request.getRemoteCluster(), request.getLeaderIndexPatterns(), request.getFollowIndexNamePattern(), - request.getMaxBatchOperationCount(), + request.getMaxReadRequestOperationCount(), + request.getMaxReadRequestSize(), request.getMaxConcurrentReadBatches(), - request.getMaxBatchSize(), + request.getMaxWriteRequestOperationCount(), + request.getMaxWriteRequestSize(), request.getMaxConcurrentWriteBatches(), request.getMaxWriteBufferCount(), request.getMaxWriteBufferSize(), request.getMaxRetryDelay(), - request.getPollTimeout()); + request.getReadPollTimeout()); patterns.put(request.getName(), autoFollowPattern); ClusterState.Builder newState = ClusterState.builder(localState); newState.metaData(MetaData.builder(localState.getMetaData()) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java index 97905f927211b..cdbe3b25f1d6a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java @@ -52,14 +52,16 @@ public class TransportResumeFollowAction extends HandledTransportAction { - static final ByteSizeValue DEFAULT_MAX_BATCH_SIZE = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); + static final ByteSizeValue DEFAULT_MAX_READ_REQUEST_SIZE = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); + static final ByteSizeValue DEFAULT_MAX_WRITE_REQUEST_SIZE = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); private static final TimeValue DEFAULT_MAX_RETRY_DELAY = new TimeValue(500); - private static final int DEFAULT_MAX_CONCURRENT_WRITE_BATCHES = 9; + private static final int DEFAULT_MAX_OUTSTANDING_WRITE_REQUESTS = 9; private static final int DEFAULT_MAX_WRITE_BUFFER_COUNT = Integer.MAX_VALUE; private static final ByteSizeValue DEFAULT_MAX_WRITE_BUFFER_SIZE = new ByteSizeValue(512, ByteSizeUnit.MB); - private static final int DEFAULT_MAX_BATCH_OPERATION_COUNT = 5120; - private static final int DEFAULT_MAX_CONCURRENT_READ_BATCHES = 12; - static final TimeValue DEFAULT_POLL_TIMEOUT = TimeValue.timeValueMinutes(1); + private static final int DEFAULT_MAX_READ_REQUEST_OPERATION_COUNT = 5120; + private static final int DEFAULT_MAX_WRITE_REQUEST_OPERATION_COUNT = 5120; + private static final int DEFAULT_MAX_OUTSTANDING_READ_REQUESTS = 12; + static final TimeValue DEFAULT_READ_POLL_TIMEOUT = TimeValue.timeValueMinutes(1); private final Client client; private final ThreadPool threadPool; @@ -232,32 +234,46 @@ private static ShardFollowTask createShardFollowTask( IndexMetaData followIndexMetadata, Map filteredHeaders ) { - int maxBatchOperationCount; - if (request.getMaxBatchOperationCount() != null) { - maxBatchOperationCount = request.getMaxBatchOperationCount(); + int maxReadRequestOperationCount; + if (request.getMaxReadRequestOperationCount() != null) { + maxReadRequestOperationCount = request.getMaxReadRequestOperationCount(); } else { - maxBatchOperationCount = DEFAULT_MAX_BATCH_OPERATION_COUNT; + maxReadRequestOperationCount = DEFAULT_MAX_READ_REQUEST_OPERATION_COUNT; } - int maxConcurrentReadBatches; - if (request.getMaxConcurrentReadBatches() != null){ - maxConcurrentReadBatches = request.getMaxConcurrentReadBatches(); + ByteSizeValue maxReadRequestSize; + if (request.getMaxReadRequestSize() != null) { + maxReadRequestSize = request.getMaxReadRequestSize(); } else { - maxConcurrentReadBatches = DEFAULT_MAX_CONCURRENT_READ_BATCHES; + maxReadRequestSize = DEFAULT_MAX_READ_REQUEST_SIZE; } - ByteSizeValue maxBatchSize; - if (request.getMaxBatchSize() != null) { - maxBatchSize = request.getMaxBatchSize(); + int maxOutstandingReadRequests; + if (request.getMaxOutstandingReadRequests() != null){ + maxOutstandingReadRequests = request.getMaxOutstandingReadRequests(); } else { - maxBatchSize = DEFAULT_MAX_BATCH_SIZE; + maxOutstandingReadRequests = DEFAULT_MAX_OUTSTANDING_READ_REQUESTS; } - int maxConcurrentWriteBatches; - if (request.getMaxConcurrentWriteBatches() != null) { - maxConcurrentWriteBatches = request.getMaxConcurrentWriteBatches(); + final int maxWriteRequestOperationCount; + if (request.getMaxWriteRequestOperationCount() != null) { + maxWriteRequestOperationCount = request.getMaxWriteRequestOperationCount(); } else { - maxConcurrentWriteBatches = DEFAULT_MAX_CONCURRENT_WRITE_BATCHES; + maxWriteRequestOperationCount = DEFAULT_MAX_WRITE_REQUEST_OPERATION_COUNT; + } + + final ByteSizeValue maxWriteRequestSize; + if (request.getMaxWriteRequestSize() != null) { + maxWriteRequestSize = request.getMaxWriteRequestSize(); + } else { + maxWriteRequestSize = DEFAULT_MAX_WRITE_REQUEST_SIZE; + } + + int maxOutstandingWriteRequests; + if (request.getMaxOutstandingWriteRequests() != null) { + maxOutstandingWriteRequests = request.getMaxOutstandingWriteRequests(); + } else { + maxOutstandingWriteRequests = DEFAULT_MAX_OUTSTANDING_WRITE_REQUESTS; } int maxWriteBufferCount; @@ -275,20 +291,22 @@ private static ShardFollowTask createShardFollowTask( } TimeValue maxRetryDelay = request.getMaxRetryDelay() == null ? DEFAULT_MAX_RETRY_DELAY : request.getMaxRetryDelay(); - TimeValue pollTimeout = request.getPollTimeout() == null ? DEFAULT_POLL_TIMEOUT : request.getPollTimeout(); + TimeValue readPollTimeout = request.getReadPollTimeout() == null ? DEFAULT_READ_POLL_TIMEOUT : request.getReadPollTimeout(); return new ShardFollowTask( clusterAliasName, new ShardId(followIndexMetadata.getIndex(), shardId), new ShardId(leaderIndexMetadata.getIndex(), shardId), - maxBatchOperationCount, - maxConcurrentReadBatches, - maxBatchSize, - maxConcurrentWriteBatches, + maxReadRequestOperationCount, + maxReadRequestSize, + maxOutstandingReadRequests, + maxWriteRequestOperationCount, + maxWriteRequestSize, + maxOutstandingWriteRequests, maxWriteBufferCount, maxWriteBufferSize, maxRetryDelay, - pollTimeout, + readPollTimeout, filteredHeaders ); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index ebcfb1ecac818..a4f9d69bfa924 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -273,8 +273,8 @@ protected void ensureEmptyWriteBuffers() throws Exception { leaderClient().execute(FollowStatsAction.INSTANCE, new FollowStatsAction.StatsRequest()).actionGet(); for (FollowStatsAction.StatsResponse statsResponse : statsResponses.getStatsResponses()) { ShardFollowNodeTaskStatus status = statsResponse.status(); - assertThat(status.numberOfQueuedWrites(), equalTo(0)); - assertThat(status.bufferSize(), equalTo(0L)); + assertThat(status.writeBufferOperationCount(), equalTo(0)); + assertThat(status.writeBufferSizeInBytes(), equalTo(0L)); } }); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java index 611fb0c27fabe..439950019a65c 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java @@ -71,7 +71,7 @@ protected ResumeFollowAction.Request getResumeFollowRequest() { ResumeFollowAction.Request request = new ResumeFollowAction.Request(); request.setFollowerIndex("follower"); request.setMaxRetryDelay(TimeValue.timeValueMillis(10)); - request.setPollTimeout(TimeValue.timeValueMillis(10)); + request.setReadPollTimeout(TimeValue.timeValueMillis(10)); return request; } @@ -89,8 +89,8 @@ protected void ensureEmptyWriteBuffers() throws Exception { client().execute(FollowStatsAction.INSTANCE, new FollowStatsAction.StatsRequest()).actionGet(); for (FollowStatsAction.StatsResponse statsResponse : statsResponses.getStatsResponses()) { ShardFollowNodeTaskStatus status = statsResponse.status(); - assertThat(status.numberOfQueuedWrites(), equalTo(0)); - assertThat(status.bufferSize(), equalTo(0L)); + assertThat(status.writeBufferOperationCount(), equalTo(0)); + assertThat(status.writeBufferSizeInBytes(), equalTo(0L)); } }); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java index 50e6008d91d2a..305daa34d3010 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java @@ -135,19 +135,25 @@ public void testAutoFollowParameterAreDelegated() throws Exception { request.setMaxConcurrentWriteBatches(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxBatchOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); + request.setMaxReadRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { request.setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong())); } if (randomBoolean()) { - request.setMaxBatchSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); + request.setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } if (randomBoolean()) { request.setMaxRetryDelay(TimeValue.timeValueMillis(500)); } if (randomBoolean()) { - request.setPollTimeout(TimeValue.timeValueMillis(500)); + request.setReadPollTimeout(TimeValue.timeValueMillis(500)); + } + if (randomBoolean()) { + request.setMaxWriteRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + request.setMaxWriteBufferSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } assertTrue(followerClient().execute(PutAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); @@ -167,22 +173,28 @@ public void testAutoFollowParameterAreDelegated() throws Exception { assertThat(shardFollowTask.getMaxWriteBufferSize(), equalTo(request.getMaxWriteBufferSize())); } if (request.getMaxConcurrentReadBatches() != null) { - assertThat(shardFollowTask.getMaxConcurrentReadBatches(), equalTo(request.getMaxConcurrentReadBatches())); + assertThat(shardFollowTask.getMaxOutstandingReadRequests(), equalTo(request.getMaxConcurrentReadBatches())); } if (request.getMaxConcurrentWriteBatches() != null) { - assertThat(shardFollowTask.getMaxConcurrentWriteBatches(), equalTo(request.getMaxConcurrentWriteBatches())); + assertThat(shardFollowTask.getMaxOutstandingWriteRequests(), equalTo(request.getMaxConcurrentWriteBatches())); } - if (request.getMaxBatchOperationCount() != null) { - assertThat(shardFollowTask.getMaxBatchOperationCount(), equalTo(request.getMaxBatchOperationCount())); + if (request.getMaxReadRequestOperationCount() != null) { + assertThat(shardFollowTask.getMaxReadRequestOperationCount(), equalTo(request.getMaxReadRequestOperationCount())); } - if (request.getMaxBatchSize() != null) { - assertThat(shardFollowTask.getMaxBatchSize(), equalTo(request.getMaxBatchSize())); + if (request.getMaxReadRequestSize() != null) { + assertThat(shardFollowTask.getMaxReadRequestSize(), equalTo(request.getMaxReadRequestSize())); } if (request.getMaxRetryDelay() != null) { assertThat(shardFollowTask.getMaxRetryDelay(), equalTo(request.getMaxRetryDelay())); } - if (request.getPollTimeout() != null) { - assertThat(shardFollowTask.getPollTimeout(), equalTo(request.getPollTimeout())); + if (request.getReadPollTimeout() != null) { + assertThat(shardFollowTask.getReadPollTimeout(), equalTo(request.getReadPollTimeout())); + } + if (request.getMaxWriteRequestOperationCount() != null) { + assertThat(shardFollowTask.getMaxWriteRequestOperationCount(), equalTo(request.getMaxWriteRequestOperationCount())); + } + if (request.getMaxWriteRequestSize() != null) { + assertThat(shardFollowTask.getMaxWriteRequestSize(), equalTo(request.getMaxWriteRequestSize())); } }); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowMetadataTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowMetadataTests.java index 0ca175cef824f..5dab22500a600 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowMetadataTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowMetadataTests.java @@ -45,6 +45,8 @@ protected AutoFollowMetadata createTestInstance() { leaderPatterns, randomAlphaOfLength(4), randomIntBetween(0, Integer.MAX_VALUE), + new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES), + randomIntBetween(0, Integer.MAX_VALUE), randomIntBetween(0, Integer.MAX_VALUE), new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES), randomIntBetween(0, Integer.MAX_VALUE), diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java index 440d34ff3ac9b..967c3a7e8c759 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrLicenseIT.java @@ -148,7 +148,7 @@ public void testAutoFollowCoordinatorLogsSkippingAutoFollowCoordinationWithNonCo @Override public ClusterState execute(ClusterState currentState) throws Exception { AutoFollowPattern autoFollowPattern = new AutoFollowPattern("test_alias", Collections.singletonList("logs-*"), - null, null, null, null, null, null, null, null, null); + null, null, null, null, null, null, null, null, null, null, null); AutoFollowMetadata autoFollowMetadata = new AutoFollowMetadata( Collections.singletonMap("test_alias", autoFollowPattern), Collections.emptyMap(), diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index 8940bdd748d31..794c64e6bc4ff 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -252,9 +252,9 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) atLeastDocsIndexed(leaderClient(), "index1", numDocsIndexed / 3); PutFollowAction.Request followRequest = putFollow("index1", "index2"); - followRequest.getFollowRequest().setMaxBatchOperationCount(maxReadSize); - followRequest.getFollowRequest().setMaxConcurrentReadBatches(randomIntBetween(2, 10)); - followRequest.getFollowRequest().setMaxConcurrentWriteBatches(randomIntBetween(2, 10)); + followRequest.getFollowRequest().setMaxReadRequestOperationCount(maxReadSize); + followRequest.getFollowRequest().setMaxOutstandingReadRequests(randomIntBetween(2, 10)); + followRequest.getFollowRequest().setMaxOutstandingWriteRequests(randomIntBetween(2, 10)); followRequest.getFollowRequest().setMaxWriteBufferCount(randomIntBetween(1024, 10240)); followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); @@ -294,13 +294,16 @@ public void testFollowIndexAndCloseNode() throws Exception { thread.start(); PutFollowAction.Request followRequest = putFollow("index1", "index2"); - followRequest.getFollowRequest().setMaxBatchOperationCount(randomIntBetween(32, 2048)); - followRequest.getFollowRequest().setMaxConcurrentReadBatches(randomIntBetween(2, 10)); - followRequest.getFollowRequest().setMaxConcurrentWriteBatches(randomIntBetween(2, 10)); + followRequest.getFollowRequest().setMaxReadRequestOperationCount(randomIntBetween(32, 2048)); + followRequest.getFollowRequest().setMaxReadRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + followRequest.getFollowRequest().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); + followRequest.getFollowRequest().setMaxWriteRequestOperationCount(randomIntBetween(32, 2048)); + followRequest.getFollowRequest().setMaxWriteRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + followRequest.getFollowRequest().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); - long maxNumDocsReplicated = Math.min(1000, randomLongBetween(followRequest.getFollowRequest().getMaxBatchOperationCount(), - followRequest.getFollowRequest().getMaxBatchOperationCount() * 10)); + long maxNumDocsReplicated = Math.min(1000, randomLongBetween(followRequest.getFollowRequest().getMaxReadRequestOperationCount(), + followRequest.getFollowRequest().getMaxReadRequestOperationCount() * 10)); long minNumDocsReplicated = maxNumDocsReplicated / 3L; logger.info("waiting for at least [{}] documents to be indexed and then stop a random data node", minNumDocsReplicated); atLeastDocsIndexed(followerClient(), "index2", minNumDocsReplicated); @@ -397,7 +400,7 @@ public void testFollowIndexMaxOperationSizeInBytes() throws Exception { } PutFollowAction.Request followRequest = putFollow("index1", "index2"); - followRequest.getFollowRequest().setMaxBatchSize(new ByteSizeValue(1, ByteSizeUnit.BYTES)); + followRequest.getFollowRequest().setMaxReadRequestSize(new ByteSizeValue(1, ByteSizeUnit.BYTES)); followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); final Map firstBatchNumDocsPerShard = new HashMap<>(); @@ -456,10 +459,10 @@ public void testCloseLeaderIndex() throws Exception { assertThat(response.getNodeFailures(), empty()); assertThat(response.getTaskFailures(), empty()); assertThat(response.getStatsResponses(), hasSize(1)); - assertThat(response.getStatsResponses().get(0).status().numberOfFailedFetches(), greaterThanOrEqualTo(1L)); - assertThat(response.getStatsResponses().get(0).status().fetchExceptions().size(), equalTo(1)); + assertThat(response.getStatsResponses().get(0).status().failedReadRequests(), greaterThanOrEqualTo(1L)); + assertThat(response.getStatsResponses().get(0).status().readExceptions().size(), equalTo(1)); ElasticsearchException exception = response.getStatsResponses().get(0).status() - .fetchExceptions().entrySet().iterator().next().getValue().v2(); + .readExceptions().entrySet().iterator().next().getValue().v2(); assertThat(exception.getRootCause().getMessage(), equalTo("blocked by: [FORBIDDEN/4/index closed];")); }); @@ -491,7 +494,7 @@ public void testCloseFollowIndex() throws Exception { assertThat(response.getNodeFailures(), empty()); assertThat(response.getTaskFailures(), empty()); assertThat(response.getStatsResponses(), hasSize(1)); - assertThat(response.getStatsResponses().get(0).status().numberOfFailedBulkOperations(), greaterThanOrEqualTo(1L)); + assertThat(response.getStatsResponses().get(0).status().failedWriteRequests(), greaterThanOrEqualTo(1L)); }); followerClient().admin().indices().open(new OpenIndexRequest("index2")).actionGet(); assertBusy(() -> assertThat(followerClient().prepareSearch("index2").get().getHits().totalHits, equalTo(2L))); @@ -519,7 +522,7 @@ public void testDeleteLeaderIndex() throws Exception { assertThat(response.getNodeFailures(), empty()); assertThat(response.getTaskFailures(), empty()); assertThat(response.getStatsResponses(), hasSize(1)); - assertThat(response.getStatsResponses().get(0).status().numberOfFailedFetches(), greaterThanOrEqualTo(1L)); + assertThat(response.getStatsResponses().get(0).status().failedReadRequests(), greaterThanOrEqualTo(1L)); ElasticsearchException fatalException = response.getStatsResponses().get(0).status().getFatalException(); assertThat(fatalException, notNullValue()); assertThat(fatalException.getRootCause().getMessage(), equalTo("no such index [index1]")); @@ -549,7 +552,7 @@ public void testDeleteFollowerIndex() throws Exception { assertThat(response.getNodeFailures(), empty()); assertThat(response.getTaskFailures(), empty()); assertThat(response.getStatsResponses(), hasSize(1)); - assertThat(response.getStatsResponses().get(0).status().numberOfFailedBulkOperations(), greaterThanOrEqualTo(1L)); + assertThat(response.getStatsResponses().get(0).status().failedWriteRequests(), greaterThanOrEqualTo(1L)); ElasticsearchException fatalException = response.getStatsResponses().get(0).status().getFatalException(); assertThat(fatalException, notNullValue()); assertThat(fatalException.getMessage(), equalTo("no such index [index2]")); @@ -613,6 +616,12 @@ public void testFailOverOnFollower() throws Exception { threads[i].start(); } PutFollowAction.Request follow = putFollow("leader-index", "follower-index"); + follow.getFollowRequest().setMaxReadRequestOperationCount(randomIntBetween(32, 2048)); + follow.getFollowRequest().setMaxReadRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + follow.getFollowRequest().setMaxOutstandingReadRequests(randomIntBetween(1, 10)); + follow.getFollowRequest().setMaxWriteRequestOperationCount(randomIntBetween(32, 2048)); + follow.getFollowRequest().setMaxWriteRequestSize(new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)); + follow.getFollowRequest().setMaxOutstandingWriteRequests(randomIntBetween(1, 10)); followerClient().execute(PutFollowAction.INSTANCE, follow).get(); ensureFollowerGreen("follower-index"); atLeastDocsIndexed(followerClient(), "follower-index", between(20, 60)); @@ -971,7 +980,7 @@ public static ResumeFollowAction.Request resumeFollow(String followerIndex) { ResumeFollowAction.Request request = new ResumeFollowAction.Request(); request.setFollowerIndex(followerIndex); request.setMaxRetryDelay(TimeValue.timeValueMillis(10)); - request.setPollTimeout(TimeValue.timeValueMillis(10)); + request.setReadPollTimeout(TimeValue.timeValueMillis(10)); return request; } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java index 9c55617d210ad..6b542d15044e5 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java @@ -56,8 +56,8 @@ public void testAutoFollower() { .numberOfReplicas(0))) .build(); - AutoFollowPattern autoFollowPattern = - new AutoFollowPattern("remote", Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null, null); + AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("logs-*"), + null, null, null, null, null, null, null, null, null, null, null); Map patterns = new HashMap<>(); patterns.put("remote", autoFollowPattern); Map> followedLeaderIndexUUIDS = new HashMap<>(); @@ -120,8 +120,8 @@ public void testAutoFollowerClusterStateApiFailure() { Client client = mock(Client.class); when(client.getRemoteClusterClient(anyString())).thenReturn(client); - AutoFollowPattern autoFollowPattern = - new AutoFollowPattern("remote", Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null, null); + AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("logs-*"), + null, null, null, null, null, null, null, null, null, null, null); Map patterns = new HashMap<>(); patterns.put("remote", autoFollowPattern); Map> followedLeaderIndexUUIDS = new HashMap<>(); @@ -178,8 +178,8 @@ public void testAutoFollowerUpdateClusterStateFailure() { .numberOfReplicas(0))) .build(); - AutoFollowPattern autoFollowPattern = - new AutoFollowPattern("remote", Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null, null); + AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("logs-*"), + null, null, null, null, null, null, null, null, null, null, null); Map patterns = new HashMap<>(); patterns.put("remote", autoFollowPattern); Map> followedLeaderIndexUUIDS = new HashMap<>(); @@ -241,8 +241,8 @@ public void testAutoFollowerCreateAndFollowApiCallFailure() { .numberOfReplicas(0))) .build(); - AutoFollowPattern autoFollowPattern = - new AutoFollowPattern("remote", Collections.singletonList("logs-*"), null, null, null, null, null, null, null, null, null); + AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("logs-*"), + null, null, null, null, null, null, null, null, null, null, null); Map patterns = new HashMap<>(); patterns.put("remote", autoFollowPattern); Map> followedLeaderIndexUUIDS = new HashMap<>(); @@ -295,8 +295,8 @@ void updateAutoFollowMetadata(Function updateFunctio } public void testGetLeaderIndicesToFollow() { - AutoFollowPattern autoFollowPattern = - new AutoFollowPattern("remote", Collections.singletonList("metrics-*"), null, null, null, null, null, null, null, null, null); + AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("metrics-*"), + null, null, null, null, null, null, null, null, null, null, null); Map> headers = new HashMap<>(); ClusterState followerState = ClusterState.builder(new ClusterName("remote")) .metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE, @@ -341,15 +341,15 @@ public void testGetLeaderIndicesToFollow() { public void testGetFollowerIndexName() { AutoFollowPattern autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("metrics-*"), null, null, - null, null, null, null, null, null, null); + null, null, null, null, null, null, null, null, null); assertThat(AutoFollower.getFollowerIndexName(autoFollowPattern, "metrics-0"), equalTo("metrics-0")); autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("metrics-*"), "eu-metrics-0", null, null, - null, null, null, null, null, null); + null, null, null, null, null, null, null, null); assertThat(AutoFollower.getFollowerIndexName(autoFollowPattern, "metrics-0"), equalTo("eu-metrics-0")); autoFollowPattern = new AutoFollowPattern("remote", Collections.singletonList("metrics-*"), "eu-{{leader_index}}", null, - null, null, null, null, null, null, null); + null, null, null, null, null, null, null, null, null); assertThat(AutoFollower.getFollowerIndexName(autoFollowPattern, "metrics-0"), equalTo("eu-metrics-0")); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/GetAutoFollowPatternResponseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/GetAutoFollowPatternResponseTests.java index 301dabeef899a..c74afd6075c95 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/GetAutoFollowPatternResponseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/GetAutoFollowPatternResponseTests.java @@ -33,6 +33,8 @@ protected GetAutoFollowPatternAction.Response createTestInstance() { Collections.singletonList(randomAlphaOfLength(4)), randomAlphaOfLength(4), randomIntBetween(0, Integer.MAX_VALUE), + new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES), + randomIntBetween(0, Integer.MAX_VALUE), randomIntBetween(0, Integer.MAX_VALUE), new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES), randomIntBetween(0, Integer.MAX_VALUE), diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java index e4e365312ad9c..3814e561b42c4 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/PutAutoFollowPatternRequestTests.java @@ -48,13 +48,13 @@ protected PutAutoFollowPatternAction.Request createTestInstance() { request.setFollowIndexNamePattern(randomAlphaOfLength(4)); } if (randomBoolean()) { - request.setPollTimeout(TimeValue.timeValueMillis(500)); + request.setReadPollTimeout(TimeValue.timeValueMillis(500)); } if (randomBoolean()) { request.setMaxRetryDelay(TimeValue.timeValueMillis(500)); } if (randomBoolean()) { - request.setMaxBatchOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); + request.setMaxReadRequestOperationCount(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { request.setMaxConcurrentReadBatches(randomIntBetween(0, Integer.MAX_VALUE)); @@ -63,7 +63,7 @@ protected PutAutoFollowPatternAction.Request createTestInstance() { request.setMaxConcurrentWriteBatches(randomIntBetween(0, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxBatchSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); + request.setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } if (randomBoolean()) { request.setMaxWriteBufferCount(randomIntBetween(0, Integer.MAX_VALUE)); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java index 122082537fd38..ae9bc1bbd3339 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ResumeFollowActionRequestTests.java @@ -45,16 +45,16 @@ static ResumeFollowAction.Request createTestRequest() { ResumeFollowAction.Request request = new ResumeFollowAction.Request(); request.setFollowerIndex(randomAlphaOfLength(4)); if (randomBoolean()) { - request.setMaxBatchOperationCount(randomIntBetween(1, Integer.MAX_VALUE)); + request.setMaxReadRequestOperationCount(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxConcurrentReadBatches(randomIntBetween(1, Integer.MAX_VALUE)); + request.setMaxOutstandingReadRequests(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxConcurrentWriteBatches(randomIntBetween(1, Integer.MAX_VALUE)); + request.setMaxOutstandingWriteRequests(randomIntBetween(1, Integer.MAX_VALUE)); } if (randomBoolean()) { - request.setMaxBatchSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); + request.setMaxReadRequestSize(new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES)); } if (randomBoolean()) { request.setMaxWriteBufferCount(randomIntBetween(1, Integer.MAX_VALUE)); @@ -66,7 +66,7 @@ static ResumeFollowAction.Request createTestRequest() { request.setMaxRetryDelay(TimeValue.timeValueMillis(500)); } if (randomBoolean()) { - request.setPollTimeout(TimeValue.timeValueMillis(500)); + request.setReadPollTimeout(TimeValue.timeValueMillis(500)); } return request; } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java index 8576bc289059a..4df2bb498b69a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTaskRandomTests.java @@ -60,17 +60,17 @@ private void startAndAssertAndStopTask(ShardFollowNodeTask task, TestRun testRun assertThat(status.followerGlobalCheckpoint(), equalTo(testRun.finalExpectedGlobalCheckpoint)); final long numberOfFailedFetches = testRun.responses.values().stream().flatMap(List::stream).filter(f -> f.exception != null).count(); - assertThat(status.numberOfFailedFetches(), equalTo(numberOfFailedFetches)); + assertThat(status.failedReadRequests(), equalTo(numberOfFailedFetches)); // the failures were able to be retried so fetch failures should have cleared - assertThat(status.fetchExceptions().entrySet(), hasSize(0)); - assertThat(status.mappingVersion(), equalTo(testRun.finalMappingVersion)); + assertThat(status.readExceptions().entrySet(), hasSize(0)); + assertThat(status.followerMappingVersion(), equalTo(testRun.finalMappingVersion)); }); task.markAsCompleted(); assertBusy(() -> { ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(0)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.outstandingReadRequests(), equalTo(0)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); }); } @@ -81,8 +81,10 @@ private ShardFollowNodeTask createShardFollowTask(int concurrency, TestRun testR new ShardId("follow_index", "", 0), new ShardId("leader_index", "", 0), testRun.maxOperationCount, + TransportResumeFollowAction.DEFAULT_MAX_READ_REQUEST_SIZE, concurrency, - TransportResumeFollowAction.DEFAULT_MAX_BATCH_SIZE, + testRun.maxOperationCount, + TransportResumeFollowAction.DEFAULT_MAX_READ_REQUEST_SIZE, concurrency, 10240, new ByteSizeValue(512, ByteSizeUnit.MB), 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 93d9556d0e48e..95f8e86e09657 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 @@ -74,23 +74,23 @@ protected void assertEqualInstances(final ShardFollowNodeTaskStatus expectedInst assertThat(newInstance.leaderMaxSeqNo(), equalTo(expectedInstance.leaderMaxSeqNo())); assertThat(newInstance.followerGlobalCheckpoint(), equalTo(expectedInstance.followerGlobalCheckpoint())); assertThat(newInstance.lastRequestedSeqNo(), equalTo(expectedInstance.lastRequestedSeqNo())); - assertThat(newInstance.numberOfConcurrentReads(), equalTo(expectedInstance.numberOfConcurrentReads())); - assertThat(newInstance.numberOfConcurrentWrites(), equalTo(expectedInstance.numberOfConcurrentWrites())); - assertThat(newInstance.numberOfQueuedWrites(), equalTo(expectedInstance.numberOfQueuedWrites())); - assertThat(newInstance.mappingVersion(), equalTo(expectedInstance.mappingVersion())); - assertThat(newInstance.totalFetchTimeMillis(), equalTo(expectedInstance.totalFetchTimeMillis())); - assertThat(newInstance.numberOfSuccessfulFetches(), equalTo(expectedInstance.numberOfSuccessfulFetches())); - assertThat(newInstance.numberOfFailedFetches(), equalTo(expectedInstance.numberOfFailedFetches())); - assertThat(newInstance.operationsReceived(), equalTo(expectedInstance.operationsReceived())); - assertThat(newInstance.totalTransferredBytes(), equalTo(expectedInstance.totalTransferredBytes())); - assertThat(newInstance.totalIndexTimeMillis(), equalTo(expectedInstance.totalIndexTimeMillis())); - assertThat(newInstance.numberOfSuccessfulBulkOperations(), equalTo(expectedInstance.numberOfSuccessfulBulkOperations())); - assertThat(newInstance.numberOfFailedBulkOperations(), equalTo(expectedInstance.numberOfFailedBulkOperations())); - assertThat(newInstance.numberOfOperationsIndexed(), equalTo(expectedInstance.numberOfOperationsIndexed())); - assertThat(newInstance.fetchExceptions().size(), equalTo(expectedInstance.fetchExceptions().size())); - assertThat(newInstance.fetchExceptions().keySet(), equalTo(expectedInstance.fetchExceptions().keySet())); - for (final Map.Entry> entry : newInstance.fetchExceptions().entrySet()) { - final Tuple expectedTuple = expectedInstance.fetchExceptions().get(entry.getKey()); + assertThat(newInstance.outstandingReadRequests(), equalTo(expectedInstance.outstandingReadRequests())); + assertThat(newInstance.outstandingWriteRequests(), equalTo(expectedInstance.outstandingWriteRequests())); + assertThat(newInstance.writeBufferOperationCount(), equalTo(expectedInstance.writeBufferOperationCount())); + assertThat(newInstance.followerMappingVersion(), equalTo(expectedInstance.followerMappingVersion())); + assertThat(newInstance.totalReadTimeMillis(), equalTo(expectedInstance.totalReadTimeMillis())); + assertThat(newInstance.successfulReadRequests(), equalTo(expectedInstance.successfulReadRequests())); + assertThat(newInstance.failedReadRequests(), equalTo(expectedInstance.failedReadRequests())); + assertThat(newInstance.operationsReads(), equalTo(expectedInstance.operationsReads())); + assertThat(newInstance.bytesRead(), equalTo(expectedInstance.bytesRead())); + assertThat(newInstance.totalWriteTimeMillis(), equalTo(expectedInstance.totalWriteTimeMillis())); + assertThat(newInstance.successfulWriteRequests(), equalTo(expectedInstance.successfulWriteRequests())); + assertThat(newInstance.failedWriteRequests(), equalTo(expectedInstance.failedWriteRequests())); + assertThat(newInstance.operationWritten(), equalTo(expectedInstance.operationWritten())); + assertThat(newInstance.readExceptions().size(), equalTo(expectedInstance.readExceptions().size())); + assertThat(newInstance.readExceptions().keySet(), equalTo(expectedInstance.readExceptions().keySet())); + for (final Map.Entry> entry : newInstance.readExceptions().entrySet()) { + final Tuple expectedTuple = expectedInstance.readExceptions().get(entry.getKey()); assertThat(entry.getValue().v1(), equalTo(expectedTuple.v1())); // x-content loses the exception final ElasticsearchException expected = expectedTuple.v2(); @@ -101,7 +101,7 @@ protected void assertEqualInstances(final ShardFollowNodeTaskStatus expectedInst anyOf(instanceOf(ElasticsearchException.class), instanceOf(IllegalStateException.class))); assertThat(entry.getValue().v2().getCause().getMessage(), containsString(expected.getCause().getMessage())); } - assertThat(newInstance.timeSinceLastFetchMillis(), equalTo(expectedInstance.timeSinceLastFetchMillis())); + assertThat(newInstance.timeSinceLastReadMillis(), equalTo(expectedInstance.timeSinceLastReadMillis())); } @Override 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 a1582d4c2f1ae..aeac0ac451806 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 @@ -63,8 +63,12 @@ public class ShardFollowNodeTaskTests extends ESTestCase { private Queue responseSizes; public void testCoordinateReads() { - ShardFollowNodeTask task = createShardFollowTask(8, between(8, 20), between(1, 20), Integer.MAX_VALUE, - new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 8; + params.maxOutstandingReadRequests = between(8, 20); + params.maxOutstandingWriteRequests = between(1, 20); + + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 3, -1); task.coordinateReads(); assertThat(shardChangesRequests, contains(new long[]{0L, 8L})); // treat this a peak request @@ -74,14 +78,17 @@ public void testCoordinateReads() { {6L, 8L}, {14L, 8L}, {22L, 8L}, {30L, 8L}, {38L, 8L}, {46L, 8L}, {54L, 7L}} )); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(7)); + assertThat(status.outstandingReadRequests(), equalTo(7)); assertThat(status.lastRequestedSeqNo(), equalTo(60L)); } public void testMaxWriteBufferCount() { - // Need to set concurrentWrites to 0, other the write buffer gets flushed immediately: - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 0, 32, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 0; // need to set outstandingWrites to 0, other the write buffer gets flushed immediately + params.maxWriteBufferCount = 32; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -95,16 +102,19 @@ public void testMaxWriteBufferCount() { assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because write buffer count limit has been reached ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(0)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.outstandingReadRequests(), equalTo(0)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(128L)); } public void testMaxWriteBufferSize() { - // Need to set concurrentWrites to 0, other the write buffer gets flushed immediately: - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 0, Integer.MAX_VALUE, new ByteSizeValue(1, ByteSizeUnit.KB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 0; // need to set outstandingWrites to 0, other the write buffer gets flushed immediately + params.maxWriteBufferSize = new ByteSizeValue(1, ByteSizeUnit.KB); + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -118,15 +128,18 @@ public void testMaxWriteBufferSize() { assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because write buffer size limit has been reached ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(0)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.outstandingReadRequests(), equalTo(0)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(128L)); } - public void testMaxConcurrentReads() { - ShardFollowNodeTask task = - createShardFollowTask(8, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + public void testMaxOutstandingReads() { + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 8; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 64, -1); task.coordinateReads(); @@ -135,13 +148,16 @@ public void testMaxConcurrentReads() { assertThat(shardChangesRequests.get(0)[1], equalTo(8L)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); + assertThat(status.outstandingReadRequests(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(7L)); } public void testTaskCancelled() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 64, -1); task.coordinateReads(); @@ -158,8 +174,11 @@ public void testTaskCancelled() { } public void testTaskCancelledAfterReadLimitHasBeenReached() { - ShardFollowNodeTask task = - createShardFollowTask(16, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 16; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 31, -1); task.coordinateReads(); @@ -175,16 +194,21 @@ public void testTaskCancelledAfterReadLimitHasBeenReached() { assertThat(bulkShardOperationRequests.size(), equalTo(0)); // no more writes, because task has been cancelled ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(0)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.outstandingReadRequests(), equalTo(0)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(15L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(31L)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } public void testTaskCancelledAfterWriteBufferLimitHasBeenReached() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, 32, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + params.maxWriteBufferCount = 32; + + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 64, -1); task.coordinateReads(); @@ -200,16 +224,19 @@ public void testTaskCancelledAfterWriteBufferLimitHasBeenReached() { assertThat(bulkShardOperationRequests.size(), equalTo(0)); // no more writes, because task has been cancelled ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(0)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.outstandingReadRequests(), equalTo(0)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(128L)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } public void testReceiveRetryableError() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); int max = randomIntBetween(1, 30); @@ -224,10 +251,10 @@ public void testReceiveRetryableError() { final AtomicLong retryCounter = new AtomicLong(); // before each retry, we assert the fetch failures; after the last retry, the fetch failure should clear beforeSendShardChangesRequest = status -> { - assertThat(status.numberOfFailedFetches(), equalTo(retryCounter.get())); + assertThat(status.failedReadRequests(), equalTo(retryCounter.get())); if (retryCounter.get() > 0) { - assertThat(status.fetchExceptions().entrySet(), hasSize(1)); - final Map.Entry> entry = status.fetchExceptions().entrySet().iterator().next(); + assertThat(status.readExceptions().entrySet(), hasSize(1)); + final Map.Entry> entry = status.readExceptions().entrySet().iterator().next(); assertThat(entry.getValue().v1(), equalTo(Math.toIntExact(retryCounter.get()))); assertThat(entry.getKey(), equalTo(0L)); assertThat(entry.getValue().v2(), instanceOf(ShardNotFoundException.class)); @@ -248,19 +275,22 @@ public void testReceiveRetryableError() { assertFalse("task is not stopped", task.isStopped()); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); - assertThat(status.numberOfFailedFetches(), equalTo((long)max)); - assertThat(status.numberOfSuccessfulFetches(), equalTo(1L)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); + assertThat(status.failedReadRequests(), equalTo((long)max)); + assertThat(status.successfulReadRequests(), equalTo(1L)); // the fetch failure has cleared - assertThat(status.fetchExceptions().entrySet(), hasSize(0)); + assertThat(status.readExceptions().entrySet(), hasSize(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); } public void testEmptyShardChangesResponseShouldClearFetchException() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, -1, -1); readFailures.add(new ShardNotFoundException(new ShardId("leader_index", "", 0))); @@ -279,18 +309,21 @@ public void testEmptyShardChangesResponseShouldClearFetchException() { assertFalse("task is not stopped", task.isStopped()); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); - assertThat(status.numberOfFailedFetches(), equalTo(1L)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); + assertThat(status.failedReadRequests(), equalTo(1L)); // the fetch failure should have been cleared: - assertThat(status.fetchExceptions().entrySet(), hasSize(0)); + assertThat(status.readExceptions().entrySet(), hasSize(0)); assertThat(status.lastRequestedSeqNo(), equalTo(-1L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(-1L)); } public void testReceiveTimeout() { - final ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + final ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); final int numberOfTimeouts = randomIntBetween(1, 32); @@ -304,14 +337,14 @@ public void testReceiveTimeout() { final AtomicInteger counter = new AtomicInteger(); beforeSendShardChangesRequest = status -> { if (counter.get() <= numberOfTimeouts) { - assertThat(status.numberOfSuccessfulFetches(), equalTo(0L)); - assertThat(status.totalFetchTimeMillis(), equalTo(0L)); - assertThat(status.operationsReceived(), equalTo(0L)); - assertThat(status.totalTransferredBytes(), equalTo(0L)); - - assertThat(status.fetchExceptions().entrySet(), hasSize(0)); - assertThat(status.totalFetchTimeMillis(), equalTo(0L)); - assertThat(status.numberOfFailedFetches(), equalTo(0L)); + assertThat(status.successfulReadRequests(), equalTo(0L)); + assertThat(status.totalReadTimeMillis(), equalTo(0L)); + assertThat(status.operationsReads(), equalTo(0L)); + assertThat(status.bytesRead(), equalTo(0L)); + + assertThat(status.readExceptions().entrySet(), hasSize(0)); + assertThat(status.totalReadTimeMillis(), equalTo(0L)); + assertThat(status.failedReadRequests(), equalTo(0L)); } else { // otherwise we will keep looping as if we were repeatedly polling and timing out simulateResponse.set(false); @@ -343,10 +376,10 @@ public void testReceiveTimeout() { assertThat(lastShardChangesRequest[1], equalTo(64L)); final ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfSuccessfulFetches(), equalTo(1L)); - assertThat(status.numberOfFailedFetches(), equalTo(0L)); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.successfulReadRequests(), equalTo(1L)); + assertThat(status.failedReadRequests(), equalTo(0L)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); assertThat(status.leaderMaxSeqNo(), equalTo(63L)); @@ -354,8 +387,11 @@ public void testReceiveTimeout() { } public void testReceiveNonRetryableError() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); Exception failure = new RuntimeException("replication failed"); @@ -364,8 +400,8 @@ public void testReceiveNonRetryableError() { // since there will be only one failure, this should only be invoked once and there should not be a fetch failure beforeSendShardChangesRequest = status -> { if (invoked.compareAndSet(false, true)) { - assertThat(status.numberOfFailedFetches(), equalTo(0L)); - assertThat(status.fetchExceptions().entrySet(), hasSize(0)); + assertThat(status.failedReadRequests(), equalTo(0L)); + assertThat(status.readExceptions().entrySet(), hasSize(0)); } else { fail("invoked twice"); } @@ -379,11 +415,11 @@ public void testReceiveNonRetryableError() { assertTrue("task is stopped", task.isStopped()); assertThat(task.getStatus().getFatalException().getRootCause(), sameInstance(failure)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); - assertThat(status.numberOfFailedFetches(), equalTo(1L)); - assertThat(status.fetchExceptions().entrySet(), hasSize(1)); - final Map.Entry> entry = status.fetchExceptions().entrySet().iterator().next(); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); + assertThat(status.failedReadRequests(), equalTo(1L)); + assertThat(status.readExceptions().entrySet(), hasSize(1)); + final Map.Entry> entry = status.readExceptions().entrySet().iterator().next(); assertThat(entry.getKey(), equalTo(0L)); assertThat(entry.getValue().v2(), instanceOf(ElasticsearchException.class)); assertNotNull(entry.getValue().v2().getCause()); @@ -395,8 +431,11 @@ public void testReceiveNonRetryableError() { } public void testHandleReadResponse() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -407,18 +446,21 @@ public void testHandleReadResponse() { assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.mappingVersion(), equalTo(0L)); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.followerMappingVersion(), equalTo(0L)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } public void testReceiveLessThanRequested() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -435,15 +477,18 @@ public void testReceiveLessThanRequested() { assertThat(shardChangesRequests.get(0)[1], equalTo(43L)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); } public void testCancelAndReceiveLessThanRequested() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -459,15 +504,18 @@ public void testCancelAndReceiveLessThanRequested() { assertThat(shardChangesRequests.size(), equalTo(0)); assertThat(bulkShardOperationRequests.size(), equalTo(0)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(0)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.outstandingReadRequests(), equalTo(0)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); } public void testReceiveNothingExpectedSomething() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -483,15 +531,18 @@ public void testReceiveNothingExpectedSomething() { assertThat(shardChangesRequests.get(0)[1], equalTo(64L)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); } public void testMappingUpdate() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); mappingVersions.add(1L); @@ -503,17 +554,20 @@ public void testMappingUpdate() { assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.mappingVersion(), equalTo(1L)); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.followerMappingVersion(), equalTo(1L)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } public void testMappingUpdateRetryableError() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); int max = randomIntBetween(1, 30); @@ -529,17 +583,20 @@ public void testMappingUpdateRetryableError() { assertThat(bulkShardOperationRequests.size(), equalTo(1)); assertThat(task.isStopped(), equalTo(false)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.mappingVersion(), equalTo(1L)); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.followerMappingVersion(), equalTo(1L)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); } public void testMappingUpdateNonRetryableError() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); mappingUpdateFailures.add(new RuntimeException()); @@ -550,22 +607,27 @@ public void testMappingUpdateNonRetryableError() { assertThat(bulkShardOperationRequests.size(), equalTo(0)); assertThat(task.isStopped(), equalTo(true)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.mappingVersion(), equalTo(0L)); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(0)); + assertThat(status.followerMappingVersion(), equalTo(0L)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(0)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); } public void testCoordinateWrites() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 128; + params.maxOutstandingReadRequests = 1; + params.maxWriteRequestOperationCount = 64; + params.maxOutstandingWriteRequests = 1; + + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); assertThat(shardChangesRequests.size(), equalTo(1)); assertThat(shardChangesRequests.get(0)[0], equalTo(0L)); - assertThat(shardChangesRequests.get(0)[1], equalTo(64L)); + assertThat(shardChangesRequests.get(0)[1], equalTo(128L)); ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 63L); // Also invokes coordinatesWrites() @@ -575,16 +637,18 @@ public void testCoordinateWrites() { assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.outstandingReadRequests(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } - public void testMaxConcurrentWrites() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 2, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + public void testMaxOutstandingWrites() { + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxWriteRequestOperationCount = 64; + params.maxOutstandingWriteRequests = 2; + ShardFollowNodeTask task = createShardFollowTask(params); ShardChangesAction.Response response = generateShardChangesResponse(0, 256, 0L, 256L); // Also invokes coordinatesWrites() task.innerHandleReadResponse(0L, 64L, response); @@ -594,9 +658,10 @@ public void testMaxConcurrentWrites() { assertThat(bulkShardOperationRequests.get(1), equalTo(Arrays.asList(response.getOperations()).subList(64, 128))); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentWrites(), equalTo(2)); + assertThat(status.outstandingWriteRequests(), equalTo(2)); - task = createShardFollowTask(64, 1, 4, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + params.maxOutstandingWriteRequests = 4; // change to 4 outstanding writers + task = createShardFollowTask(params); response = generateShardChangesResponse(0, 256, 0L, 256L); // Also invokes coordinatesWrites() task.innerHandleReadResponse(0L, 64L, response); @@ -608,12 +673,14 @@ public void testMaxConcurrentWrites() { assertThat(bulkShardOperationRequests.get(3), equalTo(Arrays.asList(response.getOperations()).subList(192, 256))); status = task.getStatus(); - assertThat(status.numberOfConcurrentWrites(), equalTo(4)); + assertThat(status.outstandingWriteRequests(), equalTo(4)); } - public void testMaxBatchOperationCount() { - ShardFollowNodeTask task = - createShardFollowTask(8, 1, 32, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + public void testMaxWriteRequestCount() { + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxWriteRequestOperationCount = 8; + params.maxOutstandingWriteRequests = 32; + ShardFollowNodeTask task = createShardFollowTask(params); ShardChangesAction.Response response = generateShardChangesResponse(0, 256, 0L, 256L); // Also invokes coordinatesWrites() task.innerHandleReadResponse(0L, 64L, response); @@ -625,12 +692,15 @@ public void testMaxBatchOperationCount() { } ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentWrites(), equalTo(32)); + assertThat(status.outstandingWriteRequests(), equalTo(32)); } public void testRetryableError() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -653,13 +723,16 @@ public void testRetryableError() { } assertThat(task.isStopped(), equalTo(false)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } public void testNonRetryableError() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -676,13 +749,18 @@ public void testNonRetryableError() { assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations()))); assertThat(task.isStopped(), equalTo(true)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentWrites(), equalTo(1)); + assertThat(status.outstandingWriteRequests(), equalTo(1)); assertThat(status.followerGlobalCheckpoint(), equalTo(-1L)); } - public void testMaxBatchBytesLimit() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 128, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), 1L); + public void testMaxWriteRequestSize() { + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxWriteRequestSize = new ByteSizeValue(1, ByteSizeUnit.BYTES); + params.maxOutstandingWriteRequests = 128; + + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 64, -1); task.coordinateReads(); @@ -698,8 +776,12 @@ public void testMaxBatchBytesLimit() { } public void testHandleWriteResponse() { - ShardFollowNodeTask task = - createShardFollowTask(64, 1, 1, Integer.MAX_VALUE, new ByteSizeValue(512, ByteSizeUnit.MB), Long.MAX_VALUE); + ShardFollowTaskParams params = new ShardFollowTaskParams(); + params.maxReadRequestOperationCount = 64; + params.maxOutstandingReadRequests = 1; + params.maxWriteRequestOperationCount = 64; + params.maxOutstandingWriteRequests = 1; + ShardFollowNodeTask task = createShardFollowTask(params); startTask(task, 63, -1); task.coordinateReads(); @@ -722,7 +804,7 @@ public void testHandleWriteResponse() { assertThat(shardChangesRequests.get(0)[1], equalTo(64L)); ShardFollowNodeTaskStatus status = task.getStatus(); - assertThat(status.numberOfConcurrentReads(), equalTo(1)); + assertThat(status.outstandingReadRequests(), equalTo(1)); assertThat(status.lastRequestedSeqNo(), equalTo(63L)); assertThat(status.leaderGlobalCheckpoint(), equalTo(63L)); assertThat(status.followerGlobalCheckpoint(), equalTo(63L)); @@ -742,26 +824,40 @@ public void testComputeDelay() { assertThat(ShardFollowNodeTask.computeDelay(1024, maxDelayInMillis), allOf(greaterThanOrEqualTo(0L), lessThanOrEqualTo(1000L))); } - private ShardFollowNodeTask createShardFollowTask(int maxBatchOperationCount, - int maxConcurrentReadBatches, - int maxConcurrentWriteBatches, - int maxWriteBufferCount, - ByteSizeValue maxWriteBufferSize, - long maxBatchSizeInBytes) { + static final class ShardFollowTaskParams { + private String remoteCluster = null; + private ShardId followShardId = new ShardId("follow_index", "", 0); + private ShardId leaderShardId = new ShardId("leader_index", "", 0); + private int maxReadRequestOperationCount = Integer.MAX_VALUE; + private ByteSizeValue maxReadRequestSize = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); + private int maxOutstandingReadRequests = Integer.MAX_VALUE; + private int maxWriteRequestOperationCount = Integer.MAX_VALUE; + private ByteSizeValue maxWriteRequestSize = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); + private int maxOutstandingWriteRequests = Integer.MAX_VALUE; + private int maxWriteBufferCount = Integer.MAX_VALUE; + private ByteSizeValue maxWriteBufferSize = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); + private TimeValue maxRetryDelay = TimeValue.ZERO; + private TimeValue readPollTimeout = TimeValue.ZERO; + private Map headers = Collections.emptyMap(); + } + + private ShardFollowNodeTask createShardFollowTask(ShardFollowTaskParams params) { AtomicBoolean stopped = new AtomicBoolean(false); - ShardFollowTask params = new ShardFollowTask( - null, - new ShardId("follow_index", "", 0), - new ShardId("leader_index", "", 0), - maxBatchOperationCount, - maxConcurrentReadBatches, - new ByteSizeValue(maxBatchSizeInBytes, ByteSizeUnit.BYTES), - maxConcurrentWriteBatches, - maxWriteBufferCount, - maxWriteBufferSize, - TimeValue.ZERO, - TimeValue.ZERO, - Collections.emptyMap() + ShardFollowTask followTask = new ShardFollowTask( + params.remoteCluster, + params.followShardId, + params.leaderShardId, + params.maxReadRequestOperationCount, + params.maxReadRequestSize, + params.maxOutstandingReadRequests, + params.maxWriteRequestOperationCount, + params.maxWriteRequestSize, + params.maxOutstandingWriteRequests, + params.maxWriteBufferCount, + params.maxWriteBufferSize, + params.maxRetryDelay, + params.readPollTimeout, + params.headers ); shardChangesRequests = new ArrayList<>(); @@ -775,7 +871,7 @@ private ShardFollowNodeTask createShardFollowTask(int maxBatchOperationCount, maxSeqNos = new LinkedList<>(); responseSizes = new LinkedList<>(); return new ShardFollowNodeTask( - 1L, "type", ShardFollowTask.NAME, "description", null, Collections.emptyMap(), params, scheduler, System::nanoTime) { + 1L, "type", ShardFollowTask.NAME, "description", null, Collections.emptyMap(), followTask, scheduler, System::nanoTime) { @Override protected void innerUpdateMapping(LongConsumer handler, Consumer errorHandler) { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java index 07c3121eba4da..d2f09c3900dfd 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java @@ -368,7 +368,9 @@ private ShardFollowNodeTask createShardFollowTask(ReplicationGroup leaderGroup, new ShardId("follow_index", "", 0), new ShardId("leader_index", "", 0), between(1, 64), + new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES), between(1, 8), + between(1, 64), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES), between(1, 4), 10240, @@ -435,7 +437,7 @@ protected void innerSendShardChangesRequest(long from, int maxOperationCount, Co return; } Translog.Operation[] ops = ShardChangesAction.getOperations(indexShard, seqNoStats.getGlobalCheckpoint(), from, - maxOperationCount, recordedLeaderIndexHistoryUUID, params.getMaxBatchSize()); + maxOperationCount, recordedLeaderIndexHistoryUUID, params.getMaxReadRequestSize()); // hard code mapping version; this is ok, as mapping updates are not tested here final ShardChangesAction.Response response = new ShardChangesAction.Response( 1L, 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 e955f77d73336..1dfe4a9897075 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 @@ -30,6 +30,8 @@ protected ShardFollowTask createTestInstance() { new ShardId(randomAlphaOfLength(4), randomAlphaOfLength(4), randomInt(5)), new ShardId(randomAlphaOfLength(4), randomAlphaOfLength(4), randomInt(5)), randomIntBetween(1, Integer.MAX_VALUE), + new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES), + randomIntBetween(1, Integer.MAX_VALUE), randomIntBetween(1, Integer.MAX_VALUE), new ByteSizeValue(randomNonNegativeLong(), ByteSizeUnit.BYTES), randomIntBetween(1, Integer.MAX_VALUE), diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java index dd144840f383e..3f6b2fcb0e3ef 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportDeleteAutoFollowPatternActionTests.java @@ -33,7 +33,7 @@ public void testInnerDelete() { List existingPatterns = new ArrayList<>(); existingPatterns.add("transactions-*"); existingAutoFollowPatterns.put("name1", - new AutoFollowPattern("eu_cluster", existingPatterns, null, null, null, null, null, null, null, null, null)); + new AutoFollowPattern("eu_cluster", existingPatterns, null, null, null, null, null, null, null, null, null, null, null)); List existingUUIDS = new ArrayList<>(); existingUUIDS.add("_val"); @@ -44,7 +44,7 @@ public void testInnerDelete() { List existingPatterns = new ArrayList<>(); existingPatterns.add("logs-*"); existingAutoFollowPatterns.put("name2", - new AutoFollowPattern("asia_cluster", existingPatterns, null, null, null, null, null, null, null, null, null)); + new AutoFollowPattern("asia_cluster", existingPatterns, null, null, null, null, null, null, null, null, null, null, null)); List existingUUIDS = new ArrayList<>(); existingUUIDS.add("_val"); @@ -78,7 +78,7 @@ public void testInnerDeleteDoesNotExist() { List existingPatterns = new ArrayList<>(); existingPatterns.add("transactions-*"); existingAutoFollowPatterns.put("name1", - new AutoFollowPattern("eu_cluster", existingPatterns, null, null, null, null, null, null, null, null, null)); + new AutoFollowPattern("eu_cluster", existingPatterns, null, null, null, null, null, null, null, null, null, null, null)); existingHeaders.put("key", Collections.singletonMap("key", "val")); } ClusterState clusterState = ClusterState.builder(new ClusterName("us_cluster")) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportGetAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportGetAutoFollowPatternActionTests.java index 128474bbc30a6..e2c7f327ab942 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportGetAutoFollowPatternActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportGetAutoFollowPatternActionTests.java @@ -24,9 +24,9 @@ public class TransportGetAutoFollowPatternActionTests extends ESTestCase { public void testGetAutoFollowPattern() { Map patterns = new HashMap<>(); patterns.put("name1", new AutoFollowPattern( - "test_alias1", Collections.singletonList("index-*"), null, null, null, null, null, null, null, null, null)); + "test_alias1", Collections.singletonList("index-*"), null, null, null, null, null, null, null, null, null, null, null)); patterns.put("name2", new AutoFollowPattern( - "test_alias1", Collections.singletonList("index-*"), null, null, null, null, null, null, null, null, null)); + "test_alias1", Collections.singletonList("index-*"), null, null, null, null, null, null, null, null, null, null, null)); MetaData metaData = MetaData.builder() .putCustom(AutoFollowMetadata.TYPE, new AutoFollowMetadata(patterns, Collections.emptyMap(), Collections.emptyMap())) .build(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java index 433ef402af856..ac556d47c85dd 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportPutAutoFollowPatternActionTests.java @@ -103,7 +103,7 @@ public void testInnerPut_existingLeaderIndicesAndAutoFollowMetadata() { List existingPatterns = new ArrayList<>(); existingPatterns.add("transactions-*"); existingAutoFollowPatterns.put("name1", - new AutoFollowPattern("eu_cluster", existingPatterns, null, null, null, null, null, null, null, null, null)); + new AutoFollowPattern("eu_cluster", existingPatterns, null, null, null, null, null, null, null, null, null, null, null)); Map> existingAlreadyFollowedIndexUUIDS = new HashMap<>(); List existingUUIDS = new ArrayList<>(); existingUUIDS.add("_val"); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowActionTests.java index 82cbe2622b7f7..4a201e37355a9 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowActionTests.java @@ -79,8 +79,10 @@ public void testUnfollowRunningShardFollowTasks() { new ShardId("follow_index", "", 0), new ShardId("leader_index", "", 0), 1024, + TransportResumeFollowAction.DEFAULT_MAX_READ_REQUEST_SIZE, 1, - TransportResumeFollowAction.DEFAULT_MAX_BATCH_SIZE, + 1024, + TransportResumeFollowAction.DEFAULT_MAX_READ_REQUEST_SIZE, 1, 10240, new ByteSizeValue(512, ByteSizeUnit.MB), diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/monitoring/collector/ccr/FollowStatsMonitoringDocTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/monitoring/collector/ccr/FollowStatsMonitoringDocTests.java index 9b4ed7c8a97d0..f3e0c2d5bd7b3 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/monitoring/collector/ccr/FollowStatsMonitoringDocTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/monitoring/collector/ccr/FollowStatsMonitoringDocTests.java @@ -91,24 +91,24 @@ public void testToXContent() throws IOException { final long lastRequestedSeqNo = randomNonNegativeLong(); final int numberOfConcurrentReads = randomIntBetween(1, Integer.MAX_VALUE); final int numberOfConcurrentWrites = randomIntBetween(1, Integer.MAX_VALUE); - final int numberOfQueuedWrites = randomIntBetween(0, Integer.MAX_VALUE); - final long bufferSize = randomNonNegativeLong(); - final long mappingVersion = randomIntBetween(0, Integer.MAX_VALUE); - final long totalFetchTimeMillis = randomLongBetween(0, 4096); - final long totalFetchTookTimeMillis = randomLongBetween(0, 4096); - final long numberOfSuccessfulFetches = randomNonNegativeLong(); - final long numberOfFailedFetches = randomLongBetween(0, 8); - final long operationsReceived = randomNonNegativeLong(); - final long totalTransferredBytes = randomNonNegativeLong(); - final long totalIndexTimeMillis = randomNonNegativeLong(); - final long numberOfSuccessfulBulkOperations = randomNonNegativeLong(); - final long numberOfFailedBulkOperations = randomNonNegativeLong(); - final long numberOfOperationsIndexed = randomNonNegativeLong(); + final int writeBufferOperationCount = randomIntBetween(0, Integer.MAX_VALUE); + final long writeBufferSizeInBytes = randomNonNegativeLong(); + final long followerMappingVersion = randomIntBetween(0, Integer.MAX_VALUE); + final long totalReadTimeMillis = randomLongBetween(0, 4096); + final long totalReadRemoteExecTimeMillis = randomLongBetween(0, 4096); + final long successfulReadRequests = randomNonNegativeLong(); + final long failedReadRequests = randomLongBetween(0, 8); + final long operationsRead = randomNonNegativeLong(); + final long bytesRead = randomNonNegativeLong(); + final long totalWriteTimeMillis = randomNonNegativeLong(); + final long successfulWriteRequests = randomNonNegativeLong(); + final long failedWriteRequests = randomNonNegativeLong(); + final long operationWritten = randomNonNegativeLong(); final NavigableMap> fetchExceptions = new TreeMap<>(Collections.singletonMap( randomNonNegativeLong(), Tuple.tuple(randomIntBetween(0, Integer.MAX_VALUE), new ElasticsearchException("shard is sad")))); - final long timeSinceLastFetchMillis = randomNonNegativeLong(); + final long timeSinceLastReadMillis = randomNonNegativeLong(); final ShardFollowNodeTaskStatus status = new ShardFollowNodeTaskStatus( "leader_cluster", "leader_index", @@ -121,21 +121,21 @@ public void testToXContent() throws IOException { lastRequestedSeqNo, numberOfConcurrentReads, numberOfConcurrentWrites, - numberOfQueuedWrites, - bufferSize, - mappingVersion, - totalFetchTimeMillis, - totalFetchTookTimeMillis, - numberOfSuccessfulFetches, - numberOfFailedFetches, - operationsReceived, - totalTransferredBytes, - totalIndexTimeMillis, - numberOfSuccessfulBulkOperations, - numberOfFailedBulkOperations, - numberOfOperationsIndexed, + writeBufferOperationCount, + writeBufferSizeInBytes, + followerMappingVersion, + totalReadTimeMillis, + totalReadRemoteExecTimeMillis, + successfulReadRequests, + failedReadRequests, + operationsRead, + bytesRead, + totalWriteTimeMillis, + successfulWriteRequests, + failedWriteRequests, + operationWritten, fetchExceptions, - timeSinceLastFetchMillis, + timeSinceLastReadMillis, new ElasticsearchException("fatal error")); final FollowStatsMonitoringDoc document = new FollowStatsMonitoringDoc("_cluster", timestamp, intervalMillis, node, status); final BytesReference xContent = XContentHelper.toXContent(document, XContentType.JSON, false); @@ -165,22 +165,22 @@ public void testToXContent() throws IOException { + "\"follower_global_checkpoint\":" + followerGlobalCheckpoint + "," + "\"follower_max_seq_no\":" + followerMaxSeqNo + "," + "\"last_requested_seq_no\":" + lastRequestedSeqNo + "," - + "\"number_of_concurrent_reads\":" + numberOfConcurrentReads + "," - + "\"number_of_concurrent_writes\":" + numberOfConcurrentWrites + "," - + "\"number_of_queued_writes\":" + numberOfQueuedWrites + "," - + "\"buffer_size_in_bytes\":" + bufferSize + "," - + "\"mapping_version\":" + mappingVersion + "," - + "\"total_fetch_time_millis\":" + totalFetchTimeMillis + "," - + "\"total_fetch_remote_time_millis\":" + totalFetchTookTimeMillis + "," - + "\"number_of_successful_fetches\":" + numberOfSuccessfulFetches + "," - + "\"number_of_failed_fetches\":" + numberOfFailedFetches + "," - + "\"operations_received\":" + operationsReceived + "," - + "\"total_transferred_bytes\":" + totalTransferredBytes + "," - + "\"total_index_time_millis\":" + totalIndexTimeMillis +"," - + "\"number_of_successful_bulk_operations\":" + numberOfSuccessfulBulkOperations + "," - + "\"number_of_failed_bulk_operations\":" + numberOfFailedBulkOperations + "," - + "\"number_of_operations_indexed\":" + numberOfOperationsIndexed + "," - + "\"fetch_exceptions\":[" + + "\"outstanding_read_requests\":" + numberOfConcurrentReads + "," + + "\"outstanding_write_requests\":" + numberOfConcurrentWrites + "," + + "\"write_buffer_operation_count\":" + writeBufferOperationCount + "," + + "\"write_buffer_size_in_bytes\":" + writeBufferSizeInBytes + "," + + "\"follower_mapping_version\":" + followerMappingVersion + "," + + "\"total_read_time_millis\":" + totalReadTimeMillis + "," + + "\"total_read_remote_exec_time_millis\":" + totalReadRemoteExecTimeMillis + "," + + "\"successful_read_requests\":" + successfulReadRequests + "," + + "\"failed_read_requests\":" + failedReadRequests + "," + + "\"operations_read\":" + operationsRead + "," + + "\"bytes_read\":" + bytesRead + "," + + "\"total_write_time_millis\":" + totalWriteTimeMillis +"," + + "\"successful_write_requests\":" + successfulWriteRequests + "," + + "\"failed_write_requests\":" + failedWriteRequests + "," + + "\"operations_written\":" + operationWritten + "," + + "\"read_exceptions\":[" + "{" + "\"from_seq_no\":" + fetchExceptions.keySet().iterator().next() + "," + "\"retries\":" + fetchExceptions.values().iterator().next().v1() + "," @@ -190,7 +190,7 @@ public void testToXContent() throws IOException { + "}" + "}" + "]," - + "\"time_since_last_fetch_millis\":" + timeSinceLastFetchMillis + "," + + "\"time_since_last_read_millis\":" + timeSinceLastReadMillis + "," + "\"fatal_exception\":{\"type\":\"exception\",\"reason\":\"fatal error\"}" + "}" + "}")); @@ -250,7 +250,7 @@ public void testShardFollowNodeTaskStatusFieldsMapped() throws IOException { anyOf(equalTo("keyword"), equalTo("text"))); } else { // Manual test specific object fields and if not just fail: - if (fieldName.equals("fetch_exceptions")) { + if (fieldName.equals("read_exceptions")) { assertThat(fieldType, equalTo("nested")); assertThat(((Map) fieldMapping.get("properties")).size(), equalTo(3)); assertThat(XContentMapValues.extractValue("properties.from_seq_no.type", fieldMapping), equalTo("long")); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java index 8172612b78ff4..379dbe7a421b6 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowMetadata.java @@ -178,34 +178,42 @@ public static class AutoFollowPattern implements Writeable, ToXContentObject { public static final ParseField REMOTE_CLUSTER_FIELD = new ParseField("remote_cluster"); public static final ParseField LEADER_PATTERNS_FIELD = new ParseField("leader_index_patterns"); public static final ParseField FOLLOW_PATTERN_FIELD = new ParseField("follow_index_pattern"); - public static final ParseField MAX_BATCH_OPERATION_COUNT = new ParseField("max_batch_operation_count"); - public static final ParseField MAX_CONCURRENT_READ_BATCHES = new ParseField("max_concurrent_read_batches"); - public static final ParseField MAX_BATCH_SIZE = new ParseField("max_batch_size"); - public static final ParseField MAX_CONCURRENT_WRITE_BATCHES = new ParseField("max_concurrent_write_batches"); + public static final ParseField MAX_READ_REQUEST_OPERATION_COUNT = new ParseField("max_read_request_operation_count"); + public static final ParseField MAX_READ_REQUEST_SIZE = new ParseField("max_read_request_size"); + public static final ParseField MAX_OUTSTANDING_READ_REQUESTS = new ParseField("max_outstanding_read_requests"); + public static final ParseField MAX_WRITE_REQUEST_OPERATION_COUNT = new ParseField("max_write_request_operation_count"); + public static final ParseField MAX_WRITE_REQUEST_SIZE = new ParseField("max_write_request_size"); + public static final ParseField MAX_OUTSTANDING_WRITE_REQUESTS = new ParseField("max_outstanding_write_requests"); public static final ParseField MAX_WRITE_BUFFER_COUNT = new ParseField("max_write_buffer_count"); public static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); public static final ParseField MAX_RETRY_DELAY = new ParseField("max_retry_delay"); - public static final ParseField POLL_TIMEOUT = new ParseField("poll_timeout"); + public static final ParseField READ_POLL_TIMEOUT = new ParseField("read_poll_timeout"); @SuppressWarnings("unchecked") private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("auto_follow_pattern", args -> new AutoFollowPattern((String) args[0], (List) args[1], (String) args[2], (Integer) args[3], - (Integer) args[4], (ByteSizeValue) args[5], (Integer) args[6], (Integer) args[7], (ByteSizeValue) args[8], - (TimeValue) args[9], (TimeValue) args[10])); + (ByteSizeValue) args[4], (Integer) args[5], (Integer) args[6], (ByteSizeValue) args[7], (Integer) args[8], + (Integer) args[9], (ByteSizeValue) args[10], (TimeValue) args[11], (TimeValue) args[12])); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), REMOTE_CLUSTER_FIELD); PARSER.declareStringArray(ConstructingObjectParser.constructorArg(), LEADER_PATTERNS_FIELD); PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), FOLLOW_PATTERN_FIELD); - PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_BATCH_OPERATION_COUNT); - PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_CONCURRENT_READ_BATCHES); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_READ_REQUEST_OPERATION_COUNT); PARSER.declareField( ConstructingObjectParser.optionalConstructorArg(), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_BATCH_SIZE.getPreferredName()), - MAX_BATCH_SIZE, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_READ_REQUEST_SIZE.getPreferredName()), + MAX_READ_REQUEST_SIZE, ObjectParser.ValueType.STRING); - PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_CONCURRENT_WRITE_BATCHES); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_OUTSTANDING_READ_REQUESTS); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_WRITE_REQUEST_OPERATION_COUNT); + PARSER.declareField( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_REQUEST_SIZE.getPreferredName()), + MAX_WRITE_REQUEST_SIZE, + ObjectParser.ValueType.STRING); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_OUTSTANDING_WRITE_REQUESTS); PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAX_WRITE_BUFFER_COUNT); PARSER.declareField( ConstructingObjectParser.optionalConstructorArg(), @@ -216,17 +224,19 @@ public static class AutoFollowPattern implements Writeable, ToXContentObject { (p, c) -> TimeValue.parseTimeValue(p.text(), MAX_RETRY_DELAY.getPreferredName()), MAX_RETRY_DELAY, ObjectParser.ValueType.STRING); PARSER.declareField(ConstructingObjectParser.optionalConstructorArg(), - (p, c) -> TimeValue.parseTimeValue(p.text(), POLL_TIMEOUT.getPreferredName()), - POLL_TIMEOUT, ObjectParser.ValueType.STRING); + (p, c) -> TimeValue.parseTimeValue(p.text(), READ_POLL_TIMEOUT.getPreferredName()), + READ_POLL_TIMEOUT, ObjectParser.ValueType.STRING); } private final String remoteCluster; private final List leaderIndexPatterns; private final String followIndexPattern; - private final Integer maxBatchOperationCount; - private final Integer maxConcurrentReadBatches; - private final ByteSizeValue maxBatchSize; - private final Integer maxConcurrentWriteBatches; + private final Integer maxReadRequestOperationCount; + private final ByteSizeValue maxReadRequestSize; + private final Integer maxOutstandingReadRequests; + private final Integer maxWriteRequestOperationCount; + private final ByteSizeValue maxWriteRequestSize; + private final Integer maxOutstandingWriteRequests; private final Integer maxWriteBufferCount; private final ByteSizeValue maxWriteBufferSize; private final TimeValue maxRetryDelay; @@ -235,20 +245,24 @@ public static class AutoFollowPattern implements Writeable, ToXContentObject { public AutoFollowPattern(String remoteCluster, List leaderIndexPatterns, String followIndexPattern, - Integer maxBatchOperationCount, - Integer maxConcurrentReadBatches, - ByteSizeValue maxBatchSize, - Integer maxConcurrentWriteBatches, + Integer maxReadRequestOperationCount, + ByteSizeValue maxReadRequestSize, + Integer maxOutstandingReadRequests, + Integer maxWriteRequestOperationCount, + ByteSizeValue maxWriteRequestSize, + Integer maxOutstandingWriteRequests, Integer maxWriteBufferCount, ByteSizeValue maxWriteBufferSize, TimeValue maxRetryDelay, TimeValue pollTimeout) { this.remoteCluster = remoteCluster; this.leaderIndexPatterns = leaderIndexPatterns; this.followIndexPattern = followIndexPattern; - this.maxBatchOperationCount = maxBatchOperationCount; - this.maxConcurrentReadBatches = maxConcurrentReadBatches; - this.maxBatchSize = maxBatchSize; - this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; + this.maxReadRequestOperationCount = maxReadRequestOperationCount; + this.maxReadRequestSize = maxReadRequestSize; + this.maxOutstandingReadRequests = maxOutstandingReadRequests; + this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; + this.maxWriteRequestSize = maxWriteRequestSize; + this.maxOutstandingWriteRequests = maxOutstandingWriteRequests; this.maxWriteBufferCount = maxWriteBufferCount; this.maxWriteBufferSize = maxWriteBufferSize; this.maxRetryDelay = maxRetryDelay; @@ -259,10 +273,12 @@ public AutoFollowPattern(StreamInput in) throws IOException { remoteCluster = in.readString(); leaderIndexPatterns = in.readList(StreamInput::readString); followIndexPattern = in.readOptionalString(); - maxBatchOperationCount = in.readOptionalVInt(); - maxConcurrentReadBatches = in.readOptionalVInt(); - maxBatchSize = in.readOptionalWriteable(ByteSizeValue::new); - maxConcurrentWriteBatches = in.readOptionalVInt(); + maxReadRequestOperationCount = in.readOptionalVInt(); + maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); + maxOutstandingReadRequests = in.readOptionalVInt(); + maxWriteRequestOperationCount = in.readOptionalVInt(); + maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new); + maxOutstandingWriteRequests = in.readOptionalVInt(); maxWriteBufferCount = in.readOptionalVInt(); maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); maxRetryDelay = in.readOptionalTimeValue(); @@ -289,20 +305,28 @@ public String getFollowIndexPattern() { return followIndexPattern; } - public Integer getMaxBatchOperationCount() { - return maxBatchOperationCount; + public Integer getMaxReadRequestOperationCount() { + return maxReadRequestOperationCount; + } + + public Integer getMaxOutstandingReadRequests() { + return maxOutstandingReadRequests; } - public Integer getMaxConcurrentReadBatches() { - return maxConcurrentReadBatches; + public ByteSizeValue getMaxReadRequestSize() { + return maxReadRequestSize; } - public ByteSizeValue getMaxBatchSize() { - return maxBatchSize; + public Integer getMaxWriteRequestOperationCount() { + return maxWriteRequestOperationCount; } - public Integer getMaxConcurrentWriteBatches() { - return maxConcurrentWriteBatches; + public ByteSizeValue getMaxWriteRequestSize() { + return maxWriteRequestSize; + } + + public Integer getMaxOutstandingWriteRequests() { + return maxOutstandingWriteRequests; } public Integer getMaxWriteBufferCount() { @@ -326,10 +350,12 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(remoteCluster); out.writeStringList(leaderIndexPatterns); out.writeOptionalString(followIndexPattern); - out.writeOptionalVInt(maxBatchOperationCount); - out.writeOptionalVInt(maxConcurrentReadBatches); - out.writeOptionalWriteable(maxBatchSize); - out.writeOptionalVInt(maxConcurrentWriteBatches); + out.writeOptionalVInt(maxReadRequestOperationCount); + out.writeOptionalWriteable(maxReadRequestSize); + out.writeOptionalVInt(maxOutstandingReadRequests); + out.writeOptionalVInt(maxWriteRequestOperationCount); + out.writeOptionalWriteable(maxWriteRequestSize); + out.writeOptionalVInt(maxOutstandingWriteRequests); out.writeOptionalVInt(maxWriteBufferCount); out.writeOptionalWriteable(maxWriteBufferSize); out.writeOptionalTimeValue(maxRetryDelay); @@ -343,17 +369,23 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (followIndexPattern != null) { builder.field(FOLLOW_PATTERN_FIELD.getPreferredName(), followIndexPattern); } - if (maxBatchOperationCount != null) { - builder.field(MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); + if (maxReadRequestOperationCount != null) { + builder.field(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), maxReadRequestOperationCount); + } + if (maxReadRequestSize != null) { + builder.field(MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep()); + } + if (maxOutstandingReadRequests != null) { + builder.field(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxOutstandingReadRequests); } - if (maxConcurrentReadBatches != null) { - builder.field(MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); + if (maxWriteRequestOperationCount != null) { + builder.field(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount); } - if (maxBatchSize != null) { - builder.field(MAX_BATCH_SIZE.getPreferredName(), maxBatchSize.getStringRep()); + if (maxWriteRequestSize != null) { + builder.field(MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep()); } - if (maxConcurrentWriteBatches != null) { - builder.field(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + if (maxOutstandingWriteRequests != null) { + builder.field(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), maxOutstandingWriteRequests); } if (maxWriteBufferCount != null){ builder.field(MAX_WRITE_BUFFER_COUNT.getPreferredName(), maxWriteBufferCount); @@ -365,7 +397,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(MAX_RETRY_DELAY.getPreferredName(), maxRetryDelay); } if (pollTimeout != null) { - builder.field(POLL_TIMEOUT.getPreferredName(), pollTimeout); + builder.field(READ_POLL_TIMEOUT.getPreferredName(), pollTimeout); } return builder; } @@ -383,10 +415,12 @@ public boolean equals(Object o) { return Objects.equals(remoteCluster, that.remoteCluster) && Objects.equals(leaderIndexPatterns, that.leaderIndexPatterns) && Objects.equals(followIndexPattern, that.followIndexPattern) && - Objects.equals(maxBatchOperationCount, that.maxBatchOperationCount) && - Objects.equals(maxConcurrentReadBatches, that.maxConcurrentReadBatches) && - Objects.equals(maxBatchSize, that.maxBatchSize) && - Objects.equals(maxConcurrentWriteBatches, that.maxConcurrentWriteBatches) && + Objects.equals(maxReadRequestOperationCount, that.maxReadRequestOperationCount) && + Objects.equals(maxReadRequestSize, that.maxReadRequestSize) && + Objects.equals(maxOutstandingReadRequests, that.maxOutstandingReadRequests) && + Objects.equals(maxWriteRequestOperationCount, that.maxWriteRequestOperationCount) && + Objects.equals(maxWriteRequestSize, that.maxWriteRequestSize) && + Objects.equals(maxOutstandingWriteRequests, that.maxOutstandingWriteRequests) && Objects.equals(maxWriteBufferCount, that.maxWriteBufferCount) && Objects.equals(maxWriteBufferSize, that.maxWriteBufferSize) && Objects.equals(maxRetryDelay, that.maxRetryDelay) && @@ -399,10 +433,12 @@ public int hashCode() { remoteCluster, leaderIndexPatterns, followIndexPattern, - maxBatchOperationCount, - maxConcurrentReadBatches, - maxBatchSize, - maxConcurrentWriteBatches, + maxReadRequestOperationCount, + maxReadRequestSize, + maxOutstandingReadRequests, + maxWriteRequestOperationCount, + maxWriteRequestSize, + maxOutstandingWriteRequests, maxWriteBufferCount, maxWriteBufferSize, maxRetryDelay, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/ShardFollowNodeTaskStatus.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/ShardFollowNodeTaskStatus.java index e2e907f80d110..b8f645eea4430 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/ShardFollowNodeTaskStatus.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/ShardFollowNodeTaskStatus.java @@ -43,23 +43,23 @@ public class ShardFollowNodeTaskStatus implements Task.Status { private static final ParseField FOLLOWER_GLOBAL_CHECKPOINT_FIELD = new ParseField("follower_global_checkpoint"); private static final ParseField FOLLOWER_MAX_SEQ_NO_FIELD = new ParseField("follower_max_seq_no"); private static final ParseField LAST_REQUESTED_SEQ_NO_FIELD = new ParseField("last_requested_seq_no"); - private static final ParseField NUMBER_OF_CONCURRENT_READS_FIELD = new ParseField("number_of_concurrent_reads"); - private static final ParseField NUMBER_OF_CONCURRENT_WRITES_FIELD = new ParseField("number_of_concurrent_writes"); - private static final ParseField NUMBER_OF_QUEUED_WRITES_FIELD = new ParseField("number_of_queued_writes"); - private static final ParseField BUFFER_SIZE_IN_BYTES_FIELD = new ParseField("buffer_size_in_bytes"); - private static final ParseField MAPPING_VERSION_FIELD = new ParseField("mapping_version"); - private static final ParseField TOTAL_FETCH_TIME_MILLIS_FIELD = new ParseField("total_fetch_time_millis"); - private static final ParseField TOTAL_FETCH_REMOTE_TIME_MILLIS_FIELD = new ParseField("total_fetch_remote_time_millis"); - private static final ParseField NUMBER_OF_SUCCESSFUL_FETCHES_FIELD = new ParseField("number_of_successful_fetches"); - private static final ParseField NUMBER_OF_FAILED_FETCHES_FIELD = new ParseField("number_of_failed_fetches"); - private static final ParseField OPERATIONS_RECEIVED_FIELD = new ParseField("operations_received"); - private static final ParseField TOTAL_TRANSFERRED_BYTES = new ParseField("total_transferred_bytes"); - private static final ParseField TOTAL_INDEX_TIME_MILLIS_FIELD = new ParseField("total_index_time_millis"); - private static final ParseField NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD = new ParseField("number_of_successful_bulk_operations"); - private static final ParseField NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD = new ParseField("number_of_failed_bulk_operations"); - private static final ParseField NUMBER_OF_OPERATIONS_INDEXED_FIELD = new ParseField("number_of_operations_indexed"); - private static final ParseField FETCH_EXCEPTIONS = new ParseField("fetch_exceptions"); - private static final ParseField TIME_SINCE_LAST_FETCH_MILLIS_FIELD = new ParseField("time_since_last_fetch_millis"); + private static final ParseField OUTSTANDING_READ_REQUESTS = new ParseField("outstanding_read_requests"); + private static final ParseField OUTSTANDING_WRITE_REQUESTS = new ParseField("outstanding_write_requests"); + private static final ParseField WRITE_BUFFER_OPERATION_COUNT_FIELD = new ParseField("write_buffer_operation_count"); + private static final ParseField WRITE_BUFFER_SIZE_IN_BYTES_FIELD = new ParseField("write_buffer_size_in_bytes"); + private static final ParseField FOLLOWER_MAPPING_VERSION_FIELD = new ParseField("follower_mapping_version"); + private static final ParseField TOTAL_READ_TIME_MILLIS_FIELD = new ParseField("total_read_time_millis"); + private static final ParseField TOTAL_READ_REMOTE_EXEC_TIME_MILLIS_FIELD = new ParseField("total_read_remote_exec_time_millis"); + private static final ParseField SUCCESSFUL_READ_REQUESTS_FIELD = new ParseField("successful_read_requests"); + private static final ParseField FAILED_READ_REQUESTS_FIELD = new ParseField("failed_read_requests"); + private static final ParseField OPERATIONS_READ_FIELD = new ParseField("operations_read"); + private static final ParseField BYTES_READ = new ParseField("bytes_read"); + private static final ParseField TOTAL_WRITE_TIME_MILLIS_FIELD = new ParseField("total_write_time_millis"); + private static final ParseField SUCCESSFUL_WRITE_REQUESTS_FIELD = new ParseField("successful_write_requests"); + private static final ParseField FAILED_WRITE_REQUEST_FIELD = new ParseField("failed_write_requests"); + private static final ParseField OPERATIONS_WRITTEN = new ParseField("operations_written"); + private static final ParseField READ_EXCEPTIONS = new ParseField("read_exceptions"); + private static final ParseField TIME_SINCE_LAST_READ_MILLIS_FIELD = new ParseField("time_since_last_read_millis"); private static final ParseField FATAL_EXCEPTION = new ParseField("fatal_exception"); @SuppressWarnings("unchecked") @@ -98,11 +98,11 @@ public class ShardFollowNodeTaskStatus implements Task.Status { (long) args[25], (ElasticsearchException) args[26])); - public static final String FETCH_EXCEPTIONS_ENTRY_PARSER_NAME = "shard-follow-node-task-status-fetch-exceptions-entry"; + public static final String READ_EXCEPTIONS_ENTRY_PARSER_NAME = "shard-follow-node-task-status-read-exceptions-entry"; - static final ConstructingObjectParser>, Void> FETCH_EXCEPTIONS_ENTRY_PARSER = + static final ConstructingObjectParser>, Void> READ_EXCEPTIONS_ENTRY_PARSER = new ConstructingObjectParser<>( - FETCH_EXCEPTIONS_ENTRY_PARSER_NAME, + READ_EXCEPTIONS_ENTRY_PARSER_NAME, args -> new AbstractMap.SimpleEntry<>((long) args[0], Tuple.tuple((Integer)args[1], (ElasticsearchException)args[2]))); static { @@ -115,39 +115,39 @@ public class ShardFollowNodeTaskStatus implements Task.Status { STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_GLOBAL_CHECKPOINT_FIELD); STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_MAX_SEQ_NO_FIELD); STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), LAST_REQUESTED_SEQ_NO_FIELD); - STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_READS_FIELD); - STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_WRITES_FIELD); - STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_QUEUED_WRITES_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), BUFFER_SIZE_IN_BYTES_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), MAPPING_VERSION_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_FETCH_TIME_MILLIS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_FETCH_REMOTE_TIME_MILLIS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_SUCCESSFUL_FETCHES_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_FAILED_FETCHES_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), OPERATIONS_RECEIVED_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_TRANSFERRED_BYTES); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_INDEX_TIME_MILLIS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), NUMBER_OF_OPERATIONS_INDEXED_FIELD); - STATUS_PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), FETCH_EXCEPTIONS_ENTRY_PARSER, FETCH_EXCEPTIONS); - STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TIME_SINCE_LAST_FETCH_MILLIS_FIELD); + STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), OUTSTANDING_READ_REQUESTS); + STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), OUTSTANDING_WRITE_REQUESTS); + STATUS_PARSER.declareInt(ConstructingObjectParser.constructorArg(), WRITE_BUFFER_OPERATION_COUNT_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), WRITE_BUFFER_SIZE_IN_BYTES_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_MAPPING_VERSION_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_READ_TIME_MILLIS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_READ_REMOTE_EXEC_TIME_MILLIS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), SUCCESSFUL_READ_REQUESTS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FAILED_READ_REQUESTS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), OPERATIONS_READ_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), BYTES_READ); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_WRITE_TIME_MILLIS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), SUCCESSFUL_WRITE_REQUESTS_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FAILED_WRITE_REQUEST_FIELD); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), OPERATIONS_WRITTEN); + STATUS_PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), READ_EXCEPTIONS_ENTRY_PARSER, READ_EXCEPTIONS); + STATUS_PARSER.declareLong(ConstructingObjectParser.constructorArg(), TIME_SINCE_LAST_READ_MILLIS_FIELD); STATUS_PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), (p, c) -> ElasticsearchException.fromXContent(p), FATAL_EXCEPTION); } - static final ParseField FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO = new ParseField("from_seq_no"); - static final ParseField FETCH_EXCEPTIONS_RETRIES = new ParseField("retries"); - static final ParseField FETCH_EXCEPTIONS_ENTRY_EXCEPTION = new ParseField("exception"); + static final ParseField READ_EXCEPTIONS_ENTRY_FROM_SEQ_NO = new ParseField("from_seq_no"); + static final ParseField READ_EXCEPTIONS_RETRIES = new ParseField("retries"); + static final ParseField READ_EXCEPTIONS_ENTRY_EXCEPTION = new ParseField("exception"); static { - FETCH_EXCEPTIONS_ENTRY_PARSER.declareLong(ConstructingObjectParser.constructorArg(), FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO); - FETCH_EXCEPTIONS_ENTRY_PARSER.declareInt(ConstructingObjectParser.constructorArg(), FETCH_EXCEPTIONS_RETRIES); - FETCH_EXCEPTIONS_ENTRY_PARSER.declareObject( + READ_EXCEPTIONS_ENTRY_PARSER.declareLong(ConstructingObjectParser.constructorArg(), READ_EXCEPTIONS_ENTRY_FROM_SEQ_NO); + READ_EXCEPTIONS_ENTRY_PARSER.declareInt(ConstructingObjectParser.constructorArg(), READ_EXCEPTIONS_RETRIES); + READ_EXCEPTIONS_ENTRY_PARSER.declareObject( ConstructingObjectParser.constructorArg(), (p, c) -> ElasticsearchException.fromXContent(p), - FETCH_EXCEPTIONS_ENTRY_EXCEPTION); + READ_EXCEPTIONS_ENTRY_EXCEPTION); } private final String remoteCluster; @@ -204,106 +204,106 @@ public long lastRequestedSeqNo() { return lastRequestedSeqNo; } - private final int numberOfConcurrentReads; + private final int outstandingReadRequests; - public int numberOfConcurrentReads() { - return numberOfConcurrentReads; + public int outstandingReadRequests() { + return outstandingReadRequests; } - private final int numberOfConcurrentWrites; + private final int outstandingWriteRequests; - public int numberOfConcurrentWrites() { - return numberOfConcurrentWrites; + public int outstandingWriteRequests() { + return outstandingWriteRequests; } - private final int numberOfQueuedWrites; + private final int writeBufferOperationCount; - public int numberOfQueuedWrites() { - return numberOfQueuedWrites; + public int writeBufferOperationCount() { + return writeBufferOperationCount; } - private final long bufferSize; + private final long writeBufferSizeInBytes; - public long bufferSize() { - return bufferSize; + public long writeBufferSizeInBytes() { + return writeBufferSizeInBytes; } - private final long mappingVersion; + private final long followerMappingVersion; - public long mappingVersion() { - return mappingVersion; + public long followerMappingVersion() { + return followerMappingVersion; } - private final long totalFetchTimeMillis; + private final long totalReadTimeMillis; - public long totalFetchTimeMillis() { - return totalFetchTimeMillis; + public long totalReadTimeMillis() { + return totalReadTimeMillis; } - private final long totalFetchRemoteTimeMillis; + private final long totalReadRemoteExecTimeMillis; - public long totalFetchRemoteTimeMillis() { - return totalFetchRemoteTimeMillis; + public long totalReadRemoteExecTimeMillis() { + return totalReadRemoteExecTimeMillis; } - private final long numberOfSuccessfulFetches; + private final long successfulReadRequests; - public long numberOfSuccessfulFetches() { - return numberOfSuccessfulFetches; + public long successfulReadRequests() { + return successfulReadRequests; } - private final long numberOfFailedFetches; + private final long failedReadRequests; - public long numberOfFailedFetches() { - return numberOfFailedFetches; + public long failedReadRequests() { + return failedReadRequests; } - private final long operationsReceived; + private final long operationsReads; - public long operationsReceived() { - return operationsReceived; + public long operationsReads() { + return operationsReads; } - private final long totalTransferredBytes; + private final long bytesRead; - public long totalTransferredBytes() { - return totalTransferredBytes; + public long bytesRead() { + return bytesRead; } - private final long totalIndexTimeMillis; + private final long totalWriteTimeMillis; - public long totalIndexTimeMillis() { - return totalIndexTimeMillis; + public long totalWriteTimeMillis() { + return totalWriteTimeMillis; } - private final long numberOfSuccessfulBulkOperations; + private final long successfulWriteRequests; - public long numberOfSuccessfulBulkOperations() { - return numberOfSuccessfulBulkOperations; + public long successfulWriteRequests() { + return successfulWriteRequests; } - private final long numberOfFailedBulkOperations; + private final long failedWriteRequests; - public long numberOfFailedBulkOperations() { - return numberOfFailedBulkOperations; + public long failedWriteRequests() { + return failedWriteRequests; } - private final long numberOfOperationsIndexed; + private final long operationWritten; - public long numberOfOperationsIndexed() { - return numberOfOperationsIndexed; + public long operationWritten() { + return operationWritten; } - private final NavigableMap> fetchExceptions; + private final NavigableMap> readExceptions; - public NavigableMap> fetchExceptions() { - return fetchExceptions; + public NavigableMap> readExceptions() { + return readExceptions; } - private final long timeSinceLastFetchMillis; + private final long timeSinceLastReadMillis; - public long timeSinceLastFetchMillis() { - return timeSinceLastFetchMillis; + public long timeSinceLastReadMillis() { + return timeSinceLastReadMillis; } private final ElasticsearchException fatalException; @@ -322,23 +322,23 @@ public ShardFollowNodeTaskStatus( final long followerGlobalCheckpoint, final long followerMaxSeqNo, final long lastRequestedSeqNo, - final int numberOfConcurrentReads, - final int numberOfConcurrentWrites, - final int numberOfQueuedWrites, - final long bufferSize, - final long mappingVersion, - final long totalFetchTimeMillis, - final long totalFetchRemoteTimeMillis, - final long numberOfSuccessfulFetches, - final long numberOfFailedFetches, - final long operationsReceived, - final long totalTransferredBytes, - final long totalIndexTimeMillis, - final long numberOfSuccessfulBulkOperations, - final long numberOfFailedBulkOperations, - final long numberOfOperationsIndexed, - final NavigableMap> fetchExceptions, - final long timeSinceLastFetchMillis, + final int outstandingReadRequests, + final int outstandingWriteRequests, + final int writeBufferOperationCount, + final long writeBufferSizeInBytes, + final long followerMappingVersion, + final long totalReadTimeMillis, + final long totalReadRemoteExecTimeMillis, + final long successfulReadRequests, + final long failedReadRequests, + final long operationsReads, + final long bytesRead, + final long totalWriteTimeMillis, + final long successfulWriteRequests, + final long failedWriteRequests, + final long operationWritten, + final NavigableMap> readExceptions, + final long timeSinceLastReadMillis, final ElasticsearchException fatalException) { this.remoteCluster = remoteCluster; this.leaderIndex = leaderIndex; @@ -349,23 +349,23 @@ public ShardFollowNodeTaskStatus( this.followerGlobalCheckpoint = followerGlobalCheckpoint; this.followerMaxSeqNo = followerMaxSeqNo; this.lastRequestedSeqNo = lastRequestedSeqNo; - this.numberOfConcurrentReads = numberOfConcurrentReads; - this.numberOfConcurrentWrites = numberOfConcurrentWrites; - this.numberOfQueuedWrites = numberOfQueuedWrites; - this.bufferSize = bufferSize; - this.mappingVersion = mappingVersion; - this.totalFetchTimeMillis = totalFetchTimeMillis; - this.totalFetchRemoteTimeMillis = totalFetchRemoteTimeMillis; - this.numberOfSuccessfulFetches = numberOfSuccessfulFetches; - this.numberOfFailedFetches = numberOfFailedFetches; - this.operationsReceived = operationsReceived; - this.totalTransferredBytes = totalTransferredBytes; - this.totalIndexTimeMillis = totalIndexTimeMillis; - this.numberOfSuccessfulBulkOperations = numberOfSuccessfulBulkOperations; - this.numberOfFailedBulkOperations = numberOfFailedBulkOperations; - this.numberOfOperationsIndexed = numberOfOperationsIndexed; - this.fetchExceptions = Objects.requireNonNull(fetchExceptions); - this.timeSinceLastFetchMillis = timeSinceLastFetchMillis; + this.outstandingReadRequests = outstandingReadRequests; + this.outstandingWriteRequests = outstandingWriteRequests; + this.writeBufferOperationCount = writeBufferOperationCount; + this.writeBufferSizeInBytes = writeBufferSizeInBytes; + this.followerMappingVersion = followerMappingVersion; + this.totalReadTimeMillis = totalReadTimeMillis; + this.totalReadRemoteExecTimeMillis = totalReadRemoteExecTimeMillis; + this.successfulReadRequests = successfulReadRequests; + this.failedReadRequests = failedReadRequests; + this.operationsReads = operationsReads; + this.bytesRead = bytesRead; + this.totalWriteTimeMillis = totalWriteTimeMillis; + this.successfulWriteRequests = successfulWriteRequests; + this.failedWriteRequests = failedWriteRequests; + this.operationWritten = operationWritten; + this.readExceptions = Objects.requireNonNull(readExceptions); + this.timeSinceLastReadMillis = timeSinceLastReadMillis; this.fatalException = fatalException; } @@ -379,24 +379,24 @@ public ShardFollowNodeTaskStatus(final StreamInput in) throws IOException { this.followerGlobalCheckpoint = in.readZLong(); this.followerMaxSeqNo = in.readZLong(); this.lastRequestedSeqNo = in.readZLong(); - this.numberOfConcurrentReads = in.readVInt(); - this.numberOfConcurrentWrites = in.readVInt(); - this.numberOfQueuedWrites = in.readVInt(); - this.bufferSize = in.readVLong(); - this.mappingVersion = in.readVLong(); - this.totalFetchTimeMillis = in.readVLong(); - this.totalFetchRemoteTimeMillis = in.readVLong(); - this.numberOfSuccessfulFetches = in.readVLong(); - this.numberOfFailedFetches = in.readVLong(); - this.operationsReceived = in.readVLong(); - this.totalTransferredBytes = in.readVLong(); - this.totalIndexTimeMillis = in.readVLong(); - this.numberOfSuccessfulBulkOperations = in.readVLong(); - this.numberOfFailedBulkOperations = in.readVLong(); - this.numberOfOperationsIndexed = in.readVLong(); - this.fetchExceptions = + this.outstandingReadRequests = in.readVInt(); + this.outstandingWriteRequests = in.readVInt(); + this.writeBufferOperationCount = in.readVInt(); + this.writeBufferSizeInBytes = in.readVLong(); + this.followerMappingVersion = in.readVLong(); + this.totalReadTimeMillis = in.readVLong(); + this.totalReadRemoteExecTimeMillis = in.readVLong(); + this.successfulReadRequests = in.readVLong(); + this.failedReadRequests = in.readVLong(); + this.operationsReads = in.readVLong(); + this.bytesRead = in.readVLong(); + this.totalWriteTimeMillis = in.readVLong(); + this.successfulWriteRequests = in.readVLong(); + this.failedWriteRequests = in.readVLong(); + this.operationWritten = in.readVLong(); + this.readExceptions = new TreeMap<>(in.readMap(StreamInput::readVLong, stream -> Tuple.tuple(stream.readVInt(), stream.readException()))); - this.timeSinceLastFetchMillis = in.readZLong(); + this.timeSinceLastReadMillis = in.readZLong(); this.fatalException = in.readException(); } @@ -416,29 +416,29 @@ public void writeTo(final StreamOutput out) throws IOException { out.writeZLong(followerGlobalCheckpoint); out.writeZLong(followerMaxSeqNo); out.writeZLong(lastRequestedSeqNo); - out.writeVInt(numberOfConcurrentReads); - out.writeVInt(numberOfConcurrentWrites); - out.writeVInt(numberOfQueuedWrites); - out.writeVLong(bufferSize); - out.writeVLong(mappingVersion); - out.writeVLong(totalFetchTimeMillis); - out.writeVLong(totalFetchRemoteTimeMillis); - out.writeVLong(numberOfSuccessfulFetches); - out.writeVLong(numberOfFailedFetches); - out.writeVLong(operationsReceived); - out.writeVLong(totalTransferredBytes); - out.writeVLong(totalIndexTimeMillis); - out.writeVLong(numberOfSuccessfulBulkOperations); - out.writeVLong(numberOfFailedBulkOperations); - out.writeVLong(numberOfOperationsIndexed); + out.writeVInt(outstandingReadRequests); + out.writeVInt(outstandingWriteRequests); + out.writeVInt(writeBufferOperationCount); + out.writeVLong(writeBufferSizeInBytes); + out.writeVLong(followerMappingVersion); + out.writeVLong(totalReadTimeMillis); + out.writeVLong(totalReadRemoteExecTimeMillis); + out.writeVLong(successfulReadRequests); + out.writeVLong(failedReadRequests); + out.writeVLong(operationsReads); + out.writeVLong(bytesRead); + out.writeVLong(totalWriteTimeMillis); + out.writeVLong(successfulWriteRequests); + out.writeVLong(failedWriteRequests); + out.writeVLong(operationWritten); out.writeMap( - fetchExceptions, + readExceptions, StreamOutput::writeVLong, (stream, value) -> { stream.writeVInt(value.v1()); stream.writeException(value.v2()); }); - out.writeZLong(timeSinceLastFetchMillis); + out.writeZLong(timeSinceLastReadMillis); out.writeException(fatalException); } @@ -462,44 +462,44 @@ public XContentBuilder toXContentFragment(final XContentBuilder builder, final P builder.field(FOLLOWER_GLOBAL_CHECKPOINT_FIELD.getPreferredName(), followerGlobalCheckpoint); builder.field(FOLLOWER_MAX_SEQ_NO_FIELD.getPreferredName(), followerMaxSeqNo); builder.field(LAST_REQUESTED_SEQ_NO_FIELD.getPreferredName(), lastRequestedSeqNo); - builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads); - builder.field(NUMBER_OF_CONCURRENT_WRITES_FIELD.getPreferredName(), numberOfConcurrentWrites); - builder.field(NUMBER_OF_QUEUED_WRITES_FIELD.getPreferredName(), numberOfQueuedWrites); + builder.field(OUTSTANDING_READ_REQUESTS.getPreferredName(), outstandingReadRequests); + builder.field(OUTSTANDING_WRITE_REQUESTS.getPreferredName(), outstandingWriteRequests); + builder.field(WRITE_BUFFER_OPERATION_COUNT_FIELD.getPreferredName(), writeBufferOperationCount); builder.humanReadableField( - BUFFER_SIZE_IN_BYTES_FIELD.getPreferredName(), - "buffer_size", - new ByteSizeValue(bufferSize)); - builder.field(MAPPING_VERSION_FIELD.getPreferredName(), mappingVersion); + WRITE_BUFFER_SIZE_IN_BYTES_FIELD.getPreferredName(), + "write_buffer_size", + new ByteSizeValue(writeBufferSizeInBytes)); + builder.field(FOLLOWER_MAPPING_VERSION_FIELD.getPreferredName(), followerMappingVersion); builder.humanReadableField( - TOTAL_FETCH_TIME_MILLIS_FIELD.getPreferredName(), - "total_fetch_time", - new TimeValue(totalFetchTimeMillis, TimeUnit.MILLISECONDS)); + TOTAL_READ_TIME_MILLIS_FIELD.getPreferredName(), + "total_read_time", + new TimeValue(totalReadTimeMillis, TimeUnit.MILLISECONDS)); builder.humanReadableField( - TOTAL_FETCH_REMOTE_TIME_MILLIS_FIELD.getPreferredName(), - "total_fetch_leader_time", - new TimeValue(totalFetchRemoteTimeMillis, TimeUnit.MILLISECONDS)); - builder.field(NUMBER_OF_SUCCESSFUL_FETCHES_FIELD.getPreferredName(), numberOfSuccessfulFetches); - builder.field(NUMBER_OF_FAILED_FETCHES_FIELD.getPreferredName(), numberOfFailedFetches); - builder.field(OPERATIONS_RECEIVED_FIELD.getPreferredName(), operationsReceived); + TOTAL_READ_REMOTE_EXEC_TIME_MILLIS_FIELD.getPreferredName(), + "total_read_remote_exec_time", + new TimeValue(totalReadRemoteExecTimeMillis, TimeUnit.MILLISECONDS)); + builder.field(SUCCESSFUL_READ_REQUESTS_FIELD.getPreferredName(), successfulReadRequests); + builder.field(FAILED_READ_REQUESTS_FIELD.getPreferredName(), failedReadRequests); + builder.field(OPERATIONS_READ_FIELD.getPreferredName(), operationsReads); builder.humanReadableField( - TOTAL_TRANSFERRED_BYTES.getPreferredName(), - "total_transferred", - new ByteSizeValue(totalTransferredBytes, ByteSizeUnit.BYTES)); + BYTES_READ.getPreferredName(), + "total_read", + new ByteSizeValue(bytesRead, ByteSizeUnit.BYTES)); builder.humanReadableField( - TOTAL_INDEX_TIME_MILLIS_FIELD.getPreferredName(), - "total_index_time", - new TimeValue(totalIndexTimeMillis, TimeUnit.MILLISECONDS)); - builder.field(NUMBER_OF_SUCCESSFUL_BULK_OPERATIONS_FIELD.getPreferredName(), numberOfSuccessfulBulkOperations); - builder.field(NUMBER_OF_FAILED_BULK_OPERATIONS_FIELD.getPreferredName(), numberOfFailedBulkOperations); - builder.field(NUMBER_OF_OPERATIONS_INDEXED_FIELD.getPreferredName(), numberOfOperationsIndexed); - builder.startArray(FETCH_EXCEPTIONS.getPreferredName()); + TOTAL_WRITE_TIME_MILLIS_FIELD.getPreferredName(), + "total_write_time", + new TimeValue(totalWriteTimeMillis, TimeUnit.MILLISECONDS)); + builder.field(SUCCESSFUL_WRITE_REQUESTS_FIELD.getPreferredName(), successfulWriteRequests); + builder.field(FAILED_WRITE_REQUEST_FIELD.getPreferredName(), failedWriteRequests); + builder.field(OPERATIONS_WRITTEN.getPreferredName(), operationWritten); + builder.startArray(READ_EXCEPTIONS.getPreferredName()); { - for (final Map.Entry> entry : fetchExceptions.entrySet()) { + for (final Map.Entry> entry : readExceptions.entrySet()) { builder.startObject(); { - builder.field(FETCH_EXCEPTIONS_ENTRY_FROM_SEQ_NO.getPreferredName(), entry.getKey()); - builder.field(FETCH_EXCEPTIONS_RETRIES.getPreferredName(), entry.getValue().v1()); - builder.field(FETCH_EXCEPTIONS_ENTRY_EXCEPTION.getPreferredName()); + builder.field(READ_EXCEPTIONS_ENTRY_FROM_SEQ_NO.getPreferredName(), entry.getKey()); + builder.field(READ_EXCEPTIONS_RETRIES.getPreferredName(), entry.getValue().v1()); + builder.field(READ_EXCEPTIONS_ENTRY_EXCEPTION.getPreferredName()); builder.startObject(); { ElasticsearchException.generateThrowableXContent(builder, params, entry.getValue().v2()); @@ -511,9 +511,9 @@ public XContentBuilder toXContentFragment(final XContentBuilder builder, final P } builder.endArray(); builder.humanReadableField( - TIME_SINCE_LAST_FETCH_MILLIS_FIELD.getPreferredName(), - "time_since_last_fetch", - new TimeValue(timeSinceLastFetchMillis, TimeUnit.MILLISECONDS)); + TIME_SINCE_LAST_READ_MILLIS_FIELD.getPreferredName(), + "time_since_last_read", + new TimeValue(timeSinceLastReadMillis, TimeUnit.MILLISECONDS)); if (fatalException != null) { builder.field(FATAL_EXCEPTION.getPreferredName()); builder.startObject(); @@ -545,28 +545,28 @@ public boolean equals(final Object o) { followerGlobalCheckpoint == that.followerGlobalCheckpoint && followerMaxSeqNo == that.followerMaxSeqNo && lastRequestedSeqNo == that.lastRequestedSeqNo && - numberOfConcurrentReads == that.numberOfConcurrentReads && - numberOfConcurrentWrites == that.numberOfConcurrentWrites && - numberOfQueuedWrites == that.numberOfQueuedWrites && - bufferSize == that.bufferSize && - mappingVersion == that.mappingVersion && - totalFetchTimeMillis == that.totalFetchTimeMillis && - totalFetchRemoteTimeMillis == that.totalFetchRemoteTimeMillis && - numberOfSuccessfulFetches == that.numberOfSuccessfulFetches && - numberOfFailedFetches == that.numberOfFailedFetches && - operationsReceived == that.operationsReceived && - totalTransferredBytes == that.totalTransferredBytes && - numberOfSuccessfulBulkOperations == that.numberOfSuccessfulBulkOperations && - numberOfFailedBulkOperations == that.numberOfFailedBulkOperations && - numberOfOperationsIndexed == that.numberOfOperationsIndexed && + outstandingReadRequests == that.outstandingReadRequests && + outstandingWriteRequests == that.outstandingWriteRequests && + writeBufferOperationCount == that.writeBufferOperationCount && + writeBufferSizeInBytes == that.writeBufferSizeInBytes && + followerMappingVersion == that.followerMappingVersion && + totalReadTimeMillis == that.totalReadTimeMillis && + totalReadRemoteExecTimeMillis == that.totalReadRemoteExecTimeMillis && + successfulReadRequests == that.successfulReadRequests && + failedReadRequests == that.failedReadRequests && + operationsReads == that.operationsReads && + bytesRead == that.bytesRead && + successfulWriteRequests == that.successfulWriteRequests && + failedWriteRequests == that.failedWriteRequests && + operationWritten == that.operationWritten && /* * ElasticsearchException does not implement equals so we will assume the fetch exceptions are equal if they are equal * up to the key set and their messages. Note that we are relying on the fact that the fetch exceptions are ordered by * keys. */ - fetchExceptions.keySet().equals(that.fetchExceptions.keySet()) && - getFetchExceptionMessages(this).equals(getFetchExceptionMessages(that)) && - timeSinceLastFetchMillis == that.timeSinceLastFetchMillis && + readExceptions.keySet().equals(that.readExceptions.keySet()) && + getReadExceptionMessages(this).equals(getReadExceptionMessages(that)) && + timeSinceLastReadMillis == that.timeSinceLastReadMillis && Objects.equals(fatalExceptionMessage, otherFatalExceptionMessage); } @@ -583,32 +583,32 @@ public int hashCode() { followerGlobalCheckpoint, followerMaxSeqNo, lastRequestedSeqNo, - numberOfConcurrentReads, - numberOfConcurrentWrites, - numberOfQueuedWrites, - bufferSize, - mappingVersion, - totalFetchTimeMillis, - totalFetchRemoteTimeMillis, - numberOfSuccessfulFetches, - numberOfFailedFetches, - operationsReceived, - totalTransferredBytes, - numberOfSuccessfulBulkOperations, - numberOfFailedBulkOperations, - numberOfOperationsIndexed, + outstandingReadRequests, + outstandingWriteRequests, + writeBufferOperationCount, + writeBufferSizeInBytes, + followerMappingVersion, + totalReadTimeMillis, + totalReadRemoteExecTimeMillis, + successfulReadRequests, + failedReadRequests, + operationsReads, + bytesRead, + successfulWriteRequests, + failedWriteRequests, + operationWritten, /* * ElasticsearchException does not implement hash code so we will compute the hash code based on the key set and the * messages. Note that we are relying on the fact that the fetch exceptions are ordered by keys. */ - fetchExceptions.keySet(), - getFetchExceptionMessages(this), - timeSinceLastFetchMillis, + readExceptions.keySet(), + getReadExceptionMessages(this), + timeSinceLastReadMillis, fatalExceptionMessage); } - private static List getFetchExceptionMessages(final ShardFollowNodeTaskStatus status) { - return status.fetchExceptions().values().stream().map(t -> t.v2().getMessage()).collect(Collectors.toList()); + private static List getReadExceptionMessages(final ShardFollowNodeTaskStatus status) { + return status.readExceptions().values().stream().map(t -> t.v2().getMessage()).collect(Collectors.toList()); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java index 8010c9bf34412..b2f966bba749a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java @@ -52,14 +52,20 @@ public static class Request extends AcknowledgedRequest implements ToXC PARSER.declareString(Request::setRemoteCluster, REMOTE_CLUSTER_FIELD); PARSER.declareStringArray(Request::setLeaderIndexPatterns, AutoFollowPattern.LEADER_PATTERNS_FIELD); PARSER.declareString(Request::setFollowIndexNamePattern, AutoFollowPattern.FOLLOW_PATTERN_FIELD); - PARSER.declareInt(Request::setMaxBatchOperationCount, AutoFollowPattern.MAX_BATCH_OPERATION_COUNT); - PARSER.declareInt(Request::setMaxConcurrentReadBatches, AutoFollowPattern.MAX_CONCURRENT_READ_BATCHES); + PARSER.declareInt(Request::setMaxReadRequestOperationCount, AutoFollowPattern.MAX_READ_REQUEST_OPERATION_COUNT); PARSER.declareField( - Request::setMaxBatchSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), AutoFollowPattern.MAX_BATCH_SIZE.getPreferredName()), - AutoFollowPattern.MAX_BATCH_SIZE, + Request::setMaxReadRequestSize, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), AutoFollowPattern.MAX_READ_REQUEST_SIZE.getPreferredName()), + AutoFollowPattern.MAX_READ_REQUEST_SIZE, ObjectParser.ValueType.STRING); - PARSER.declareInt(Request::setMaxConcurrentWriteBatches, AutoFollowPattern.MAX_CONCURRENT_WRITE_BATCHES); + PARSER.declareInt(Request::setMaxConcurrentReadBatches, AutoFollowPattern.MAX_OUTSTANDING_READ_REQUESTS); + PARSER.declareInt(Request::setMaxWriteRequestOperationCount, AutoFollowPattern.MAX_WRITE_REQUEST_OPERATION_COUNT); + PARSER.declareField( + Request::setMaxWriteRequestSize, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), AutoFollowPattern.MAX_WRITE_REQUEST_SIZE.getPreferredName()), + AutoFollowPattern.MAX_WRITE_REQUEST_SIZE, + ObjectParser.ValueType.STRING); + PARSER.declareInt(Request::setMaxConcurrentWriteBatches, AutoFollowPattern.MAX_OUTSTANDING_WRITE_REQUESTS); PARSER.declareInt(Request::setMaxWriteBufferCount, AutoFollowPattern.MAX_WRITE_BUFFER_COUNT); PARSER.declareField( Request::setMaxWriteBufferSize, @@ -69,9 +75,9 @@ public static class Request extends AcknowledgedRequest implements ToXC PARSER.declareField(Request::setMaxRetryDelay, (p, c) -> TimeValue.parseTimeValue(p.text(), AutoFollowPattern.MAX_RETRY_DELAY.getPreferredName()), AutoFollowPattern.MAX_RETRY_DELAY, ObjectParser.ValueType.STRING); - PARSER.declareField(Request::setPollTimeout, - (p, c) -> TimeValue.parseTimeValue(p.text(), AutoFollowPattern.POLL_TIMEOUT.getPreferredName()), - AutoFollowPattern.POLL_TIMEOUT, ObjectParser.ValueType.STRING); + PARSER.declareField(Request::setReadPollTimeout, + (p, c) -> TimeValue.parseTimeValue(p.text(), AutoFollowPattern.READ_POLL_TIMEOUT.getPreferredName()), + AutoFollowPattern.READ_POLL_TIMEOUT, ObjectParser.ValueType.STRING); } public static Request fromXContent(XContentParser parser, String name) throws IOException { @@ -93,14 +99,16 @@ public static Request fromXContent(XContentParser parser, String name) throws IO private List leaderIndexPatterns; private String followIndexNamePattern; - private Integer maxBatchOperationCount; + private Integer maxReadRequestOperationCount; + private ByteSizeValue maxReadRequestSize; private Integer maxConcurrentReadBatches; - private ByteSizeValue maxBatchSize; + private Integer maxWriteRequestOperationCount; + private ByteSizeValue maxWriteRequestSize; private Integer maxConcurrentWriteBatches; private Integer maxWriteBufferCount; private ByteSizeValue maxWriteBufferSize; private TimeValue maxRetryDelay; - private TimeValue pollTimeout; + private TimeValue readPollTimeout; @Override public ActionRequestValidationException validate() { @@ -164,12 +172,12 @@ public void setFollowIndexNamePattern(String followIndexNamePattern) { this.followIndexNamePattern = followIndexNamePattern; } - public Integer getMaxBatchOperationCount() { - return maxBatchOperationCount; + public Integer getMaxReadRequestOperationCount() { + return maxReadRequestOperationCount; } - public void setMaxBatchOperationCount(Integer maxBatchOperationCount) { - this.maxBatchOperationCount = maxBatchOperationCount; + public void setMaxReadRequestOperationCount(Integer maxReadRequestOperationCount) { + this.maxReadRequestOperationCount = maxReadRequestOperationCount; } public Integer getMaxConcurrentReadBatches() { @@ -180,12 +188,28 @@ public void setMaxConcurrentReadBatches(Integer maxConcurrentReadBatches) { this.maxConcurrentReadBatches = maxConcurrentReadBatches; } - public ByteSizeValue getMaxBatchSize() { - return maxBatchSize; + public ByteSizeValue getMaxReadRequestSize() { + return maxReadRequestSize; + } + + public void setMaxReadRequestSize(ByteSizeValue maxReadRequestSize) { + this.maxReadRequestSize = maxReadRequestSize; + } + + public Integer getMaxWriteRequestOperationCount() { + return maxWriteRequestOperationCount; } - public void setMaxBatchSize(ByteSizeValue maxBatchSize) { - this.maxBatchSize = maxBatchSize; + public void setMaxWriteRequestOperationCount(Integer maxWriteRequestOperationCount) { + this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; + } + + public ByteSizeValue getMaxWriteRequestSize() { + return maxWriteRequestSize; + } + + public void setMaxWriteRequestSize(ByteSizeValue maxWriteRequestSize) { + this.maxWriteRequestSize = maxWriteRequestSize; } public Integer getMaxConcurrentWriteBatches() { @@ -220,12 +244,12 @@ public void setMaxRetryDelay(TimeValue maxRetryDelay) { this.maxRetryDelay = maxRetryDelay; } - public TimeValue getPollTimeout() { - return pollTimeout; + public TimeValue getReadPollTimeout() { + return readPollTimeout; } - public void setPollTimeout(TimeValue pollTimeout) { - this.pollTimeout = pollTimeout; + public void setReadPollTimeout(TimeValue readPollTimeout) { + this.readPollTimeout = readPollTimeout; } @Override @@ -235,14 +259,16 @@ public void readFrom(StreamInput in) throws IOException { remoteCluster = in.readString(); leaderIndexPatterns = in.readList(StreamInput::readString); followIndexNamePattern = in.readOptionalString(); - maxBatchOperationCount = in.readOptionalVInt(); + maxReadRequestOperationCount = in.readOptionalVInt(); + maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); maxConcurrentReadBatches = in.readOptionalVInt(); - maxBatchSize = in.readOptionalWriteable(ByteSizeValue::new); + maxWriteRequestOperationCount = in.readOptionalVInt(); + maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new); maxConcurrentWriteBatches = in.readOptionalVInt(); maxWriteBufferCount = in.readOptionalVInt(); maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); maxRetryDelay = in.readOptionalTimeValue(); - pollTimeout = in.readOptionalTimeValue(); + readPollTimeout = in.readOptionalTimeValue(); } @Override @@ -252,14 +278,16 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(remoteCluster); out.writeStringList(leaderIndexPatterns); out.writeOptionalString(followIndexNamePattern); - out.writeOptionalVInt(maxBatchOperationCount); + out.writeOptionalVInt(maxReadRequestOperationCount); + out.writeOptionalWriteable(maxReadRequestSize); out.writeOptionalVInt(maxConcurrentReadBatches); - out.writeOptionalWriteable(maxBatchSize); + out.writeOptionalVInt(maxWriteRequestOperationCount); + out.writeOptionalWriteable(maxWriteRequestSize); out.writeOptionalVInt(maxConcurrentWriteBatches); out.writeOptionalVInt(maxWriteBufferCount); out.writeOptionalWriteable(maxWriteBufferSize); out.writeOptionalTimeValue(maxRetryDelay); - out.writeOptionalTimeValue(pollTimeout); + out.writeOptionalTimeValue(readPollTimeout); } @Override @@ -272,11 +300,17 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (followIndexNamePattern != null) { builder.field(AutoFollowPattern.FOLLOW_PATTERN_FIELD.getPreferredName(), followIndexNamePattern); } - if (maxBatchOperationCount != null) { - builder.field(AutoFollowPattern.MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); + if (maxReadRequestOperationCount != null) { + builder.field(AutoFollowPattern.MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), maxReadRequestOperationCount); + } + if (maxReadRequestSize != null) { + builder.field(AutoFollowPattern.MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep()); + } + if (maxWriteRequestOperationCount != null) { + builder.field(AutoFollowPattern.MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount); } - if (maxBatchSize != null) { - builder.field(AutoFollowPattern.MAX_BATCH_SIZE.getPreferredName(), maxBatchSize.getStringRep()); + if (maxWriteRequestSize != null) { + builder.field(AutoFollowPattern.MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep()); } if (maxWriteBufferCount != null) { builder.field(AutoFollowPattern.MAX_WRITE_BUFFER_COUNT.getPreferredName(), maxWriteBufferCount); @@ -285,16 +319,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(AutoFollowPattern.MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize.getStringRep()); } if (maxConcurrentReadBatches != null) { - builder.field(AutoFollowPattern.MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); + builder.field(AutoFollowPattern.MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxConcurrentReadBatches); } if (maxConcurrentWriteBatches != null) { - builder.field(AutoFollowPattern.MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + builder.field(AutoFollowPattern.MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), maxConcurrentWriteBatches); } if (maxRetryDelay != null) { builder.field(AutoFollowPattern.MAX_RETRY_DELAY.getPreferredName(), maxRetryDelay.getStringRep()); } - if (pollTimeout != null) { - builder.field(AutoFollowPattern.POLL_TIMEOUT.getPreferredName(), pollTimeout.getStringRep()); + if (readPollTimeout != null) { + builder.field(AutoFollowPattern.READ_POLL_TIMEOUT.getPreferredName(), readPollTimeout.getStringRep()); } } builder.endObject(); @@ -310,14 +344,16 @@ public boolean equals(Object o) { Objects.equals(remoteCluster, request.remoteCluster) && Objects.equals(leaderIndexPatterns, request.leaderIndexPatterns) && Objects.equals(followIndexNamePattern, request.followIndexNamePattern) && - Objects.equals(maxBatchOperationCount, request.maxBatchOperationCount) && + Objects.equals(maxReadRequestOperationCount, request.maxReadRequestOperationCount) && + Objects.equals(maxReadRequestSize, request.maxReadRequestSize) && Objects.equals(maxConcurrentReadBatches, request.maxConcurrentReadBatches) && - Objects.equals(maxBatchSize, request.maxBatchSize) && + Objects.equals(maxWriteRequestOperationCount, request.maxWriteRequestOperationCount) && + Objects.equals(maxWriteRequestSize, request.maxWriteRequestSize) && Objects.equals(maxConcurrentWriteBatches, request.maxConcurrentWriteBatches) && Objects.equals(maxWriteBufferCount, request.maxWriteBufferCount) && Objects.equals(maxWriteBufferSize, request.maxWriteBufferSize) && Objects.equals(maxRetryDelay, request.maxRetryDelay) && - Objects.equals(pollTimeout, request.pollTimeout); + Objects.equals(readPollTimeout, request.readPollTimeout); } @Override @@ -327,14 +363,16 @@ public int hashCode() { remoteCluster, leaderIndexPatterns, followIndexNamePattern, - maxBatchOperationCount, + maxReadRequestOperationCount, + maxReadRequestSize, maxConcurrentReadBatches, - maxBatchSize, + maxWriteRequestOperationCount, + maxWriteRequestSize, maxConcurrentWriteBatches, maxWriteBufferCount, maxWriteBufferSize, maxRetryDelay, - pollTimeout); + readPollTimeout); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java index 0f36af4db1010..b242b8cc8ec4c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java @@ -27,14 +27,14 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.FOLLOWER_INDEX_FIELD; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_BATCH_OPERATION_COUNT; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_BATCH_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_CONCURRENT_READ_BATCHES; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_CONCURRENT_WRITE_BATCHES; +import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_READ_REQUEST_OPERATION_COUNT; +import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_READ_REQUEST_SIZE; +import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_OUTSTANDING_READ_REQUESTS; +import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_OUTSTANDING_WRITE_REQUESTS; import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_RETRY_DELAY_FIELD; import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_BUFFER_COUNT; import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.MAX_WRITE_BUFFER_SIZE; -import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.POLL_TIMEOUT; +import static org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction.Request.READ_POLL_TIMEOUT; public final class PutFollowAction extends Action { @@ -64,30 +64,30 @@ public static class Request extends AcknowledgedRequest implements Indi static { PARSER.declareString(Request::setRemoteCluster, REMOTE_CLUSTER_FIELD); PARSER.declareString(Request::setLeaderIndex, LEADER_INDEX_FIELD); - PARSER.declareString((request, value) -> request.followRequest.setFollowerIndex(value), FOLLOWER_INDEX_FIELD); - PARSER.declareInt((request, value) -> request.followRequest.setMaxBatchOperationCount(value), MAX_BATCH_OPERATION_COUNT); - PARSER.declareInt((request, value) -> request.followRequest.setMaxConcurrentReadBatches(value), MAX_CONCURRENT_READ_BATCHES); + PARSER.declareString((req, val) -> req.followRequest.setFollowerIndex(val), FOLLOWER_INDEX_FIELD); + PARSER.declareInt((req, val) -> req.followRequest.setMaxReadRequestOperationCount(val), MAX_READ_REQUEST_OPERATION_COUNT); + PARSER.declareInt((req, val) -> req.followRequest.setMaxOutstandingReadRequests(val), MAX_OUTSTANDING_READ_REQUESTS); PARSER.declareField( - (request, value) -> request.followRequest.setMaxBatchSize(value), - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_BATCH_SIZE.getPreferredName()), - MAX_BATCH_SIZE, + (req, val) -> req.followRequest.setMaxReadRequestSize(val), + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_READ_REQUEST_SIZE.getPreferredName()), + MAX_READ_REQUEST_SIZE, ObjectParser.ValueType.STRING); - PARSER.declareInt((request, value) -> request.followRequest.setMaxConcurrentWriteBatches(value), MAX_CONCURRENT_WRITE_BATCHES); - PARSER.declareInt((request, value) -> request.followRequest.setMaxWriteBufferCount(value), MAX_WRITE_BUFFER_COUNT); + PARSER.declareInt((req, val) -> req.followRequest.setMaxOutstandingWriteRequests(val), MAX_OUTSTANDING_WRITE_REQUESTS); + PARSER.declareInt((req, val) -> req.followRequest.setMaxWriteBufferCount(val), MAX_WRITE_BUFFER_COUNT); PARSER.declareField( - (request, value) -> request.followRequest.setMaxWriteBufferSize(value), + (req, val) -> req.followRequest.setMaxWriteBufferSize(val), (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_BUFFER_SIZE.getPreferredName()), MAX_WRITE_BUFFER_SIZE, ObjectParser.ValueType.STRING); PARSER.declareField( - (request, value) -> request.followRequest.setMaxRetryDelay(value), + (req, val) -> req.followRequest.setMaxRetryDelay(val), (p, c) -> TimeValue.parseTimeValue(p.text(), MAX_RETRY_DELAY_FIELD.getPreferredName()), MAX_RETRY_DELAY_FIELD, ObjectParser.ValueType.STRING); PARSER.declareField( - (request, value) -> request.followRequest.setPollTimeout(value), - (p, c) -> TimeValue.parseTimeValue(p.text(), POLL_TIMEOUT.getPreferredName()), - POLL_TIMEOUT, + (req, val) -> req.followRequest.setReadPollTimeout(val), + (p, c) -> TimeValue.parseTimeValue(p.text(), READ_POLL_TIMEOUT.getPreferredName()), + READ_POLL_TIMEOUT, ObjectParser.ValueType.STRING); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java index 587223e3fbc4b..11c46492cc0f0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/ResumeFollowAction.java @@ -44,26 +44,31 @@ public AcknowledgedResponse newResponse() { public static class Request extends ActionRequest implements ToXContentObject { static final ParseField FOLLOWER_INDEX_FIELD = new ParseField("follower_index"); - static final ParseField MAX_BATCH_OPERATION_COUNT = new ParseField("max_batch_operation_count"); - static final ParseField MAX_CONCURRENT_READ_BATCHES = new ParseField("max_concurrent_read_batches"); - static final ParseField MAX_BATCH_SIZE = new ParseField("max_batch_size"); - static final ParseField MAX_CONCURRENT_WRITE_BATCHES = new ParseField("max_concurrent_write_batches"); + static final ParseField MAX_READ_REQUEST_OPERATION_COUNT = new ParseField("max_read_request_operation_count"); + static final ParseField MAX_READ_REQUEST_SIZE = new ParseField("max_read_request_size"); + static final ParseField MAX_OUTSTANDING_READ_REQUESTS = new ParseField("max_outstanding_read_requests"); + static final ParseField MAX_WRITE_REQUEST_OPERATION_COUNT = new ParseField("max_write_request_operation_count"); + static final ParseField MAX_WRITE_REQUEST_SIZE = new ParseField("max_write_request_size"); + static final ParseField MAX_OUTSTANDING_WRITE_REQUESTS = new ParseField("max_outstanding_write_requests"); static final ParseField MAX_WRITE_BUFFER_COUNT = new ParseField("max_write_buffer_count"); static final ParseField MAX_WRITE_BUFFER_SIZE = new ParseField("max_write_buffer_size"); static final ParseField MAX_RETRY_DELAY_FIELD = new ParseField("max_retry_delay"); - static final ParseField POLL_TIMEOUT = new ParseField("poll_timeout"); + static final ParseField READ_POLL_TIMEOUT = new ParseField("read_poll_timeout"); static final ObjectParser PARSER = new ObjectParser<>(NAME, Request::new); static { PARSER.declareString(Request::setFollowerIndex, FOLLOWER_INDEX_FIELD); - PARSER.declareInt(Request::setMaxBatchOperationCount, MAX_BATCH_OPERATION_COUNT); - PARSER.declareInt(Request::setMaxConcurrentReadBatches, MAX_CONCURRENT_READ_BATCHES); + PARSER.declareInt(Request::setMaxReadRequestOperationCount, MAX_READ_REQUEST_OPERATION_COUNT); PARSER.declareField( - Request::setMaxBatchSize, - (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_BATCH_SIZE.getPreferredName()), - MAX_BATCH_SIZE, + Request::setMaxReadRequestSize, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_READ_REQUEST_SIZE.getPreferredName()), MAX_READ_REQUEST_SIZE, ObjectParser.ValueType.STRING); - PARSER.declareInt(Request::setMaxConcurrentWriteBatches, MAX_CONCURRENT_WRITE_BATCHES); + PARSER.declareInt(Request::setMaxOutstandingReadRequests, MAX_OUTSTANDING_READ_REQUESTS); + PARSER.declareInt(Request::setMaxWriteRequestOperationCount, MAX_WRITE_REQUEST_OPERATION_COUNT); + PARSER.declareField(Request::setMaxWriteRequestSize, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MAX_WRITE_REQUEST_SIZE.getPreferredName()), MAX_WRITE_REQUEST_SIZE, + ObjectParser.ValueType.STRING); + PARSER.declareInt(Request::setMaxOutstandingWriteRequests, MAX_OUTSTANDING_WRITE_REQUESTS); PARSER.declareInt(Request::setMaxWriteBufferCount, MAX_WRITE_BUFFER_COUNT); PARSER.declareField( Request::setMaxWriteBufferSize, @@ -76,9 +81,9 @@ public static class Request extends ActionRequest implements ToXContentObject { MAX_RETRY_DELAY_FIELD, ObjectParser.ValueType.STRING); PARSER.declareField( - Request::setPollTimeout, - (p, c) -> TimeValue.parseTimeValue(p.text(), POLL_TIMEOUT.getPreferredName()), - POLL_TIMEOUT, + Request::setReadPollTimeout, + (p, c) -> TimeValue.parseTimeValue(p.text(), READ_POLL_TIMEOUT.getPreferredName()), + READ_POLL_TIMEOUT, ObjectParser.ValueType.STRING); } @@ -106,44 +111,64 @@ public void setFollowerIndex(String followerIndex) { this.followerIndex = followerIndex; } - private Integer maxBatchOperationCount; + private Integer maxReadRequestOperationCount; - public Integer getMaxBatchOperationCount() { - return maxBatchOperationCount; + public Integer getMaxReadRequestOperationCount() { + return maxReadRequestOperationCount; } - public void setMaxBatchOperationCount(Integer maxBatchOperationCount) { - this.maxBatchOperationCount = maxBatchOperationCount; + public void setMaxReadRequestOperationCount(Integer maxReadRequestOperationCount) { + this.maxReadRequestOperationCount = maxReadRequestOperationCount; } - private Integer maxConcurrentReadBatches; + private Integer maxOutstandingReadRequests; - public Integer getMaxConcurrentReadBatches() { - return maxConcurrentReadBatches; + public Integer getMaxOutstandingReadRequests() { + return maxOutstandingReadRequests; } - public void setMaxConcurrentReadBatches(Integer maxConcurrentReadBatches) { - this.maxConcurrentReadBatches = maxConcurrentReadBatches; + public void setMaxOutstandingReadRequests(Integer maxOutstandingReadRequests) { + this.maxOutstandingReadRequests = maxOutstandingReadRequests; } - private ByteSizeValue maxBatchSize; + private ByteSizeValue maxReadRequestSize; - public ByteSizeValue getMaxBatchSize() { - return maxBatchSize; + public ByteSizeValue getMaxReadRequestSize() { + return maxReadRequestSize; } - public void setMaxBatchSize(ByteSizeValue maxBatchSize) { - this.maxBatchSize = maxBatchSize; + public void setMaxReadRequestSize(ByteSizeValue maxReadRequestSize) { + this.maxReadRequestSize = maxReadRequestSize; } - private Integer maxConcurrentWriteBatches; + private Integer maxWriteRequestOperationCount; - public Integer getMaxConcurrentWriteBatches() { - return maxConcurrentWriteBatches; + public Integer getMaxWriteRequestOperationCount() { + return maxWriteRequestOperationCount; } - public void setMaxConcurrentWriteBatches(Integer maxConcurrentWriteBatches) { - this.maxConcurrentWriteBatches = maxConcurrentWriteBatches; + public void setMaxWriteRequestOperationCount(Integer maxWriteRequestOperationCount) { + this.maxWriteRequestOperationCount = maxWriteRequestOperationCount; + } + + private ByteSizeValue maxWriteRequestSize; + + public ByteSizeValue getMaxWriteRequestSize() { + return maxWriteRequestSize; + } + + public void setMaxWriteRequestSize(ByteSizeValue maxWriteRequestSize) { + this.maxWriteRequestSize = maxWriteRequestSize; + } + + private Integer maxOutstandingWriteRequests; + + public Integer getMaxOutstandingWriteRequests() { + return maxOutstandingWriteRequests; + } + + public void setMaxOutstandingWriteRequests(Integer maxOutstandingWriteRequests) { + this.maxOutstandingWriteRequests = maxOutstandingWriteRequests; } private Integer maxWriteBufferCount; @@ -176,14 +201,14 @@ public TimeValue getMaxRetryDelay() { return maxRetryDelay; } - private TimeValue pollTimeout; + private TimeValue readPollTimeout; - public TimeValue getPollTimeout() { - return pollTimeout; + public TimeValue getReadPollTimeout() { + return readPollTimeout; } - public void setPollTimeout(TimeValue pollTimeout) { - this.pollTimeout = pollTimeout; + public void setReadPollTimeout(TimeValue readPollTimeout) { + this.readPollTimeout = readPollTimeout; } public Request() { @@ -196,17 +221,23 @@ public ActionRequestValidationException validate() { if (followerIndex == null) { e = addValidationError(FOLLOWER_INDEX_FIELD.getPreferredName() + " is missing", e); } - if (maxBatchOperationCount != null && maxBatchOperationCount < 1) { - e = addValidationError(MAX_BATCH_OPERATION_COUNT.getPreferredName() + " must be larger than 0", e); + if (maxReadRequestOperationCount != null && maxReadRequestOperationCount < 1) { + e = addValidationError(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName() + " must be larger than 0", e); + } + if (maxReadRequestSize != null && maxReadRequestSize.compareTo(ByteSizeValue.ZERO) <= 0) { + e = addValidationError(MAX_READ_REQUEST_SIZE.getPreferredName() + " must be larger than 0", e); } - if (maxConcurrentReadBatches != null && maxConcurrentReadBatches < 1) { - e = addValidationError(MAX_CONCURRENT_READ_BATCHES.getPreferredName() + " must be larger than 0", e); + if (maxOutstandingReadRequests != null && maxOutstandingReadRequests < 1) { + e = addValidationError(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName() + " must be larger than 0", e); } - if (maxBatchSize != null && maxBatchSize.compareTo(ByteSizeValue.ZERO) <= 0) { - e = addValidationError(MAX_BATCH_SIZE.getPreferredName() + " must be larger than 0", e); + if (maxWriteRequestOperationCount != null && maxWriteRequestOperationCount < 1) { + e = addValidationError(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName() + " must be larger than 0", e); } - if (maxConcurrentWriteBatches != null && maxConcurrentWriteBatches < 1) { - e = addValidationError(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName() + " must be larger than 0", e); + if (maxWriteRequestSize != null && maxWriteRequestSize.compareTo(ByteSizeValue.ZERO) <= 0) { + e = addValidationError(MAX_WRITE_REQUEST_SIZE.getPreferredName() + " must be larger than 0", e); + } + if (maxOutstandingWriteRequests != null && maxOutstandingWriteRequests < 1) { + e = addValidationError(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName() + " must be larger than 0", e); } if (maxWriteBufferCount != null && maxWriteBufferCount < 1) { e = addValidationError(MAX_WRITE_BUFFER_COUNT.getPreferredName() + " must be larger than 0", e); @@ -232,28 +263,28 @@ public ActionRequestValidationException validate() { public void readFrom(final StreamInput in) throws IOException { super.readFrom(in); followerIndex = in.readString(); - maxBatchOperationCount = in.readOptionalVInt(); - maxConcurrentReadBatches = in.readOptionalVInt(); - maxBatchSize = in.readOptionalWriteable(ByteSizeValue::new); - maxConcurrentWriteBatches = in.readOptionalVInt(); + maxReadRequestOperationCount = in.readOptionalVInt(); + maxOutstandingReadRequests = in.readOptionalVInt(); + maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); + maxOutstandingWriteRequests = in.readOptionalVInt(); maxWriteBufferCount = in.readOptionalVInt(); maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); maxRetryDelay = in.readOptionalTimeValue(); - pollTimeout = in.readOptionalTimeValue(); + readPollTimeout = in.readOptionalTimeValue(); } @Override public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); out.writeString(followerIndex); - out.writeOptionalVInt(maxBatchOperationCount); - out.writeOptionalVInt(maxConcurrentReadBatches); - out.writeOptionalWriteable(maxBatchSize); - out.writeOptionalVInt(maxConcurrentWriteBatches); + out.writeOptionalVInt(maxReadRequestOperationCount); + out.writeOptionalVInt(maxOutstandingReadRequests); + out.writeOptionalWriteable(maxReadRequestSize); + out.writeOptionalVInt(maxOutstandingWriteRequests); out.writeOptionalVInt(maxWriteBufferCount); out.writeOptionalWriteable(maxWriteBufferSize); out.writeOptionalTimeValue(maxRetryDelay); - out.writeOptionalTimeValue(pollTimeout); + out.writeOptionalTimeValue(readPollTimeout); } @Override @@ -268,11 +299,17 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa void toXContentFragment(final XContentBuilder builder, final Params params) throws IOException { builder.field(FOLLOWER_INDEX_FIELD.getPreferredName(), followerIndex); - if (maxBatchOperationCount != null) { - builder.field(MAX_BATCH_OPERATION_COUNT.getPreferredName(), maxBatchOperationCount); + if (maxReadRequestOperationCount != null) { + builder.field(MAX_READ_REQUEST_OPERATION_COUNT.getPreferredName(), maxReadRequestOperationCount); + } + if (maxReadRequestSize != null) { + builder.field(MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep()); + } + if (maxWriteRequestOperationCount != null) { + builder.field(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount); } - if (maxBatchSize != null) { - builder.field(MAX_BATCH_SIZE.getPreferredName(), maxBatchSize.getStringRep()); + if (maxWriteRequestSize != null) { + builder.field(MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep()); } if (maxWriteBufferCount != null) { builder.field(MAX_WRITE_BUFFER_COUNT.getPreferredName(), maxWriteBufferCount); @@ -280,17 +317,17 @@ void toXContentFragment(final XContentBuilder builder, final Params params) thro if (maxWriteBufferSize != null) { builder.field(MAX_WRITE_BUFFER_SIZE.getPreferredName(), maxWriteBufferSize.getStringRep()); } - if (maxConcurrentReadBatches != null) { - builder.field(MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches); + if (maxOutstandingReadRequests != null) { + builder.field(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxOutstandingReadRequests); } - if (maxConcurrentWriteBatches != null) { - builder.field(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches); + if (maxOutstandingWriteRequests != null) { + builder.field(MAX_OUTSTANDING_WRITE_REQUESTS.getPreferredName(), maxOutstandingWriteRequests); } if (maxRetryDelay != null) { builder.field(MAX_RETRY_DELAY_FIELD.getPreferredName(), maxRetryDelay.getStringRep()); } - if (pollTimeout != null) { - builder.field(POLL_TIMEOUT.getPreferredName(), pollTimeout.getStringRep()); + if (readPollTimeout != null) { + builder.field(READ_POLL_TIMEOUT.getPreferredName(), readPollTimeout.getStringRep()); } } @@ -299,14 +336,16 @@ public boolean equals(final Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; - return Objects.equals(maxBatchOperationCount, request.maxBatchOperationCount) && - Objects.equals(maxConcurrentReadBatches, request.maxConcurrentReadBatches) && - Objects.equals(maxBatchSize, request.maxBatchSize) && - Objects.equals(maxConcurrentWriteBatches, request.maxConcurrentWriteBatches) && + return Objects.equals(maxReadRequestOperationCount, request.maxReadRequestOperationCount) && + Objects.equals(maxReadRequestSize, request.maxReadRequestSize) && + Objects.equals(maxOutstandingReadRequests, request.maxOutstandingReadRequests) && + Objects.equals(maxWriteRequestOperationCount, request.maxWriteRequestOperationCount) && + Objects.equals(maxWriteRequestSize, request.maxWriteRequestSize) && + Objects.equals(maxOutstandingWriteRequests, request.maxOutstandingWriteRequests) && Objects.equals(maxWriteBufferCount, request.maxWriteBufferCount) && Objects.equals(maxWriteBufferSize, request.maxWriteBufferSize) && Objects.equals(maxRetryDelay, request.maxRetryDelay) && - Objects.equals(pollTimeout, request.pollTimeout) && + Objects.equals(readPollTimeout, request.readPollTimeout) && Objects.equals(followerIndex, request.followerIndex); } @@ -314,14 +353,16 @@ public boolean equals(final Object o) { public int hashCode() { return Objects.hash( followerIndex, - maxBatchOperationCount, - maxConcurrentReadBatches, - maxBatchSize, - maxConcurrentWriteBatches, + maxReadRequestOperationCount, + maxReadRequestSize, + maxOutstandingReadRequests, + maxWriteRequestOperationCount, + maxWriteRequestSize, + maxOutstandingWriteRequests, maxWriteBufferCount, maxWriteBufferSize, maxRetryDelay, - pollTimeout); + readPollTimeout); } } diff --git a/x-pack/plugin/core/src/main/resources/monitoring-es.json b/x-pack/plugin/core/src/main/resources/monitoring-es.json index 4c2a979244004..bdd16d3b58cc6 100644 --- a/x-pack/plugin/core/src/main/resources/monitoring-es.json +++ b/x-pack/plugin/core/src/main/resources/monitoring-es.json @@ -956,52 +956,52 @@ "last_requested_seq_no": { "type": "long" }, - "number_of_concurrent_reads": { + "outstanding_read_requests": { "type": "long" }, - "number_of_concurrent_writes": { + "outstanding_write_requests": { "type": "long" }, - "number_of_queued_writes": { + "write_buffer_operation_count": { "type": "long" }, - "buffer_size_in_bytes": { + "write_buffer_size_in_bytes": { "type": "long" }, - "mapping_version": { + "follower_mapping_version": { "type": "long" }, - "total_fetch_time_millis": { + "total_read_time_millis": { "type": "long" }, - "total_fetch_remote_time_millis": { + "total_read_remote_exec_time_millis": { "type": "long" }, - "number_of_successful_fetches": { + "successful_read_requests": { "type": "long" }, - "number_of_failed_fetches": { + "failed_read_requests": { "type": "long" }, - "operations_received": { + "operations_read": { "type": "long" }, - "total_transferred_bytes": { + "bytes_read": { "type": "long" }, - "total_index_time_millis": { + "total_write_time_millis": { "type": "long" }, - "number_of_successful_bulk_operations": { + "successful_write_requests": { "type": "long" }, - "number_of_failed_bulk_operations": { + "failed_write_requests": { "type": "long" }, - "number_of_operations_indexed": { + "operations_written": { "type": "long" }, - "fetch_exceptions": { + "read_exceptions": { "type": "nested", "properties": { "from_seq_no": { @@ -1023,7 +1023,7 @@ } } }, - "time_since_last_fetch_millis": { + "time_since_last_read_millis": { "type": "long" } }