Skip to content

Commit

Permalink
[CCR] Add more unit tests for shard follow task (#32121)
Browse files Browse the repository at this point in the history
The added tests are based on specific scenarios as described in the test plan.
Before this change the ShardFollowNodeTaskTests contained more random like tests,
but these have been removed and in a followup pr better random tests will
be added in a new test class as is described in the test plan.
  • Loading branch information
martijnvg authored Jul 20, 2018
1 parent d0f3ed5 commit a6b7497
Show file tree
Hide file tree
Showing 7 changed files with 730 additions and 227 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -277,7 +277,8 @@ static Translog.Operation[] getOperations(IndexShard indexShard, long globalChec
return EMPTY_OPERATIONS_ARRAY;
}
int seenBytes = 0;
long toSeqNo = Math.min(globalCheckpoint, fromSeqNo + maxOperationCount);
// - 1 is needed, because toSeqNo is inclusive
long toSeqNo = Math.min(globalCheckpoint, (fromSeqNo + maxOperationCount) - 1);
final List<Translog.Operation> operations = new ArrayList<>();
try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", fromSeqNo, toSeqNo, true)) {
Translog.Operation op;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
private volatile int numConcurrentWrites = 0;
private volatile long followerGlobalCheckpoint = 0;
private volatile long currentIndexMetadataVersion = 0;
private final Queue<Translog.Operation> buffer = new PriorityQueue<>(Comparator.comparing(Translog.Operation::seqNo).reversed());
private final Queue<Translog.Operation> buffer = new PriorityQueue<>(Comparator.comparing(Translog.Operation::seqNo));

ShardFollowNodeTask(long id, String type, String action, String description, TaskId parentTask, Map<String, String> headers,
ShardFollowTask params, BiConsumer<TimeValue, Runnable> scheduler) {
Expand All @@ -78,10 +78,10 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
this.idleShardChangesRequestDelay = params.getIdleShardRetryDelay();
}

void start(long followerGlobalCheckpoint) {
void start(long leaderGlobalCheckpoint, long followerGlobalCheckpoint) {
this.lastRequestedSeqno = followerGlobalCheckpoint;
this.followerGlobalCheckpoint = followerGlobalCheckpoint;
this.leaderGlobalCheckpoint = followerGlobalCheckpoint;
this.leaderGlobalCheckpoint = leaderGlobalCheckpoint;

// Forcefully updates follower mapping, this gets us the leader imd version and
// makes sure that leader and follower mapping are identical.
Expand All @@ -93,7 +93,7 @@ void start(long followerGlobalCheckpoint) {
});
}

private synchronized void coordinateReads() {
synchronized void coordinateReads() {
if (isStopped()) {
LOGGER.info("{} shard follow task has been stopped", params.getFollowShardId());
return;
Expand All @@ -105,7 +105,8 @@ private synchronized void coordinateReads() {
while (hasReadBudget() && lastRequestedSeqno < leaderGlobalCheckpoint) {
numConcurrentReads++;
long from = lastRequestedSeqno + 1;
long maxRequiredSeqno = Math.min(leaderGlobalCheckpoint, from + maxBatchOperationCount);
// -1 is needed, because maxRequiredSeqno is inclusive
long maxRequiredSeqno = Math.min(leaderGlobalCheckpoint, (from + maxBatchOperationCount) - 1);
LOGGER.trace("{}[{}] read [{}/{}]", params.getFollowShardId(), numConcurrentReads, maxRequiredSeqno, maxBatchOperationCount);
sendShardChangesRequest(from, maxBatchOperationCount, maxRequiredSeqno);
lastRequestedSeqno = maxRequiredSeqno;
Expand Down Expand Up @@ -137,6 +138,11 @@ private boolean hasReadBudget() {
}

private synchronized void coordinateWrites() {
if (isStopped()) {
LOGGER.info("{} shard follow task has been stopped", params.getFollowShardId());
return;
}

while (hasWriteBudget() && buffer.isEmpty() == false) {
long sumEstimatedSize = 0L;
int length = Math.min(params.getMaxBatchOperationCount(), buffer.size());
Expand Down Expand Up @@ -176,48 +182,48 @@ private void sendShardChangesRequest(long from, int maxOperationCount, long maxR
e -> handleFailure(e, retryCounter, () -> sendShardChangesRequest(from, maxOperationCount, maxRequiredSeqNo, retryCounter)));
}

private void handleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAction.Response response) {
maybeUpdateMapping(response.getIndexMetadataVersion(), () -> {
synchronized (ShardFollowNodeTask.this) {
leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getGlobalCheckpoint());
final long newMinRequiredSeqNo;
if (response.getOperations().length == 0) {
newMinRequiredSeqNo = from;
} else {
assert response.getOperations()[0].seqNo() == from :
"first operation is not what we asked for. From is [" + from + "], got " + response.getOperations()[0];
buffer.addAll(Arrays.asList(response.getOperations()));
final long maxSeqNo = response.getOperations()[response.getOperations().length - 1].seqNo();
assert maxSeqNo==
Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::seqNo).max().getAsLong();
newMinRequiredSeqNo = maxSeqNo + 1;
// update last requested seq no as we may have gotten more than we asked for and we don't want to ask it again.
lastRequestedSeqno = Math.max(lastRequestedSeqno, maxSeqNo);
assert lastRequestedSeqno <= leaderGlobalCheckpoint : "lastRequestedSeqno [" + lastRequestedSeqno +
"] is larger than the global checkpoint [" + leaderGlobalCheckpoint + "]";
coordinateWrites();
}
void handleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAction.Response response) {
maybeUpdateMapping(response.getIndexMetadataVersion(), () -> innerHandleReadResponse(from, maxRequiredSeqNo, response));
}

if (newMinRequiredSeqNo < maxRequiredSeqNo) {
int newSize = (int) (maxRequiredSeqNo - newMinRequiredSeqNo) + 1;
LOGGER.trace("{} received [{}] ops, still missing [{}/{}], continuing to read...",
params.getFollowShardId(), response.getOperations().length, newMinRequiredSeqNo, maxRequiredSeqNo);
sendShardChangesRequest(newMinRequiredSeqNo, newSize, maxRequiredSeqNo);
} else {
// read is completed, decrement
numConcurrentReads--;
if (response.getOperations().length == 0 && leaderGlobalCheckpoint == lastRequestedSeqno) {
// we got nothing and we have no reason to believe asking again well get us more, treat shard as idle and delay
// future requests
LOGGER.trace("{} received no ops and no known ops to fetch, scheduling to coordinate reads",
params.getFollowShardId());
scheduler.accept(idleShardChangesRequestDelay, this::coordinateReads);
} else {
coordinateReads();
}
}
synchronized void innerHandleReadResponse(long from, long maxRequiredSeqNo, ShardChangesAction.Response response) {
leaderGlobalCheckpoint = Math.max(leaderGlobalCheckpoint, response.getGlobalCheckpoint());
final long newMinRequiredSeqNo;
if (response.getOperations().length == 0) {
newMinRequiredSeqNo = from;
} else {
assert response.getOperations()[0].seqNo() == from :
"first operation is not what we asked for. From is [" + from + "], got " + response.getOperations()[0];
buffer.addAll(Arrays.asList(response.getOperations()));
final long maxSeqNo = response.getOperations()[response.getOperations().length - 1].seqNo();
assert maxSeqNo ==
Arrays.stream(response.getOperations()).mapToLong(Translog.Operation::seqNo).max().getAsLong();
newMinRequiredSeqNo = maxSeqNo + 1;
// update last requested seq no as we may have gotten more than we asked for and we don't want to ask it again.
lastRequestedSeqno = Math.max(lastRequestedSeqno, maxSeqNo);
assert lastRequestedSeqno <= leaderGlobalCheckpoint : "lastRequestedSeqno [" + lastRequestedSeqno +
"] is larger than the global checkpoint [" + leaderGlobalCheckpoint + "]";
coordinateWrites();
}

if (newMinRequiredSeqNo < maxRequiredSeqNo && isStopped() == false) {
int newSize = (int) (maxRequiredSeqNo - newMinRequiredSeqNo) + 1;
LOGGER.trace("{} received [{}] ops, still missing [{}/{}], continuing to read...",
params.getFollowShardId(), response.getOperations().length, newMinRequiredSeqNo, maxRequiredSeqNo);
sendShardChangesRequest(newMinRequiredSeqNo, newSize, maxRequiredSeqNo);
} else {
// read is completed, decrement
numConcurrentReads--;
if (response.getOperations().length == 0 && leaderGlobalCheckpoint == lastRequestedSeqno) {
// we got nothing and we have no reason to believe asking again well get us more, treat shard as idle and delay
// future requests
LOGGER.trace("{} received no ops and no known ops to fetch, scheduling to coordinate reads",
params.getFollowShardId());
scheduler.accept(idleShardChangesRequestDelay, this::coordinateReads);
} else {
coordinateReads();
}
});
}
}

private void sendBulkShardOperationsRequest(List<Translog.Operation> operations) {
Expand Down Expand Up @@ -306,7 +312,8 @@ protected boolean isStopped() {

@Override
public Status getStatus() {
return new Status(leaderGlobalCheckpoint, lastRequestedSeqno, followerGlobalCheckpoint, numConcurrentReads, numConcurrentWrites);
return new Status(leaderGlobalCheckpoint, lastRequestedSeqno, followerGlobalCheckpoint, numConcurrentReads, numConcurrentWrites,
currentIndexMetadataVersion);
}

public static class Status implements Task.Status {
Expand All @@ -318,31 +325,35 @@ public static class Status implements Task.Status {
static final ParseField LAST_REQUESTED_SEQNO_FIELD = new ParseField("last_requested_seqno");
static final ParseField NUMBER_OF_CONCURRENT_READS_FIELD = new ParseField("number_of_concurrent_reads");
static final ParseField NUMBER_OF_CONCURRENT_WRITES_FIELD = new ParseField("number_of_concurrent_writes");
static final ParseField INDEX_METADATA_VERSION_FIELD = new ParseField("index_metadata_version");

static final ConstructingObjectParser<Status, Void> PARSER = new ConstructingObjectParser<>(NAME,
args -> new Status((long) args[0], (long) args[1], (long) args[2], (int) args[3], (int) args[4]));
args -> new Status((long) args[0], (long) args[1], (long) args[2], (int) args[3], (int) args[4], (long) args[5]));

static {
PARSER.declareLong(ConstructingObjectParser.constructorArg(), LEADER_GLOBAL_CHECKPOINT_FIELD);
PARSER.declareLong(ConstructingObjectParser.constructorArg(), LAST_REQUESTED_SEQNO_FIELD);
PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_GLOBAL_CHECKPOINT_FIELD);
PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_READS_FIELD);
PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_CONCURRENT_WRITES_FIELD);
PARSER.declareLong(ConstructingObjectParser.constructorArg(), INDEX_METADATA_VERSION_FIELD);
}

private final long leaderGlobalCheckpoint;
private final long lastRequestedSeqno;
private final long followerGlobalCheckpoint;
private final int numberOfConcurrentReads;
private final int numberOfConcurrentWrites;
private final long indexMetadataVersion;

Status(long leaderGlobalCheckpoint, long lastRequestedSeqno, long followerGlobalCheckpoint,
int numberOfConcurrentReads, int numberOfConcurrentWrites) {
int numberOfConcurrentReads, int numberOfConcurrentWrites, long indexMetadataVersion) {
this.leaderGlobalCheckpoint = leaderGlobalCheckpoint;
this.lastRequestedSeqno = lastRequestedSeqno;
this.followerGlobalCheckpoint = followerGlobalCheckpoint;
this.numberOfConcurrentReads = numberOfConcurrentReads;
this.numberOfConcurrentWrites = numberOfConcurrentWrites;
this.indexMetadataVersion = indexMetadataVersion;
}

public Status(StreamInput in) throws IOException {
Expand All @@ -351,6 +362,7 @@ public Status(StreamInput in) throws IOException {
this.followerGlobalCheckpoint = in.readZLong();
this.numberOfConcurrentReads = in.readVInt();
this.numberOfConcurrentWrites = in.readVInt();
this.indexMetadataVersion = in.readVLong();
}

public long getLeaderGlobalCheckpoint() {
Expand All @@ -373,6 +385,10 @@ public int getNumberOfConcurrentWrites() {
return numberOfConcurrentWrites;
}

public long getIndexMetadataVersion() {
return indexMetadataVersion;
}

@Override
public String getWriteableName() {
return NAME;
Expand All @@ -385,6 +401,7 @@ public void writeTo(StreamOutput out) throws IOException {
out.writeZLong(followerGlobalCheckpoint);
out.writeVInt(numberOfConcurrentReads);
out.writeVInt(numberOfConcurrentWrites);
out.writeVLong(indexMetadataVersion);
}

@Override
Expand All @@ -396,6 +413,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
builder.field(LAST_REQUESTED_SEQNO_FIELD.getPreferredName(), lastRequestedSeqno);
builder.field(NUMBER_OF_CONCURRENT_READS_FIELD.getPreferredName(), numberOfConcurrentReads);
builder.field(NUMBER_OF_CONCURRENT_WRITES_FIELD.getPreferredName(), numberOfConcurrentWrites);
builder.field(INDEX_METADATA_VERSION_FIELD.getPreferredName(), indexMetadataVersion);
}
builder.endObject();
return builder;
Expand All @@ -414,13 +432,14 @@ public boolean equals(Object o) {
lastRequestedSeqno == status.lastRequestedSeqno &&
followerGlobalCheckpoint == status.followerGlobalCheckpoint &&
numberOfConcurrentReads == status.numberOfConcurrentReads &&
numberOfConcurrentWrites == status.numberOfConcurrentWrites;
numberOfConcurrentWrites == status.numberOfConcurrentWrites &&
indexMetadataVersion == status.indexMetadataVersion;
}

@Override
public int hashCode() {
return Objects.hash(leaderGlobalCheckpoint, lastRequestedSeqno, followerGlobalCheckpoint, numberOfConcurrentReads,
numberOfConcurrentWrites);
numberOfConcurrentWrites, indexMetadataVersion);
}

public String toString() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,8 @@ protected void nodeOperation(final AllocatedPersistentTask task, final ShardFoll
Client followerClient = wrapClient(client, params);
ShardFollowNodeTask shardFollowNodeTask = (ShardFollowNodeTask) task;
logger.info("{} Started to track leader shard {}", params.getFollowShardId(), params.getLeaderShardId());
fetchGlobalCheckpoint(followerClient, params.getFollowShardId(), shardFollowNodeTask::start, task::markAsFailed);
fetchGlobalCheckpoint(followerClient, params.getFollowShardId(),
followerGCP -> shardFollowNodeTask.start(followerGCP, followerGCP), task::markAsFailed);
}

private void fetchGlobalCheckpoint(Client client, ShardId shardId, LongConsumer handler, Consumer<Exception> errorHandler) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public void testGetOperations() throws Exception {
final Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard,
indexShard.getGlobalCheckpoint(), min, size, Long.MAX_VALUE);
final List<Long> seenSeqNos = Arrays.stream(operations).map(Translog.Operation::seqNo).collect(Collectors.toList());
final List<Long> expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toList());
final List<Long> expectedSeqNos = LongStream.range(min, max).boxed().collect(Collectors.toList());
assertThat(seenSeqNos, equalTo(expectedSeqNos));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ protected ShardFollowNodeTask.Status doParseInstance(XContentParser parser) thro
@Override
protected ShardFollowNodeTask.Status createTestInstance() {
return new ShardFollowNodeTask.Status(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(),
randomIntBetween(0, Integer.MAX_VALUE), randomIntBetween(0, Integer.MAX_VALUE));
randomIntBetween(0, Integer.MAX_VALUE), randomIntBetween(0, Integer.MAX_VALUE), randomNonNegativeLong());
}

@Override
Expand Down
Loading

0 comments on commit a6b7497

Please sign in to comment.