|
@@ -57,6 +57,8 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
private Queue<Exception> writeFailures;
|
|
|
private Queue<Exception> mappingUpdateFailures;
|
|
|
private Queue<Long> mappingVersions;
|
|
|
+ private Queue<Exception> settingsUpdateFailures;
|
|
|
+ private Queue<Long> settingsVersions;
|
|
|
private Queue<Long> leaderGlobalCheckpoints;
|
|
|
private Queue<Long> followerGlobalCheckpoints;
|
|
|
private Queue<Long> maxSeqNos;
|
|
@@ -73,7 +75,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
task.coordinateReads();
|
|
|
assertThat(shardChangesRequests, contains(new long[]{0L, 8L})); // treat this a peak request
|
|
|
shardChangesRequests.clear();
|
|
|
- task.innerHandleReadResponse(0, 5L, generateShardChangesResponse(0, 5L, 0L, 60L));
|
|
|
+ task.innerHandleReadResponse(0, 5L, generateShardChangesResponse(0, 5L, 0L, 0L, 60L));
|
|
|
assertThat(shardChangesRequests, contains(new long[][]{
|
|
|
{6L, 8L}, {14L, 8L}, {22L, 8L}, {30L, 8L}, {38L, 8L}, {46L, 8L}, {54L, 7L}}
|
|
|
));
|
|
@@ -98,7 +100,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
|
|
|
shardChangesRequests.clear();
|
|
|
// Also invokes the coordinatesReads() method:
|
|
|
- task.innerHandleReadResponse(0L, 63L, generateShardChangesResponse(0, 63, 0L, 128L));
|
|
|
+ task.innerHandleReadResponse(0L, 63L, generateShardChangesResponse(0, 63, 0L, 0L, 128L));
|
|
|
assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because write buffer count limit has been reached
|
|
|
|
|
|
ShardFollowNodeTaskStatus status = task.getStatus();
|
|
@@ -124,7 +126,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
|
|
|
shardChangesRequests.clear();
|
|
|
// Also invokes the coordinatesReads() method:
|
|
|
- task.innerHandleReadResponse(0L, 63L, generateShardChangesResponse(0, 63, 0L, 128L));
|
|
|
+ task.innerHandleReadResponse(0L, 63L, generateShardChangesResponse(0, 63, 0L, 0L, 128L));
|
|
|
assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because write buffer size limit has been reached
|
|
|
|
|
|
ShardFollowNodeTaskStatus status = task.getStatus();
|
|
@@ -189,7 +191,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
task.markAsCompleted();
|
|
|
shardChangesRequests.clear();
|
|
|
// Also invokes the coordinatesReads() method:
|
|
|
- task.innerHandleReadResponse(0L, 15L, generateShardChangesResponse(0, 15, 0L, 31L));
|
|
|
+ task.innerHandleReadResponse(0L, 15L, generateShardChangesResponse(0, 15, 0L, 0L, 31L));
|
|
|
assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because task has been cancelled
|
|
|
assertThat(bulkShardOperationRequests.size(), equalTo(0)); // no more writes, because task has been cancelled
|
|
|
|
|
@@ -219,7 +221,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
task.markAsCompleted();
|
|
|
shardChangesRequests.clear();
|
|
|
// Also invokes the coordinatesReads() method:
|
|
|
- task.innerHandleReadResponse(0L, 63L, generateShardChangesResponse(0, 63, 0L, 128L));
|
|
|
+ task.innerHandleReadResponse(0L, 63L, generateShardChangesResponse(0, 63, 0L, 0L, 128L));
|
|
|
assertThat(shardChangesRequests.size(), equalTo(0)); // no more reads, because task has been cancelled
|
|
|
assertThat(bulkShardOperationRequests.size(), equalTo(0)); // no more writes, because task has been cancelled
|
|
|
|
|
@@ -439,7 +441,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
startTask(task, 63, -1);
|
|
|
|
|
|
task.coordinateReads();
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 63L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 0L, 63L);
|
|
|
task.innerHandleReadResponse(0L, 63L, response);
|
|
|
|
|
|
assertThat(bulkShardOperationRequests.size(), equalTo(1));
|
|
@@ -469,7 +471,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
assertThat(shardChangesRequests.get(0)[1], equalTo(64L));
|
|
|
|
|
|
shardChangesRequests.clear();
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 20, 0L, 31L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 20, 0L, 0L, 31L);
|
|
|
task.innerHandleReadResponse(0L, 63L, response);
|
|
|
|
|
|
assertThat(shardChangesRequests.size(), equalTo(1));
|
|
@@ -498,7 +500,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
|
|
|
shardChangesRequests.clear();
|
|
|
task.markAsCompleted();
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 31, 0L, 31L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 31, 0L, 0L, 31L);
|
|
|
task.innerHandleReadResponse(0L, 64L, response);
|
|
|
|
|
|
assertThat(shardChangesRequests.size(), equalTo(0));
|
|
@@ -524,7 +526,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
assertThat(shardChangesRequests.get(0)[1], equalTo(64L));
|
|
|
|
|
|
shardChangesRequests.clear();
|
|
|
- task.innerHandleReadResponse(0L, 63L, new ShardChangesAction.Response(0, 0, 0, 100, new Translog.Operation[0], 1L));
|
|
|
+ task.innerHandleReadResponse(0L, 63L, new ShardChangesAction.Response(0, 0, 0, 0, 100, new Translog.Operation[0], 1L));
|
|
|
|
|
|
assertThat(shardChangesRequests.size(), equalTo(1));
|
|
|
assertThat(shardChangesRequests.get(0)[0], equalTo(0L));
|
|
@@ -547,7 +549,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
|
|
|
mappingVersions.add(1L);
|
|
|
task.coordinateReads();
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 1L, 63L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 1L, 0L, 63L);
|
|
|
task.handleReadResponse(0L, 63L, response);
|
|
|
|
|
|
assertThat(bulkShardOperationRequests.size(), equalTo(1));
|
|
@@ -576,7 +578,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
}
|
|
|
mappingVersions.add(1L);
|
|
|
task.coordinateReads();
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 1L, 63L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 1L, 0L, 63L);
|
|
|
task.handleReadResponse(0L, 63L, response);
|
|
|
|
|
|
assertThat(mappingUpdateFailures.size(), equalTo(0));
|
|
@@ -601,7 +603,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
|
|
|
mappingUpdateFailures.add(new RuntimeException());
|
|
|
task.coordinateReads();
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 64, 1L, 64L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 64, 1L, 0L, 64L);
|
|
|
task.handleReadResponse(0L, 64L, response);
|
|
|
|
|
|
assertThat(bulkShardOperationRequests.size(), equalTo(0));
|
|
@@ -614,6 +616,85 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
assertThat(status.leaderGlobalCheckpoint(), equalTo(63L));
|
|
|
}
|
|
|
|
|
|
+ public void testSettingsUpdate() {
|
|
|
+ ShardFollowTaskParams params = new ShardFollowTaskParams();
|
|
|
+ params.maxReadRequestOperationCount = 64;
|
|
|
+ params.maxOutstandingReadRequests = 1;
|
|
|
+ params.maxOutstandingWriteRequests = 1;
|
|
|
+ ShardFollowNodeTask task = createShardFollowTask(params);
|
|
|
+ startTask(task, 63, -1);
|
|
|
+
|
|
|
+ settingsVersions.add(1L);
|
|
|
+ task.coordinateReads();
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 1L, 63L);
|
|
|
+ task.handleReadResponse(0L, 63L, response);
|
|
|
+
|
|
|
+ assertThat(bulkShardOperationRequests.size(), equalTo(1));
|
|
|
+ assertThat(bulkShardOperationRequests.get(0), equalTo(Arrays.asList(response.getOperations())));
|
|
|
+
|
|
|
+ ShardFollowNodeTaskStatus status = task.getStatus();
|
|
|
+ assertThat(status.followerMappingVersion(), equalTo(0L));
|
|
|
+ assertThat(status.followerSettingsVersion(), 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 testSettingsUpdateRetryableError() {
|
|
|
+ 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);
|
|
|
+ for (int i = 0; i < max; i++) {
|
|
|
+ settingsUpdateFailures.add(new ConnectException());
|
|
|
+ }
|
|
|
+ settingsVersions.add(1L);
|
|
|
+ task.coordinateReads();
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 1L, 63L);
|
|
|
+ task.handleReadResponse(0L, 63L, response);
|
|
|
+
|
|
|
+ assertThat(mappingUpdateFailures.size(), equalTo(0));
|
|
|
+ assertThat(bulkShardOperationRequests.size(), equalTo(1));
|
|
|
+ assertThat(task.isStopped(), equalTo(false));
|
|
|
+ ShardFollowNodeTaskStatus status = task.getStatus();
|
|
|
+ assertThat(status.followerMappingVersion(), equalTo(0L));
|
|
|
+ assertThat(status.followerSettingsVersion(), equalTo(1L));
|
|
|
+ assertThat(status.outstandingReadRequests(), equalTo(1));
|
|
|
+ assertThat(status.outstandingWriteRequests(), equalTo(1));
|
|
|
+ assertThat(status.lastRequestedSeqNo(), equalTo(63L));
|
|
|
+ assertThat(status.leaderGlobalCheckpoint(), equalTo(63L));
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testSettingsUpdateNonRetryableError() {
|
|
|
+ ShardFollowTaskParams params = new ShardFollowTaskParams();
|
|
|
+ params.maxReadRequestOperationCount = 64;
|
|
|
+ params.maxOutstandingReadRequests = 1;
|
|
|
+ params.maxOutstandingWriteRequests = 1;
|
|
|
+ ShardFollowNodeTask task = createShardFollowTask(params);
|
|
|
+ startTask(task, 63, -1);
|
|
|
+
|
|
|
+ settingsUpdateFailures.add(new RuntimeException());
|
|
|
+ task.coordinateReads();
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 64, 0L, 1L, 64L);
|
|
|
+ task.handleReadResponse(0L, 64L, response);
|
|
|
+
|
|
|
+ assertThat(bulkShardOperationRequests.size(), equalTo(0));
|
|
|
+ assertThat(task.isStopped(), equalTo(true));
|
|
|
+ ShardFollowNodeTaskStatus status = task.getStatus();
|
|
|
+ assertThat(status.followerMappingVersion(), equalTo(0L));
|
|
|
+ assertThat(status.followerSettingsVersion(), 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() {
|
|
|
ShardFollowTaskParams params = new ShardFollowTaskParams();
|
|
|
params.maxReadRequestOperationCount = 128;
|
|
@@ -629,7 +710,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
assertThat(shardChangesRequests.get(0)[0], equalTo(0L));
|
|
|
assertThat(shardChangesRequests.get(0)[1], equalTo(128L));
|
|
|
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 63L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 0L, 63L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 63L, response);
|
|
|
|
|
@@ -649,7 +730,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
params.maxWriteRequestOperationCount = 64;
|
|
|
params.maxOutstandingWriteRequests = 2;
|
|
|
ShardFollowNodeTask task = createShardFollowTask(params);
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 256, 0L, 256L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 256, 0L, 0L, 256L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 64L, response);
|
|
|
|
|
@@ -662,7 +743,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
|
|
|
params.maxOutstandingWriteRequests = 4; // change to 4 outstanding writers
|
|
|
task = createShardFollowTask(params);
|
|
|
- response = generateShardChangesResponse(0, 256, 0L, 256L);
|
|
|
+ response = generateShardChangesResponse(0, 256, 0L, 0L, 256L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 64L, response);
|
|
|
|
|
@@ -681,7 +762,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
params.maxWriteRequestOperationCount = 8;
|
|
|
params.maxOutstandingWriteRequests = 32;
|
|
|
ShardFollowNodeTask task = createShardFollowTask(params);
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 256, 0L, 256L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 256, 0L, 0L, 256L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 64L, response);
|
|
|
|
|
@@ -712,7 +793,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
for (int i = 0; i < max; i++) {
|
|
|
writeFailures.add(new ShardNotFoundException(new ShardId("leader_index", "", 0)));
|
|
|
}
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 63L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 0L, 63L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 63L, response);
|
|
|
|
|
@@ -741,7 +822,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
assertThat(shardChangesRequests.get(0)[1], equalTo(64L));
|
|
|
|
|
|
writeFailures.add(new RuntimeException());
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 63L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 0L, 63L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 63L, response);
|
|
|
|
|
@@ -768,7 +849,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
assertThat(shardChangesRequests.get(0)[0], equalTo(0L));
|
|
|
assertThat(shardChangesRequests.get(0)[1], equalTo(64L));
|
|
|
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 64L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 0L, 64L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 64L, response);
|
|
|
|
|
@@ -791,7 +872,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
|
|
|
shardChangesRequests.clear();
|
|
|
followerGlobalCheckpoints.add(63L);
|
|
|
- ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 63L);
|
|
|
+ ShardChangesAction.Response response = generateShardChangesResponse(0, 63, 0L, 0L, 63L);
|
|
|
// Also invokes coordinatesWrites()
|
|
|
task.innerHandleReadResponse(0L, 63L, response);
|
|
|
|
|
@@ -866,6 +947,8 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
writeFailures = new LinkedList<>();
|
|
|
mappingUpdateFailures = new LinkedList<>();
|
|
|
mappingVersions = new LinkedList<>();
|
|
|
+ settingsUpdateFailures = new LinkedList<>();
|
|
|
+ settingsVersions = new LinkedList<>();
|
|
|
leaderGlobalCheckpoints = new LinkedList<>();
|
|
|
followerGlobalCheckpoints = new LinkedList<>();
|
|
|
maxSeqNos = new LinkedList<>();
|
|
@@ -887,6 +970,20 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ protected void innerUpdateSettings(LongConsumer handler, Consumer<Exception> errorHandler) {
|
|
|
+ Exception failure = settingsUpdateFailures.poll();
|
|
|
+ if (failure != null) {
|
|
|
+ errorHandler.accept(failure);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ final Long settingsVersion = settingsVersions.poll();
|
|
|
+ if (settingsVersion != null) {
|
|
|
+ handler.accept(settingsVersion);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
protected void innerSendBulkShardOperationsRequest(
|
|
|
String followerHistoryUUID, final List<Translog.Operation> operations,
|
|
@@ -924,6 +1021,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
}
|
|
|
final ShardChangesAction.Response response = new ShardChangesAction.Response(
|
|
|
mappingVersions.poll(),
|
|
|
+ 0L,
|
|
|
leaderGlobalCheckpoints.poll(),
|
|
|
maxSeqNos.poll(),
|
|
|
randomNonNegativeLong(),
|
|
@@ -946,7 +1044,10 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
};
|
|
|
}
|
|
|
|
|
|
- private static ShardChangesAction.Response generateShardChangesResponse(long fromSeqNo, long toSeqNo, long mappingVersion,
|
|
|
+ private static ShardChangesAction.Response generateShardChangesResponse(long fromSeqNo,
|
|
|
+ long toSeqNo,
|
|
|
+ long mappingVersion,
|
|
|
+ long settingsVersion,
|
|
|
long leaderGlobalCheckPoint) {
|
|
|
List<Translog.Operation> ops = new ArrayList<>();
|
|
|
for (long seqNo = fromSeqNo; seqNo <= toSeqNo; seqNo++) {
|
|
@@ -956,6 +1057,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
|
|
|
}
|
|
|
return new ShardChangesAction.Response(
|
|
|
mappingVersion,
|
|
|
+ settingsVersion,
|
|
|
leaderGlobalCheckPoint,
|
|
|
leaderGlobalCheckPoint,
|
|
|
randomNonNegativeLong(),
|