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)
This commit is contained in:
parent
13ca942152
commit
ff49e79d40
|
@ -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:
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -59,7 +59,7 @@ public class ESCCRRestTestCase extends ESRestTestCase {
|
|||
|
||||
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 @@ public class ESCCRRestTestCase extends ESRestTestCase {
|
|||
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 @@ public class ESCCRRestTestCase extends ESRestTestCase {
|
|||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -324,14 +324,16 @@ public class AutoFollowCoordinator implements ClusterStateApplier {
|
|||
|
||||
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);
|
||||
|
|
|
@ -65,8 +65,8 @@ public class ShardChangesAction extends Action<ShardChangesAction.Response> {
|
|||
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;
|
||||
|
||||
|
|
|
@ -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<Translog.Operation> 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<Long, Tuple<AtomicInteger, ElasticsearchException>>() {
|
||||
@Override
|
||||
protected boolean removeEldestEntry(final Map.Entry<Long, Tuple<AtomicInteger, ElasticsearchException>> eldest) {
|
||||
return size() > params.getMaxConcurrentReadBatches();
|
||||
return size() > params.getMaxOutstandingReadRequests();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -129,12 +129,12 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
}
|
||||
|
||||
// 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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
|
||||
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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
|
||||
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<Translog.Operation> 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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
|
||||
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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
},
|
||||
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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
sendShardChangesRequest(newFromSeqNo, newSize, maxRequiredSeqNo);
|
||||
} else {
|
||||
// read is completed, decrement
|
||||
numConcurrentReads--;
|
||||
numOutstandingReads--;
|
||||
coordinateReads();
|
||||
}
|
||||
}
|
||||
|
@ -322,16 +322,16 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
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 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
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 abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
|
|||
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()
|
||||
|
|
|
@ -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<ShardFollowTask, Void> 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<String, String>) 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<String, String>) 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<String, String> 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<String, String> 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 class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams {
|
|||
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 class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams {
|
|||
return leaderShardId;
|
||||
}
|
||||
|
||||
public int getMaxBatchOperationCount() {
|
||||
return maxBatchOperationCount;
|
||||
public int getMaxReadRequestOperationCount() {
|
||||
return maxReadRequestOperationCount;
|
||||
}
|
||||
|
||||
public int getMaxConcurrentReadBatches() {
|
||||
return maxConcurrentReadBatches;
|
||||
public int getMaxOutstandingReadRequests() {
|
||||
return maxOutstandingReadRequests;
|
||||
}
|
||||
|
||||
public int getMaxConcurrentWriteBatches() {
|
||||
return maxConcurrentWriteBatches;
|
||||
public int getMaxWriteRequestOperationCount() {
|
||||
return maxWriteRequestOperationCount;
|
||||
}
|
||||
|
||||
public ByteSizeValue getMaxWriteRequestSize() {
|
||||
return maxWriteRequestSize;
|
||||
}
|
||||
|
||||
public int getMaxOutstandingWriteRequests() {
|
||||
return maxOutstandingWriteRequests;
|
||||
}
|
||||
|
||||
public int getMaxWriteBufferCount() {
|
||||
|
@ -177,16 +200,16 @@ public class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams {
|
|||
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 class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams {
|
|||
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 class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams {
|
|||
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 class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams {
|
|||
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 class ShardFollowTask implements XPackPlugin.XPackPersistentTaskParams {
|
|||
remoteCluster,
|
||||
followShardId,
|
||||
leaderShardId,
|
||||
maxBatchOperationCount,
|
||||
maxConcurrentReadBatches,
|
||||
maxConcurrentWriteBatches,
|
||||
maxBatchSize,
|
||||
maxReadRequestOperationCount,
|
||||
maxReadRequestSize,
|
||||
maxOutstandingReadRequests,
|
||||
maxWriteRequestOperationCount,
|
||||
maxWriteRequestSize,
|
||||
maxOutstandingWriteRequests,
|
||||
maxWriteBufferCount,
|
||||
maxWriteBufferSize,
|
||||
maxRetryDelay,
|
||||
pollTimeout,
|
||||
readPollTimeout,
|
||||
headers
|
||||
);
|
||||
}
|
||||
|
|
|
@ -153,8 +153,8 @@ public class ShardFollowTasksExecutor extends PersistentTasksExecutor<ShardFollo
|
|||
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));
|
||||
}
|
||||
};
|
||||
|
|
|
@ -160,14 +160,16 @@ public class TransportPutAutoFollowPatternAction extends
|
|||
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())
|
||||
|
|
|
@ -52,14 +52,16 @@ import java.util.stream.Collectors;
|
|||
|
||||
public class TransportResumeFollowAction extends HandledTransportAction<ResumeFollowAction.Request, AcknowledgedResponse> {
|
||||
|
||||
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 @@ public class TransportResumeFollowAction extends HandledTransportAction<ResumeFo
|
|||
IndexMetaData followIndexMetadata,
|
||||
Map<String, String> 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 @@ public class TransportResumeFollowAction extends HandledTransportAction<ResumeFo
|
|||
}
|
||||
|
||||
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
|
||||
);
|
||||
}
|
||||
|
|
|
@ -273,8 +273,8 @@ public abstract class CcrIntegTestCase extends ESTestCase {
|
|||
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));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -71,7 +71,7 @@ public abstract class CcrSingleNodeTestCase extends ESSingleNodeTestCase {
|
|||
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 @@ public abstract class CcrSingleNodeTestCase extends ESSingleNodeTestCase {
|
|||
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));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -135,19 +135,25 @@ public class AutoFollowIT extends CcrIntegTestCase {
|
|||
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 class AutoFollowIT extends CcrIntegTestCase {
|
|||
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()));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -45,6 +45,8 @@ public class AutoFollowMetadataTests extends AbstractSerializingTestCase<AutoFol
|
|||
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),
|
||||
|
|
|
@ -148,7 +148,7 @@ public class CcrLicenseIT extends CcrSingleNodeTestCase {
|
|||
@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(),
|
||||
|
|
|
@ -252,9 +252,9 @@ public class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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 class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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 class IndexFollowingIT extends CcrIntegTestCase {
|
|||
}
|
||||
|
||||
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<ShardId, Long> firstBatchNumDocsPerShard = new HashMap<>();
|
||||
|
@ -456,10 +459,10 @@ public class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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 class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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 class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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 class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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 class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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 class IndexFollowingIT extends CcrIntegTestCase {
|
|||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,8 +56,8 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
|
|||
.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<String, AutoFollowPattern> patterns = new HashMap<>();
|
||||
patterns.put("remote", autoFollowPattern);
|
||||
Map<String, List<String>> followedLeaderIndexUUIDS = new HashMap<>();
|
||||
|
@ -120,8 +120,8 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
|
|||
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<String, AutoFollowPattern> patterns = new HashMap<>();
|
||||
patterns.put("remote", autoFollowPattern);
|
||||
Map<String, List<String>> followedLeaderIndexUUIDS = new HashMap<>();
|
||||
|
@ -178,8 +178,8 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
|
|||
.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<String, AutoFollowPattern> patterns = new HashMap<>();
|
||||
patterns.put("remote", autoFollowPattern);
|
||||
Map<String, List<String>> followedLeaderIndexUUIDS = new HashMap<>();
|
||||
|
@ -241,8 +241,8 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
|
|||
.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<String, AutoFollowPattern> patterns = new HashMap<>();
|
||||
patterns.put("remote", autoFollowPattern);
|
||||
Map<String, List<String>> followedLeaderIndexUUIDS = new HashMap<>();
|
||||
|
@ -295,8 +295,8 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
|
|||
}
|
||||
|
||||
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<String, Map<String, String>> headers = new HashMap<>();
|
||||
ClusterState followerState = ClusterState.builder(new ClusterName("remote"))
|
||||
.metaData(MetaData.builder().putCustom(AutoFollowMetadata.TYPE,
|
||||
|
@ -341,15 +341,15 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
|
|||
|
||||
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"));
|
||||
}
|
||||
|
||||
|
|
|
@ -33,6 +33,8 @@ public class GetAutoFollowPatternResponseTests extends AbstractStreamableTestCas
|
|||
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),
|
||||
|
|
|
@ -48,13 +48,13 @@ public class PutAutoFollowPatternRequestTests extends AbstractStreamableXContent
|
|||
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 @@ public class PutAutoFollowPatternRequestTests extends AbstractStreamableXContent
|
|||
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));
|
||||
|
|
|
@ -45,16 +45,16 @@ public class ResumeFollowActionRequestTests extends AbstractStreamableXContentTe
|
|||
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 @@ public class ResumeFollowActionRequestTests extends AbstractStreamableXContentTe
|
|||
request.setMaxRetryDelay(TimeValue.timeValueMillis(500));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
request.setPollTimeout(TimeValue.timeValueMillis(500));
|
||||
request.setReadPollTimeout(TimeValue.timeValueMillis(500));
|
||||
}
|
||||
return request;
|
||||
}
|
||||
|
|
|
@ -60,17 +60,17 @@ public class ShardFollowNodeTaskRandomTests extends ESTestCase {
|
|||
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 @@ public class ShardFollowNodeTaskRandomTests extends ESTestCase {
|
|||
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),
|
||||
|
|
|
@ -74,23 +74,23 @@ public class ShardFollowNodeTaskStatusTests extends AbstractSerializingTestCase<
|
|||
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<Long, Tuple<Integer, ElasticsearchException>> entry : newInstance.fetchExceptions().entrySet()) {
|
||||
final Tuple<Integer, ElasticsearchException> 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<Long, Tuple<Integer, ElasticsearchException>> entry : newInstance.readExceptions().entrySet()) {
|
||||
final Tuple<Integer, ElasticsearchException> 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 @@ public class ShardFollowNodeTaskStatusTests extends AbstractSerializingTestCase<
|
|||
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
|
||||
|
|
|
@ -63,8 +63,12 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
private Queue<Integer> 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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
{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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
}
|
||||
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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<Long, Tuple<Integer, ElasticsearchException>> entry = status.fetchExceptions().entrySet().iterator().next();
|
||||
assertThat(status.readExceptions().entrySet(), hasSize(1));
|
||||
final Map.Entry<Long, Tuple<Integer, ElasticsearchException>> 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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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.successfulReadRequests(), equalTo(0L));
|
||||
assertThat(status.totalReadTimeMillis(), equalTo(0L));
|
||||
assertThat(status.operationsReads(), equalTo(0L));
|
||||
assertThat(status.bytesRead(), equalTo(0L));
|
||||
|
||||
assertThat(status.fetchExceptions().entrySet(), hasSize(0));
|
||||
assertThat(status.totalFetchTimeMillis(), equalTo(0L));
|
||||
assertThat(status.numberOfFailedFetches(), 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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
}
|
||||
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
// 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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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<Long, Tuple<Integer, ElasticsearchException>> 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<Long, Tuple<Integer, ElasticsearchException>> 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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
}
|
||||
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
}
|
||||
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
}
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
}
|
||||
|
||||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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 class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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<String, String> 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 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|||
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<Exception> errorHandler) {
|
||||
|
|
|
@ -368,7 +368,9 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
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 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
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,
|
||||
|
|
|
@ -30,6 +30,8 @@ public class ShardFollowTaskTests extends AbstractSerializingTestCase<ShardFollo
|
|||
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),
|
||||
|
|
|
@ -33,7 +33,7 @@ public class TransportDeleteAutoFollowPatternActionTests extends ESTestCase {
|
|||
List<String> 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<String> existingUUIDS = new ArrayList<>();
|
||||
existingUUIDS.add("_val");
|
||||
|
@ -44,7 +44,7 @@ public class TransportDeleteAutoFollowPatternActionTests extends ESTestCase {
|
|||
List<String> 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<String> existingUUIDS = new ArrayList<>();
|
||||
existingUUIDS.add("_val");
|
||||
|
@ -78,7 +78,7 @@ public class TransportDeleteAutoFollowPatternActionTests extends ESTestCase {
|
|||
List<String> 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"))
|
||||
|
|
|
@ -24,9 +24,9 @@ public class TransportGetAutoFollowPatternActionTests extends ESTestCase {
|
|||
public void testGetAutoFollowPattern() {
|
||||
Map<String, AutoFollowPattern> 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();
|
||||
|
|
|
@ -103,7 +103,7 @@ public class TransportPutAutoFollowPatternActionTests extends ESTestCase {
|
|||
List<String> 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<String, List<String>> existingAlreadyFollowedIndexUUIDS = new HashMap<>();
|
||||
List<String> existingUUIDS = new ArrayList<>();
|
||||
existingUUIDS.add("_val");
|
||||
|
|
|
@ -79,8 +79,10 @@ public class TransportUnfollowActionTests extends ESTestCase {
|
|||
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),
|
||||
|
|
|
@ -91,24 +91,24 @@ public class FollowStatsMonitoringDocTests extends BaseMonitoringDocTestCase<Fol
|
|||
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<Long, Tuple<Integer, ElasticsearchException>> 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 class FollowStatsMonitoringDocTests extends BaseMonitoringDocTestCase<Fol
|
|||
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 class FollowStatsMonitoringDocTests extends BaseMonitoringDocTestCase<Fol
|
|||
+ "\"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 class FollowStatsMonitoringDocTests extends BaseMonitoringDocTestCase<Fol
|
|||
+ "}"
|
||||
+ "}"
|
||||
+ "],"
|
||||
+ "\"time_since_last_fetch_millis\":" + timeSinceLastFetchMillis + ","
|
||||
+ "\"time_since_last_read_millis\":" + timeSinceLastReadMillis + ","
|
||||
+ "\"fatal_exception\":{\"type\":\"exception\",\"reason\":\"fatal error\"}"
|
||||
+ "}"
|
||||
+ "}"));
|
||||
|
@ -250,7 +250,7 @@ public class FollowStatsMonitoringDocTests extends BaseMonitoringDocTestCase<Fol
|
|||
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"));
|
||||
|
|
|
@ -178,34 +178,42 @@ public class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
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<AutoFollowPattern, Void> PARSER =
|
||||
new ConstructingObjectParser<>("auto_follow_pattern",
|
||||
args -> new AutoFollowPattern((String) args[0], (List<String>) 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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
(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<String> 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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
public AutoFollowPattern(String remoteCluster,
|
||||
List<String> 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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
return followIndexPattern;
|
||||
}
|
||||
|
||||
public Integer getMaxBatchOperationCount() {
|
||||
return maxBatchOperationCount;
|
||||
public Integer getMaxReadRequestOperationCount() {
|
||||
return maxReadRequestOperationCount;
|
||||
}
|
||||
|
||||
public Integer getMaxConcurrentReadBatches() {
|
||||
return maxConcurrentReadBatches;
|
||||
public Integer getMaxOutstandingReadRequests() {
|
||||
return maxOutstandingReadRequests;
|
||||
}
|
||||
|
||||
public ByteSizeValue getMaxBatchSize() {
|
||||
return maxBatchSize;
|
||||
public ByteSizeValue getMaxReadRequestSize() {
|
||||
return maxReadRequestSize;
|
||||
}
|
||||
|
||||
public Integer getMaxConcurrentWriteBatches() {
|
||||
return maxConcurrentWriteBatches;
|
||||
public Integer getMaxWriteRequestOperationCount() {
|
||||
return maxWriteRequestOperationCount;
|
||||
}
|
||||
|
||||
public ByteSizeValue getMaxWriteRequestSize() {
|
||||
return maxWriteRequestSize;
|
||||
}
|
||||
|
||||
public Integer getMaxOutstandingWriteRequests() {
|
||||
return maxOutstandingWriteRequests;
|
||||
}
|
||||
|
||||
public Integer getMaxWriteBufferCount() {
|
||||
|
@ -326,10 +350,12 @@ public class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
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 (maxConcurrentReadBatches != null) {
|
||||
builder.field(MAX_CONCURRENT_READ_BATCHES.getPreferredName(), maxConcurrentReadBatches);
|
||||
if (maxReadRequestSize != null) {
|
||||
builder.field(MAX_READ_REQUEST_SIZE.getPreferredName(), maxReadRequestSize.getStringRep());
|
||||
}
|
||||
if (maxBatchSize != null) {
|
||||
builder.field(MAX_BATCH_SIZE.getPreferredName(), maxBatchSize.getStringRep());
|
||||
if (maxOutstandingReadRequests != null) {
|
||||
builder.field(MAX_OUTSTANDING_READ_REQUESTS.getPreferredName(), maxOutstandingReadRequests);
|
||||
}
|
||||
if (maxConcurrentWriteBatches != null) {
|
||||
builder.field(MAX_CONCURRENT_WRITE_BATCHES.getPreferredName(), maxConcurrentWriteBatches);
|
||||
if (maxWriteRequestOperationCount != null) {
|
||||
builder.field(MAX_WRITE_REQUEST_OPERATION_COUNT.getPreferredName(), maxWriteRequestOperationCount);
|
||||
}
|
||||
if (maxWriteRequestSize != null) {
|
||||
builder.field(MAX_WRITE_REQUEST_SIZE.getPreferredName(), maxWriteRequestSize.getStringRep());
|
||||
}
|
||||
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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
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 class AutoFollowMetadata extends AbstractNamedDiffable<MetaData.Custom> i
|
|||
remoteCluster,
|
||||
leaderIndexPatterns,
|
||||
followIndexPattern,
|
||||
maxBatchOperationCount,
|
||||
maxConcurrentReadBatches,
|
||||
maxBatchSize,
|
||||
maxConcurrentWriteBatches,
|
||||
maxReadRequestOperationCount,
|
||||
maxReadRequestSize,
|
||||
maxOutstandingReadRequests,
|
||||
maxWriteRequestOperationCount,
|
||||
maxWriteRequestSize,
|
||||
maxOutstandingWriteRequests,
|
||||
maxWriteBufferCount,
|
||||
maxWriteBufferSize,
|
||||
maxRetryDelay,
|
||||
|
|
|
@ -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<Map.Entry<Long, Tuple<Integer, ElasticsearchException>>, Void> FETCH_EXCEPTIONS_ENTRY_PARSER =
|
||||
static final ConstructingObjectParser<Map.Entry<Long, Tuple<Integer, ElasticsearchException>>, 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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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<Long, Tuple<Integer, ElasticsearchException>> fetchExceptions;
|
||||
private final NavigableMap<Long, Tuple<Integer, ElasticsearchException>> readExceptions;
|
||||
|
||||
public NavigableMap<Long, Tuple<Integer, ElasticsearchException>> fetchExceptions() {
|
||||
return fetchExceptions;
|
||||
public NavigableMap<Long, Tuple<Integer, ElasticsearchException>> 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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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<Long, Tuple<Integer, ElasticsearchException>> 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<Long, Tuple<Integer, ElasticsearchException>> readExceptions,
|
||||
final long timeSinceLastReadMillis,
|
||||
final ElasticsearchException fatalException) {
|
||||
this.remoteCluster = remoteCluster;
|
||||
this.leaderIndex = leaderIndex;
|
||||
|
@ -349,23 +349,23 @@ public class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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<Long, Tuple<Integer, ElasticsearchException>> entry : fetchExceptions.entrySet()) {
|
||||
for (final Map.Entry<Long, Tuple<Integer, ElasticsearchException>> 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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
}
|
||||
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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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 class ShardFollowNodeTaskStatus implements Task.Status {
|
|||
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<String> getFetchExceptionMessages(final ShardFollowNodeTaskStatus status) {
|
||||
return status.fetchExceptions().values().stream().map(t -> t.v2().getMessage()).collect(Collectors.toList());
|
||||
private static List<String> getReadExceptionMessages(final ShardFollowNodeTaskStatus status) {
|
||||
return status.readExceptions().values().stream().map(t -> t.v2().getMessage()).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -52,14 +52,20 @@ public class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
private List<String> 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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
this.maxConcurrentReadBatches = maxConcurrentReadBatches;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
public Integer getMaxWriteRequestOperationCount() {
|
||||
return maxWriteRequestOperationCount;
|
||||
}
|
||||
|
||||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 (maxBatchSize != null) {
|
||||
builder.field(AutoFollowPattern.MAX_BATCH_SIZE.getPreferredName(), maxBatchSize.getStringRep());
|
||||
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 (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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
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 class PutAutoFollowPatternAction extends Action<AcknowledgedResponse> {
|
|||
remoteCluster,
|
||||
leaderIndexPatterns,
|
||||
followIndexNamePattern,
|
||||
maxBatchOperationCount,
|
||||
maxReadRequestOperationCount,
|
||||
maxReadRequestSize,
|
||||
maxConcurrentReadBatches,
|
||||
maxBatchSize,
|
||||
maxWriteRequestOperationCount,
|
||||
maxWriteRequestSize,
|
||||
maxConcurrentWriteBatches,
|
||||
maxWriteBufferCount,
|
||||
maxWriteBufferSize,
|
||||
maxRetryDelay,
|
||||
pollTimeout);
|
||||
readPollTimeout);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -27,14 +27,14 @@ import java.util.Objects;
|
|||
|
||||
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<PutFollowAction.Response> {
|
||||
|
||||
|
@ -64,30 +64,30 @@ public final class PutFollowAction extends Action<PutFollowAction.Response> {
|
|||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -44,26 +44,31 @@ public final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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<Request, String> 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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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 (maxConcurrentReadBatches != null && maxConcurrentReadBatches < 1) {
|
||||
e = addValidationError(MAX_CONCURRENT_READ_BATCHES.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 (maxBatchSize != null && maxBatchSize.compareTo(ByteSizeValue.ZERO) <= 0) {
|
||||
e = addValidationError(MAX_BATCH_SIZE.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 (maxConcurrentWriteBatches != null && maxConcurrentWriteBatches < 1) {
|
||||
e = addValidationError(MAX_CONCURRENT_WRITE_BATCHES.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 (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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
|
||||
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 (maxBatchSize != null) {
|
||||
builder.field(MAX_BATCH_SIZE.getPreferredName(), maxBatchSize.getStringRep());
|
||||
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 (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 @@ public final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
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 final class ResumeFollowAction extends Action<AcknowledgedResponse> {
|
|||
public int hashCode() {
|
||||
return Objects.hash(
|
||||
followerIndex,
|
||||
maxBatchOperationCount,
|
||||
maxConcurrentReadBatches,
|
||||
maxBatchSize,
|
||||
maxConcurrentWriteBatches,
|
||||
maxReadRequestOperationCount,
|
||||
maxReadRequestSize,
|
||||
maxOutstandingReadRequests,
|
||||
maxWriteRequestOperationCount,
|
||||
maxWriteRequestSize,
|
||||
maxOutstandingWriteRequests,
|
||||
maxWriteBufferCount,
|
||||
maxWriteBufferSize,
|
||||
maxRetryDelay,
|
||||
pollTimeout);
|
||||
readPollTimeout);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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"
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue