|
|
@@ -59,10 +59,10 @@ import static org.hamcrest.Matchers.equalTo;
|
|
|
import static org.hamcrest.Matchers.greaterThan;
|
|
|
import static org.hamcrest.Matchers.not;
|
|
|
|
|
|
-public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
+public class ReplicationTrackerTests extends ESTestCase {
|
|
|
|
|
|
public void testEmptyShards() {
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(AllocationId.newInitializing());
|
|
|
+ final ReplicationTracker tracker = newTracker(AllocationId.newInitializing());
|
|
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
|
}
|
|
|
|
|
|
@@ -116,7 +116,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
|
|
|
|
final AllocationId primaryId = active.iterator().next();
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(primaryId);
|
|
|
+ final ReplicationTracker tracker = newTracker(primaryId);
|
|
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
|
|
|
|
logger.info("--> using allocations");
|
|
|
@@ -134,7 +134,9 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
|
tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet());
|
|
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
|
|
- initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED));
|
|
|
+ assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1));
|
|
|
+ initializing.forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED));
|
|
|
+ assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1 + initializing.size()));
|
|
|
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId.getId(), allocations.get(aId)));
|
|
|
|
|
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(minLocalCheckpoint));
|
|
|
@@ -164,9 +166,9 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
// now notify for the new id
|
|
|
if (randomBoolean()) {
|
|
|
tracker.updateLocalCheckpoint(extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
|
|
- markAllocationIdAsInSyncQuietly(tracker, extraId.getId(), randomInt((int) minLocalCheckpointAfterUpdates));
|
|
|
+ markAsTrackingAndInSyncQuietly(tracker, extraId.getId(), randomInt((int) minLocalCheckpointAfterUpdates));
|
|
|
} else {
|
|
|
- markAllocationIdAsInSyncQuietly(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
|
|
+ markAsTrackingAndInSyncQuietly(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
|
|
}
|
|
|
|
|
|
// now it should be incremented
|
|
|
@@ -180,10 +182,10 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
assigned.putAll(active);
|
|
|
assigned.putAll(initializing);
|
|
|
AllocationId primaryId = active.keySet().iterator().next();
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(primaryId);
|
|
|
+ final ReplicationTracker tracker = newTracker(primaryId);
|
|
|
tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet());
|
|
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
|
|
- randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
+ randomSubsetOf(initializing.keySet()).forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
final AllocationId missingActiveID = randomFrom(active.keySet());
|
|
|
assigned
|
|
|
.entrySet()
|
|
|
@@ -205,11 +207,11 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
logger.info("active: {}, initializing: {}", active, initializing);
|
|
|
|
|
|
AllocationId primaryId = active.keySet().iterator().next();
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(primaryId);
|
|
|
+ final ReplicationTracker tracker = newTracker(primaryId);
|
|
|
tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet());
|
|
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
|
|
randomSubsetOf(randomIntBetween(1, initializing.size() - 1),
|
|
|
- initializing.keySet()).forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED));
|
|
|
+ initializing.keySet()).forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED));
|
|
|
|
|
|
active.forEach((aid, localCP) -> tracker.updateLocalCheckpoint(aid.getId(), localCP));
|
|
|
|
|
|
@@ -225,12 +227,12 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final Map<AllocationId, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
final Map<AllocationId, Long> nonApproved = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
final AllocationId primaryId = active.keySet().iterator().next();
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(primaryId);
|
|
|
+ final ReplicationTracker tracker = newTracker(primaryId);
|
|
|
tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet());
|
|
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
|
|
- initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
+ initializing.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
nonApproved.keySet().forEach(k ->
|
|
|
- expectThrows(IllegalStateException.class, () -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)));
|
|
|
+ expectThrows(IllegalStateException.class, () -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)));
|
|
|
|
|
|
List<Map<AllocationId, Long>> allocations = Arrays.asList(active, initializing, nonApproved);
|
|
|
Collections.shuffle(allocations, random());
|
|
|
@@ -260,13 +262,13 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
if (randomBoolean()) {
|
|
|
allocations.putAll(initializingToBeRemoved);
|
|
|
}
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(primaryId);
|
|
|
+ final ReplicationTracker tracker = newTracker(primaryId);
|
|
|
tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet());
|
|
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
|
|
if (randomBoolean()) {
|
|
|
- initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
+ initializingToStay.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
} else {
|
|
|
- initializing.forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
+ initializing.forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
|
|
}
|
|
|
if (randomBoolean()) {
|
|
|
allocations.forEach((aid, localCP) -> tracker.updateLocalCheckpoint(aid.getId(), localCP));
|
|
|
@@ -302,7 +304,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final AtomicBoolean complete = new AtomicBoolean();
|
|
|
final AllocationId inSyncAllocationId = AllocationId.newInitializing();
|
|
|
final AllocationId trackingAllocationId = AllocationId.newInitializing();
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(inSyncAllocationId);
|
|
|
+ final ReplicationTracker tracker = newTracker(inSyncAllocationId);
|
|
|
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()),
|
|
|
routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId), emptySet());
|
|
|
tracker.activatePrimaryMode(globalCheckpoint);
|
|
|
@@ -310,6 +312,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
try {
|
|
|
// synchronize starting with the test thread
|
|
|
barrier.await();
|
|
|
+ tracker.initiateTracking(trackingAllocationId.getId());
|
|
|
tracker.markAllocationIdAsInSync(trackingAllocationId.getId(), localCheckpoint);
|
|
|
complete.set(true);
|
|
|
// synchronize with the test thread checking if we are no longer waiting
|
|
|
@@ -343,8 +346,8 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
thread.join();
|
|
|
}
|
|
|
|
|
|
- private GlobalCheckpointTracker newTracker(final AllocationId allocationId) {
|
|
|
- return new GlobalCheckpointTracker(
|
|
|
+ private ReplicationTracker newTracker(final AllocationId allocationId) {
|
|
|
+ return new ReplicationTracker(
|
|
|
new ShardId("test", "_na_", 0),
|
|
|
allocationId.getId(),
|
|
|
IndexSettingsModule.newIndexSettings("test", Settings.EMPTY),
|
|
|
@@ -358,7 +361,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final AtomicBoolean interrupted = new AtomicBoolean();
|
|
|
final AllocationId inSyncAllocationId = AllocationId.newInitializing();
|
|
|
final AllocationId trackingAllocationId = AllocationId.newInitializing();
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(inSyncAllocationId);
|
|
|
+ final ReplicationTracker tracker = newTracker(inSyncAllocationId);
|
|
|
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()),
|
|
|
routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId), emptySet());
|
|
|
tracker.activatePrimaryMode(globalCheckpoint);
|
|
|
@@ -370,6 +373,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
throw new RuntimeException(e);
|
|
|
}
|
|
|
try {
|
|
|
+ tracker.initiateTracking(trackingAllocationId.getId());
|
|
|
tracker.markAllocationIdAsInSync(trackingAllocationId.getId(), localCheckpoint);
|
|
|
} catch (final InterruptedException e) {
|
|
|
interrupted.set(true);
|
|
|
@@ -407,7 +411,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final Set<AllocationId> initializingIds = activeAndInitializingAllocationIds.v2();
|
|
|
AllocationId primaryId = activeAllocationIds.iterator().next();
|
|
|
IndexShardRoutingTable routingTable = routingTable(initializingIds, primaryId);
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(primaryId);
|
|
|
+ final ReplicationTracker tracker = newTracker(primaryId);
|
|
|
tracker.updateFromMaster(initialClusterStateVersion, ids(activeAllocationIds), routingTable, emptySet());
|
|
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
|
|
assertThat(tracker.getReplicationGroup().getInSyncAllocationIds(), equalTo(ids(activeAllocationIds)));
|
|
|
@@ -508,6 +512,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final Thread thread = new Thread(() -> {
|
|
|
try {
|
|
|
barrier.await();
|
|
|
+ tracker.initiateTracking(newSyncingAllocationId.getId());
|
|
|
tracker.markAllocationIdAsInSync(newSyncingAllocationId.getId(), localCheckpoint);
|
|
|
barrier.await();
|
|
|
} catch (final BrokenBarrierException | InterruptedException e) {
|
|
|
@@ -547,13 +552,13 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * If we do not update the global checkpoint in {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} after adding the
|
|
|
+ * If we do not update the global checkpoint in {@link ReplicationTracker#markAllocationIdAsInSync(String, long)} after adding the
|
|
|
* allocation ID to the in-sync set and removing it from pending, the local checkpoint update that freed the thread waiting for the
|
|
|
* local checkpoint to advance could miss updating the global checkpoint in a race if the waiting thread did not add the allocation
|
|
|
* ID to the in-sync set and remove it from the pending set before the local checkpoint updating thread executed the global checkpoint
|
|
|
- * update. This test fails without an additional call to {@link GlobalCheckpointTracker#updateGlobalCheckpointOnPrimary()} after
|
|
|
- * removing the allocation ID from the pending set in {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} (even if a
|
|
|
- * call is added after notifying all waiters in {@link GlobalCheckpointTracker#updateLocalCheckpoint(String, long)}).
|
|
|
+ * update. This test fails without an additional call to {@link ReplicationTracker#updateGlobalCheckpointOnPrimary()} after
|
|
|
+ * removing the allocation ID from the pending set in {@link ReplicationTracker#markAllocationIdAsInSync(String, long)} (even if a
|
|
|
+ * call is added after notifying all waiters in {@link ReplicationTracker#updateLocalCheckpoint(String, long)}).
|
|
|
*
|
|
|
* @throws InterruptedException if the main test thread was interrupted while waiting
|
|
|
* @throws BrokenBarrierException if the barrier was broken while the main test thread was waiting
|
|
|
@@ -565,7 +570,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final CyclicBarrier barrier = new CyclicBarrier(4);
|
|
|
|
|
|
final int activeLocalCheckpoint = randomIntBetween(0, Integer.MAX_VALUE - 1);
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(active);
|
|
|
+ final ReplicationTracker tracker = newTracker(active);
|
|
|
tracker.updateFromMaster(
|
|
|
randomNonNegativeLong(),
|
|
|
Collections.singleton(active.getId()),
|
|
|
@@ -595,6 +600,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final Thread markingThread = new Thread(() -> {
|
|
|
try {
|
|
|
barrier.await();
|
|
|
+ tracker.initiateTracking(initializing.getId());
|
|
|
tracker.markAllocationIdAsInSync(initializing.getId(), initializingLocalCheckpoint - 1);
|
|
|
} catch (final BrokenBarrierException | InterruptedException e) {
|
|
|
throw new RuntimeException(e);
|
|
|
@@ -619,10 +625,10 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
|
FakeClusterState clusterState = initialState();
|
|
|
final AllocationId primaryAllocationId = clusterState.routingTable.primaryShard().allocationId();
|
|
|
- GlobalCheckpointTracker oldPrimary =
|
|
|
- new GlobalCheckpointTracker(shardId, primaryAllocationId.getId(), indexSettings, UNASSIGNED_SEQ_NO);
|
|
|
- GlobalCheckpointTracker newPrimary =
|
|
|
- new GlobalCheckpointTracker(shardId, primaryAllocationId.getRelocationId(), indexSettings, UNASSIGNED_SEQ_NO);
|
|
|
+ ReplicationTracker oldPrimary =
|
|
|
+ new ReplicationTracker(shardId, primaryAllocationId.getId(), indexSettings, UNASSIGNED_SEQ_NO);
|
|
|
+ ReplicationTracker newPrimary =
|
|
|
+ new ReplicationTracker(shardId, primaryAllocationId.getRelocationId(), indexSettings, UNASSIGNED_SEQ_NO);
|
|
|
|
|
|
Set<String> allocationIds = new HashSet<>(Arrays.asList(oldPrimary.shardAllocationId, newPrimary.shardAllocationId));
|
|
|
|
|
|
@@ -647,12 +653,12 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
}
|
|
|
|
|
|
// simulate transferring the global checkpoint to the new primary after finalizing recovery before the handoff
|
|
|
- markAllocationIdAsInSyncQuietly(
|
|
|
+ markAsTrackingAndInSyncQuietly(
|
|
|
oldPrimary,
|
|
|
newPrimary.shardAllocationId,
|
|
|
Math.max(SequenceNumbers.NO_OPS_PERFORMED, oldPrimary.getGlobalCheckpoint() + randomInt(5)));
|
|
|
oldPrimary.updateGlobalCheckpointForShard(newPrimary.shardAllocationId, oldPrimary.getGlobalCheckpoint());
|
|
|
- GlobalCheckpointTracker.PrimaryContext primaryContext = oldPrimary.startRelocationHandoff();
|
|
|
+ ReplicationTracker.PrimaryContext primaryContext = oldPrimary.startRelocationHandoff();
|
|
|
|
|
|
if (randomBoolean()) {
|
|
|
// cluster state update after primary context handoff
|
|
|
@@ -685,7 +691,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
BytesStreamOutput output = new BytesStreamOutput();
|
|
|
primaryContext.writeTo(output);
|
|
|
StreamInput streamInput = output.bytes().streamInput();
|
|
|
- primaryContext = new GlobalCheckpointTracker.PrimaryContext(streamInput);
|
|
|
+ primaryContext = new ReplicationTracker.PrimaryContext(streamInput);
|
|
|
switch (randomInt(3)) {
|
|
|
case 0: {
|
|
|
// apply cluster state update on old primary while primary context is being transferred
|
|
|
@@ -730,10 +736,10 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
* will update its global checkpoint state without the old primary learning of it, and the old primary could have updated its
|
|
|
* global checkpoint state after the primary context was transferred.
|
|
|
*/
|
|
|
- Map<String, GlobalCheckpointTracker.CheckpointState> oldPrimaryCheckpointsCopy = new HashMap<>(oldPrimary.checkpoints);
|
|
|
+ Map<String, ReplicationTracker.CheckpointState> oldPrimaryCheckpointsCopy = new HashMap<>(oldPrimary.checkpoints);
|
|
|
oldPrimaryCheckpointsCopy.remove(oldPrimary.shardAllocationId);
|
|
|
oldPrimaryCheckpointsCopy.remove(newPrimary.shardAllocationId);
|
|
|
- Map<String, GlobalCheckpointTracker.CheckpointState> newPrimaryCheckpointsCopy = new HashMap<>(newPrimary.checkpoints);
|
|
|
+ Map<String, ReplicationTracker.CheckpointState> newPrimaryCheckpointsCopy = new HashMap<>(newPrimary.checkpoints);
|
|
|
newPrimaryCheckpointsCopy.remove(oldPrimary.shardAllocationId);
|
|
|
newPrimaryCheckpointsCopy.remove(newPrimary.shardAllocationId);
|
|
|
assertThat(newPrimaryCheckpointsCopy, equalTo(oldPrimaryCheckpointsCopy));
|
|
|
@@ -761,7 +767,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
public void testIllegalStateExceptionIfUnknownAllocationId() {
|
|
|
final AllocationId active = AllocationId.newInitializing();
|
|
|
final AllocationId initializing = AllocationId.newInitializing();
|
|
|
- final GlobalCheckpointTracker tracker = newTracker(active);
|
|
|
+ final ReplicationTracker tracker = newTracker(active);
|
|
|
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active.getId()),
|
|
|
routingTable(Collections.singleton(initializing), active), emptySet());
|
|
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
|
|
@@ -790,7 +796,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
.map(ShardRouting::allocationId).collect(Collectors.toSet());
|
|
|
}
|
|
|
|
|
|
- public void apply(GlobalCheckpointTracker gcp) {
|
|
|
+ public void apply(ReplicationTracker gcp) {
|
|
|
gcp.updateFromMaster(version, ids(inSyncIds), routingTable, Collections.emptySet());
|
|
|
}
|
|
|
}
|
|
|
@@ -818,20 +824,20 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
routingTable(initializingAllocationIds, primaryShard));
|
|
|
}
|
|
|
|
|
|
- private static void activatePrimary(GlobalCheckpointTracker gcp) {
|
|
|
+ private static void activatePrimary(ReplicationTracker gcp) {
|
|
|
gcp.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10));
|
|
|
}
|
|
|
|
|
|
- private static void randomLocalCheckpointUpdate(GlobalCheckpointTracker gcp) {
|
|
|
+ private static void randomLocalCheckpointUpdate(ReplicationTracker gcp) {
|
|
|
String allocationId = randomFrom(gcp.checkpoints.keySet());
|
|
|
long currentLocalCheckpoint = gcp.checkpoints.get(allocationId).getLocalCheckpoint();
|
|
|
gcp.updateLocalCheckpoint(allocationId, Math.max(SequenceNumbers.NO_OPS_PERFORMED, currentLocalCheckpoint + randomInt(5)));
|
|
|
}
|
|
|
|
|
|
- private static void randomMarkInSync(GlobalCheckpointTracker gcp) {
|
|
|
+ private static void randomMarkInSync(ReplicationTracker gcp) {
|
|
|
String allocationId = randomFrom(gcp.checkpoints.keySet());
|
|
|
long newLocalCheckpoint = Math.max(NO_OPS_PERFORMED, gcp.getGlobalCheckpoint() + randomInt(5));
|
|
|
- markAllocationIdAsInSyncQuietly(gcp, allocationId, newLocalCheckpoint);
|
|
|
+ markAsTrackingAndInSyncQuietly(gcp, allocationId, newLocalCheckpoint);
|
|
|
}
|
|
|
|
|
|
private static FakeClusterState randomUpdateClusterState(Set<String> allocationIds, FakeClusterState clusterState) {
|
|
|
@@ -876,9 +882,10 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
}).collect(Collectors.toSet());
|
|
|
}
|
|
|
|
|
|
- private static void markAllocationIdAsInSyncQuietly(
|
|
|
- final GlobalCheckpointTracker tracker, final String allocationId, final long localCheckpoint) {
|
|
|
+ private static void markAsTrackingAndInSyncQuietly(
|
|
|
+ final ReplicationTracker tracker, final String allocationId, final long localCheckpoint) {
|
|
|
try {
|
|
|
+ tracker.initiateTracking(allocationId);
|
|
|
tracker.markAllocationIdAsInSync(allocationId, localCheckpoint);
|
|
|
} catch (final InterruptedException e) {
|
|
|
throw new RuntimeException(e);
|