|
@@ -19,18 +19,18 @@
|
|
|
|
|
|
package org.elasticsearch.index.seqno;
|
|
|
|
|
|
-import com.carrotsearch.hppc.ObjectLongHashMap;
|
|
|
-import com.carrotsearch.hppc.ObjectLongMap;
|
|
|
import org.elasticsearch.common.Randomness;
|
|
|
import org.elasticsearch.common.collect.Tuple;
|
|
|
+import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
|
|
+import org.elasticsearch.common.io.stream.StreamInput;
|
|
|
import org.elasticsearch.common.settings.Settings;
|
|
|
import org.elasticsearch.common.util.set.Sets;
|
|
|
-import org.elasticsearch.index.shard.PrimaryContext;
|
|
|
import org.elasticsearch.index.shard.ShardId;
|
|
|
import org.elasticsearch.test.ESTestCase;
|
|
|
import org.elasticsearch.test.IndexSettingsModule;
|
|
|
import org.junit.Before;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
@@ -46,15 +46,13 @@ import java.util.function.Function;
|
|
|
import java.util.stream.Collectors;
|
|
|
import java.util.stream.IntStream;
|
|
|
import java.util.stream.Stream;
|
|
|
-import java.util.stream.StreamSupport;
|
|
|
|
|
|
+import static java.util.Collections.emptySet;
|
|
|
+import static org.elasticsearch.index.seqno.SequenceNumbersService.NO_OPS_PERFORMED;
|
|
|
import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
|
|
-import static org.hamcrest.Matchers.containsString;
|
|
|
import static org.hamcrest.Matchers.equalTo;
|
|
|
import static org.hamcrest.Matchers.greaterThan;
|
|
|
-import static org.hamcrest.Matchers.hasToString;
|
|
|
import static org.hamcrest.Matchers.not;
|
|
|
-import static org.mockito.Mockito.mock;
|
|
|
|
|
|
public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
@@ -88,7 +86,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
public void testGlobalCheckpointUpdate() {
|
|
|
final long initialClusterStateVersion = randomNonNegativeLong();
|
|
|
Map<String, Long> allocations = new HashMap<>();
|
|
|
- Map<String, Long> activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5);
|
|
|
+ Map<String, Long> activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
Set<String> active = new HashSet<>(activeWithCheckpoints.keySet());
|
|
|
allocations.putAll(activeWithCheckpoints);
|
|
|
Map<String, Long> initializingWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5);
|
|
@@ -115,8 +113,9 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type);
|
|
|
});
|
|
|
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, active, initializing);
|
|
|
- initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, tracker.getGlobalCheckpoint()));
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion, active, initializing, emptySet());
|
|
|
+ tracker.activatePrimaryMode(active.iterator().next(), NO_OPS_PERFORMED);
|
|
|
+ initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, NO_OPS_PERFORMED));
|
|
|
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId)));
|
|
|
|
|
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(minLocalCheckpoint));
|
|
@@ -134,30 +133,37 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
|
// first check that adding it without the master blessing doesn't change anything.
|
|
|
tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
|
|
- assertThat(tracker.getLocalCheckpointForAllocationId(extraId), equalTo(UNASSIGNED_SEQ_NO));
|
|
|
+ assertNull(tracker.localCheckpoints.get(extraId));
|
|
|
+ expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(extraId));
|
|
|
|
|
|
- Set<String> newActive = new HashSet<>(active);
|
|
|
- newActive.add(extraId);
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 1, newActive, initializing);
|
|
|
+ Set<String> newInitializing = new HashSet<>(initializing);
|
|
|
+ newInitializing.add(extraId);
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion + 1, active, newInitializing, emptySet());
|
|
|
+
|
|
|
+ tracker.initiateTracking(extraId);
|
|
|
|
|
|
// now notify for the new id
|
|
|
- tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
|
|
+ if (randomBoolean()) {
|
|
|
+ tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
|
|
+ markAllocationIdAsInSyncQuietly(tracker, extraId, randomInt((int) minLocalCheckpointAfterUpdates));
|
|
|
+ } else {
|
|
|
+ markAllocationIdAsInSyncQuietly(tracker, extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
|
|
+ }
|
|
|
|
|
|
// now it should be incremented
|
|
|
assertThat(tracker.getGlobalCheckpoint(), greaterThan(minLocalCheckpoint));
|
|
|
}
|
|
|
|
|
|
public void testMissingActiveIdsPreventAdvance() {
|
|
|
- final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
+ final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(2, 5);
|
|
|
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(0, 5);
|
|
|
final Map<String, Long> assigned = new HashMap<>();
|
|
|
assigned.putAll(active);
|
|
|
assigned.putAll(initializing);
|
|
|
- tracker.updateAllocationIdsFromMaster(
|
|
|
- randomNonNegativeLong(),
|
|
|
- active.keySet(),
|
|
|
- initializing.keySet());
|
|
|
- randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
|
|
+ tracker.updateFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet(), emptySet());
|
|
|
+ String primary = active.keySet().iterator().next();
|
|
|
+ tracker.activatePrimaryMode(primary, NO_OPS_PERFORMED);
|
|
|
+ randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
|
|
final String missingActiveID = randomFrom(active.keySet());
|
|
|
assigned
|
|
|
.entrySet()
|
|
@@ -165,24 +171,27 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
.filter(e -> !e.getKey().equals(missingActiveID))
|
|
|
.forEach(e -> tracker.updateLocalCheckpoint(e.getKey(), e.getValue()));
|
|
|
|
|
|
- assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
|
-
|
|
|
+ if (missingActiveID.equals(primary) == false) {
|
|
|
+ assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
|
+ }
|
|
|
// now update all knowledge of all shards
|
|
|
assigned.forEach(tracker::updateLocalCheckpoint);
|
|
|
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
|
|
}
|
|
|
|
|
|
public void testMissingInSyncIdsPreventAdvance() {
|
|
|
- final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(0, 5);
|
|
|
- final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
- tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet());
|
|
|
- initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
|
|
- randomSubsetOf(randomInt(initializing.size() - 1),
|
|
|
- initializing.keySet()).forEach(aId -> tracker.updateLocalCheckpoint(aId, initializing.get(aId)));
|
|
|
+ final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
+ final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(2, 5);
|
|
|
+ logger.info("active: {}, initializing: {}", active, initializing);
|
|
|
+ tracker.updateFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet(), emptySet());
|
|
|
+ String primary = active.keySet().iterator().next();
|
|
|
+ tracker.activatePrimaryMode(primary, NO_OPS_PERFORMED);
|
|
|
+ randomSubsetOf(randomIntBetween(1, initializing.size() - 1),
|
|
|
+ initializing.keySet()).forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, NO_OPS_PERFORMED));
|
|
|
|
|
|
active.forEach(tracker::updateLocalCheckpoint);
|
|
|
|
|
|
- assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
|
+ assertThat(tracker.getGlobalCheckpoint(), equalTo(NO_OPS_PERFORMED));
|
|
|
|
|
|
// update again
|
|
|
initializing.forEach(tracker::updateLocalCheckpoint);
|
|
@@ -193,9 +202,11 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
final Map<String, Long> nonApproved = randomAllocationsWithLocalCheckpoints(1, 5);
|
|
|
- tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet());
|
|
|
- initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
|
|
- nonApproved.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
|
|
+ tracker.updateFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet(), emptySet());
|
|
|
+ tracker.activatePrimaryMode(active.keySet().iterator().next(), NO_OPS_PERFORMED);
|
|
|
+ initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
|
|
+ nonApproved.keySet().forEach(k ->
|
|
|
+ expectThrows(IllegalStateException.class, () -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED)));
|
|
|
|
|
|
List<Map<String, Long>> allocations = Arrays.asList(active, initializing, nonApproved);
|
|
|
Collections.shuffle(allocations, random());
|
|
@@ -221,11 +232,12 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
if (randomBoolean()) {
|
|
|
allocations.putAll(initializingToBeRemoved);
|
|
|
}
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, active, initializing);
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion, active, initializing, emptySet());
|
|
|
+ tracker.activatePrimaryMode(active.iterator().next(), NO_OPS_PERFORMED);
|
|
|
if (randomBoolean()) {
|
|
|
- initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
|
|
+ initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
|
|
} else {
|
|
|
- initializing.forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
|
|
+ initializing.forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
|
|
}
|
|
|
if (randomBoolean()) {
|
|
|
allocations.forEach(tracker::updateLocalCheckpoint);
|
|
@@ -233,11 +245,13 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
|
// now remove shards
|
|
|
if (randomBoolean()) {
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 1, activeToStay.keySet(), initializingToStay.keySet());
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion + 1, activeToStay.keySet(), initializingToStay.keySet(),
|
|
|
+ emptySet());
|
|
|
allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L));
|
|
|
} else {
|
|
|
allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L));
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 2, activeToStay.keySet(), initializingToStay.keySet());
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion + 2, activeToStay.keySet(), initializingToStay.keySet(),
|
|
|
+ emptySet());
|
|
|
}
|
|
|
|
|
|
final long checkpoint = Stream.concat(activeToStay.values().stream(), initializingToStay.values().stream())
|
|
@@ -246,16 +260,16 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(checkpoint));
|
|
|
}
|
|
|
|
|
|
- public void testWaitForAllocationIdToBeInSync() throws BrokenBarrierException, InterruptedException {
|
|
|
+ public void testWaitForAllocationIdToBeInSync() throws Exception {
|
|
|
final int localCheckpoint = randomIntBetween(1, 32);
|
|
|
final int globalCheckpoint = randomIntBetween(localCheckpoint + 1, 64);
|
|
|
final CyclicBarrier barrier = new CyclicBarrier(2);
|
|
|
final AtomicBoolean complete = new AtomicBoolean();
|
|
|
final String inSyncAllocationId =randomAlphaOfLength(16);
|
|
|
final String trackingAllocationId = randomAlphaOfLength(16);
|
|
|
- tracker.updateAllocationIdsFromMaster(
|
|
|
- randomNonNegativeLong(), Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId));
|
|
|
- tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint);
|
|
|
+ tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId),
|
|
|
+ Collections.singleton(trackingAllocationId), emptySet());
|
|
|
+ tracker.activatePrimaryMode(inSyncAllocationId, globalCheckpoint);
|
|
|
final Thread thread = new Thread(() -> {
|
|
|
try {
|
|
|
// synchronize starting with the test thread
|
|
@@ -279,18 +293,16 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
for (int i = 0; i < elements.size(); i++) {
|
|
|
tracker.updateLocalCheckpoint(trackingAllocationId, elements.get(i));
|
|
|
assertFalse(complete.get());
|
|
|
- assertTrue(awaitBusy(() -> tracker.trackingLocalCheckpoints.containsKey(trackingAllocationId)));
|
|
|
- assertTrue(awaitBusy(() -> tracker.pendingInSync.contains(trackingAllocationId)));
|
|
|
- assertFalse(tracker.inSyncLocalCheckpoints.containsKey(trackingAllocationId));
|
|
|
+ assertFalse(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId).inSync);
|
|
|
+ assertBusy(() -> assertTrue(tracker.pendingInSync.contains(trackingAllocationId)));
|
|
|
}
|
|
|
|
|
|
tracker.updateLocalCheckpoint(trackingAllocationId, randomIntBetween(globalCheckpoint, 64));
|
|
|
// synchronize with the waiting thread to mark that it is complete
|
|
|
barrier.await();
|
|
|
assertTrue(complete.get());
|
|
|
- assertTrue(tracker.trackingLocalCheckpoints.isEmpty());
|
|
|
- assertTrue(tracker.pendingInSync.isEmpty());
|
|
|
- assertTrue(tracker.inSyncLocalCheckpoints.containsKey(trackingAllocationId));
|
|
|
+ assertTrue(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId).inSync);
|
|
|
+ assertFalse(tracker.pendingInSync.contains(trackingAllocationId));
|
|
|
|
|
|
thread.join();
|
|
|
}
|
|
@@ -302,9 +314,9 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final AtomicBoolean interrupted = new AtomicBoolean();
|
|
|
final String inSyncAllocationId = randomAlphaOfLength(16);
|
|
|
final String trackingAllocationId = randomAlphaOfLength(32);
|
|
|
- tracker.updateAllocationIdsFromMaster(
|
|
|
- randomNonNegativeLong(), Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId));
|
|
|
- tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint);
|
|
|
+ tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId),
|
|
|
+ Collections.singleton(trackingAllocationId), emptySet());
|
|
|
+ tracker.activatePrimaryMode(inSyncAllocationId, globalCheckpoint);
|
|
|
final Thread thread = new Thread(() -> {
|
|
|
try {
|
|
|
// synchronize starting with the test thread
|
|
@@ -348,19 +360,25 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds);
|
|
|
final Set<String> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
|
|
final Set<String> initializingIds = activeAndInitializingAllocationIds.v2();
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, activeAllocationIds, initializingIds);
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion, activeAllocationIds, initializingIds, emptySet());
|
|
|
+ String primaryId = activeAllocationIds.iterator().next();
|
|
|
+ tracker.activatePrimaryMode(primaryId, NO_OPS_PERFORMED);
|
|
|
|
|
|
// first we assert that the in-sync and tracking sets are set up correctly
|
|
|
- assertTrue(activeAllocationIds.stream().allMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a)));
|
|
|
+ assertTrue(activeAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
|
|
assertTrue(
|
|
|
activeAllocationIds
|
|
|
.stream()
|
|
|
- .allMatch(a -> tracker.inSyncLocalCheckpoints.get(a) == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
- assertTrue(initializingIds.stream().allMatch(a -> tracker.trackingLocalCheckpoints.containsKey(a)));
|
|
|
+ .filter(a -> a.equals(primaryId) == false)
|
|
|
+ .allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
|
|
+ == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
+ assertTrue(initializingIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
|
|
assertTrue(
|
|
|
initializingIds
|
|
|
.stream()
|
|
|
- .allMatch(a -> tracker.trackingLocalCheckpoints.get(a) == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
+ .filter(a -> a.equals(primaryId) == false)
|
|
|
+ .allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
|
|
+ == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
|
|
|
// now we will remove some allocation IDs from these and ensure that they propagate through
|
|
|
final List<String> removingActiveAllocationIds = randomSubsetOf(activeAllocationIds);
|
|
@@ -369,29 +387,32 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
final List<String> removingInitializingAllocationIds = randomSubsetOf(initializingIds);
|
|
|
final Set<String> newInitializingAllocationIds =
|
|
|
initializingIds.stream().filter(a -> !removingInitializingAllocationIds.contains(a)).collect(Collectors.toSet());
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 1, newActiveAllocationIds, newInitializingAllocationIds);
|
|
|
- assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a)));
|
|
|
- assertTrue(removingActiveAllocationIds.stream().noneMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a)));
|
|
|
- assertTrue(newInitializingAllocationIds.stream().allMatch(a -> tracker.trackingLocalCheckpoints.containsKey(a)));
|
|
|
- assertTrue(removingInitializingAllocationIds.stream().noneMatch(a -> tracker.trackingLocalCheckpoints.containsKey(a)));
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion + 1, newActiveAllocationIds, newInitializingAllocationIds,
|
|
|
+ emptySet());
|
|
|
+ assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
|
|
+ assertTrue(removingActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a) == null));
|
|
|
+ assertTrue(newInitializingAllocationIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
|
|
+ assertTrue(removingInitializingAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a) == null));
|
|
|
|
|
|
/*
|
|
|
* Now we will add an allocation ID to each of active and initializing and ensure they propagate through. Using different lengths
|
|
|
* than we have been using above ensures that we can not collide with a previous allocation ID
|
|
|
*/
|
|
|
- newActiveAllocationIds.add(randomAlphaOfLength(32));
|
|
|
newInitializingAllocationIds.add(randomAlphaOfLength(64));
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 2, newActiveAllocationIds, newInitializingAllocationIds);
|
|
|
- assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a)));
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion + 2, newActiveAllocationIds, newInitializingAllocationIds, emptySet());
|
|
|
+ assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
|
|
assertTrue(
|
|
|
newActiveAllocationIds
|
|
|
.stream()
|
|
|
- .allMatch(a -> tracker.inSyncLocalCheckpoints.get(a) == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
- assertTrue(newInitializingAllocationIds.stream().allMatch(a -> tracker.trackingLocalCheckpoints.containsKey(a)));
|
|
|
+ .filter(a -> a.equals(primaryId) == false)
|
|
|
+ .allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
|
|
+ == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
+ assertTrue(newInitializingAllocationIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
|
|
assertTrue(
|
|
|
newInitializingAllocationIds
|
|
|
.stream()
|
|
|
- .allMatch(a -> tracker.trackingLocalCheckpoints.get(a) == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
+ .allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
|
|
+ == SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
|
|
|
// the tracking allocation IDs should play no role in determining the global checkpoint
|
|
|
final Map<String, Integer> activeLocalCheckpoints =
|
|
@@ -404,12 +425,12 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
activeLocalCheckpoints
|
|
|
.entrySet()
|
|
|
.stream()
|
|
|
- .allMatch(e -> tracker.getLocalCheckpointForAllocationId(e.getKey()) == e.getValue()));
|
|
|
+ .allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey()).getLocalCheckpoint() == e.getValue()));
|
|
|
assertTrue(
|
|
|
initializingLocalCheckpoints
|
|
|
.entrySet()
|
|
|
.stream()
|
|
|
- .allMatch(e -> tracker.trackingLocalCheckpoints.get(e.getKey()) == e.getValue()));
|
|
|
+ .allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey()).getLocalCheckpoint() == e.getValue()));
|
|
|
final long minimumActiveLocalCheckpoint = (long) activeLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
|
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(minimumActiveLocalCheckpoint));
|
|
|
final long minimumInitailizingLocalCheckpoint = (long) initializingLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
|
@@ -421,7 +442,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
// using a different length than we have been using above ensures that we can not collide with a previous allocation ID
|
|
|
final String newSyncingAllocationId = randomAlphaOfLength(128);
|
|
|
newInitializingAllocationIds.add(newSyncingAllocationId);
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 3, newActiveAllocationIds, newInitializingAllocationIds);
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion + 3, newActiveAllocationIds, newInitializingAllocationIds, emptySet());
|
|
|
final CyclicBarrier barrier = new CyclicBarrier(2);
|
|
|
final Thread thread = new Thread(() -> {
|
|
|
try {
|
|
@@ -439,7 +460,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
|
assertBusy(() -> {
|
|
|
assertTrue(tracker.pendingInSync.contains(newSyncingAllocationId));
|
|
|
- assertTrue(tracker.trackingLocalCheckpoints.containsKey(newSyncingAllocationId));
|
|
|
+ assertFalse(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId).inSync);
|
|
|
});
|
|
|
|
|
|
tracker.updateLocalCheckpoint(newSyncingAllocationId, randomIntBetween(Math.toIntExact(minimumActiveLocalCheckpoint), 1024));
|
|
@@ -447,17 +468,16 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
barrier.await();
|
|
|
|
|
|
assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId));
|
|
|
- assertFalse(tracker.trackingLocalCheckpoints.containsKey(newSyncingAllocationId));
|
|
|
- assertTrue(tracker.inSyncLocalCheckpoints.containsKey(newSyncingAllocationId));
|
|
|
+ assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId).inSync);
|
|
|
|
|
|
/*
|
|
|
* The new in-sync allocation ID is in the in-sync set now yet the master does not know this; the allocation ID should still be in
|
|
|
* the in-sync set even if we receive a cluster state update that does not reflect this.
|
|
|
*
|
|
|
*/
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 4, newActiveAllocationIds, newInitializingAllocationIds);
|
|
|
- assertFalse(tracker.trackingLocalCheckpoints.containsKey(newSyncingAllocationId));
|
|
|
- assertTrue(tracker.inSyncLocalCheckpoints.containsKey(newSyncingAllocationId));
|
|
|
+ tracker.updateFromMaster(initialClusterStateVersion + 4, newActiveAllocationIds, newInitializingAllocationIds, emptySet());
|
|
|
+ assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId).inSync);
|
|
|
+ assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -476,12 +496,11 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
|
|
|
final String active = randomAlphaOfLength(16);
|
|
|
final String initializing = randomAlphaOfLength(32);
|
|
|
- tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), Collections.singleton(active), Collections.singleton(initializing));
|
|
|
-
|
|
|
final CyclicBarrier barrier = new CyclicBarrier(4);
|
|
|
|
|
|
final int activeLocalCheckpoint = randomIntBetween(0, Integer.MAX_VALUE - 1);
|
|
|
- tracker.updateLocalCheckpoint(active, activeLocalCheckpoint);
|
|
|
+ tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active), Collections.singleton(initializing), emptySet());
|
|
|
+ tracker.activatePrimaryMode(active, activeLocalCheckpoint);
|
|
|
final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE);
|
|
|
final Thread activeThread = new Thread(() -> {
|
|
|
try {
|
|
@@ -523,205 +542,194 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
assertThat(tracker.getGlobalCheckpoint(), equalTo((long) nextActiveLocalCheckpoint));
|
|
|
}
|
|
|
|
|
|
- public void testPrimaryContextOlderThanAppliedClusterState() {
|
|
|
- final long initialClusterStateVersion = randomIntBetween(0, Integer.MAX_VALUE - 1) + 1;
|
|
|
- final int numberOfActiveAllocationsIds = randomIntBetween(0, 8);
|
|
|
- final int numberOfInitializingIds = randomIntBetween(0, 8);
|
|
|
- final Tuple<Set<String>, Set<String>> activeAndInitializingAllocationIds =
|
|
|
- randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds);
|
|
|
- final Set<String> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
|
|
- final Set<String> initializingAllocationIds = activeAndInitializingAllocationIds.v2();
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, activeAllocationIds, initializingAllocationIds);
|
|
|
+ public void testPrimaryContextHandoff() throws IOException {
|
|
|
+ GlobalCheckpointTracker oldPrimary = new GlobalCheckpointTracker(new ShardId("test", "_na_", 0),
|
|
|
+ IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO);
|
|
|
+ GlobalCheckpointTracker newPrimary = new GlobalCheckpointTracker(new ShardId("test", "_na_", 0),
|
|
|
+ IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO);
|
|
|
|
|
|
- /*
|
|
|
- * We are going to establish a primary context from a cluster state version older than the applied cluster state version on the
|
|
|
- * tracker. Because of recovery barriers established during relocation handoff, we know that the set of active allocation IDs in the
|
|
|
- * newer cluster state is a superset of the allocation IDs in the applied cluster state with the caveat that an existing
|
|
|
- * initializing allocation ID could have moved to an in-sync allocation ID within the tracker due to recovery finalization, and the
|
|
|
- * set of initializing allocation IDs is otherwise arbitrary.
|
|
|
- */
|
|
|
- final int numberOfAdditionalInitializingAllocationIds = randomIntBetween(0, 8);
|
|
|
- final Set<String> initializedAllocationIds = new HashSet<>(randomSubsetOf(initializingAllocationIds));
|
|
|
- final Set<String> newInitializingAllocationIds =
|
|
|
- randomAllocationIdsExcludingExistingIds(
|
|
|
- Sets.union(activeAllocationIds, initializingAllocationIds), numberOfAdditionalInitializingAllocationIds);
|
|
|
- final Set<String> contextInitializingIds = Sets.union(
|
|
|
- new HashSet<>(randomSubsetOf(Sets.difference(initializingAllocationIds, initializedAllocationIds))),
|
|
|
- newInitializingAllocationIds);
|
|
|
-
|
|
|
- final int numberOfAdditionalActiveAllocationIds = randomIntBetween(0, 8);
|
|
|
- final Set<String> contextActiveAllocationIds = Sets.union(
|
|
|
- Sets.union(
|
|
|
- activeAllocationIds,
|
|
|
- randomAllocationIdsExcludingExistingIds(activeAllocationIds, numberOfAdditionalActiveAllocationIds)),
|
|
|
- initializedAllocationIds);
|
|
|
-
|
|
|
- final ObjectLongMap<String> activeAllocationIdsLocalCheckpoints = new ObjectLongHashMap<>();
|
|
|
- for (final String allocationId : contextActiveAllocationIds) {
|
|
|
- activeAllocationIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong());
|
|
|
- }
|
|
|
- final ObjectLongMap<String> initializingAllocationIdsLocalCheckpoints = new ObjectLongHashMap<>();
|
|
|
- for (final String allocationId : contextInitializingIds) {
|
|
|
- initializingAllocationIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong());
|
|
|
+ FakeClusterState clusterState = initialState();
|
|
|
+ clusterState.apply(oldPrimary);
|
|
|
+ clusterState.apply(newPrimary);
|
|
|
+
|
|
|
+ activatePrimary(clusterState, oldPrimary);
|
|
|
+
|
|
|
+ final int numUpdates = randomInt(10);
|
|
|
+ for (int i = 0; i < numUpdates; i++) {
|
|
|
+ if (rarely()) {
|
|
|
+ clusterState = randomUpdateClusterState(clusterState);
|
|
|
+ clusterState.apply(oldPrimary);
|
|
|
+ clusterState.apply(newPrimary);
|
|
|
+ }
|
|
|
+ if (randomBoolean()) {
|
|
|
+ randomLocalCheckpointUpdate(oldPrimary);
|
|
|
+ }
|
|
|
+ if (randomBoolean()) {
|
|
|
+ randomMarkInSync(oldPrimary);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- final PrimaryContext primaryContext = new PrimaryContext(
|
|
|
- initialClusterStateVersion - randomIntBetween(0, Math.toIntExact(initialClusterStateVersion) - 1),
|
|
|
- activeAllocationIdsLocalCheckpoints,
|
|
|
- initializingAllocationIdsLocalCheckpoints);
|
|
|
-
|
|
|
- tracker.updateAllocationIdsFromPrimaryContext(primaryContext);
|
|
|
-
|
|
|
- // the primary context carries an older cluster state version
|
|
|
- assertThat(tracker.appliedClusterStateVersion, equalTo(initialClusterStateVersion));
|
|
|
-
|
|
|
- // only existing active allocation IDs and initializing allocation IDs that moved to initialized should be in-sync
|
|
|
- assertThat(
|
|
|
- Sets.union(activeAllocationIds, initializedAllocationIds),
|
|
|
- equalTo(
|
|
|
- StreamSupport
|
|
|
- .stream(tracker.inSyncLocalCheckpoints.keys().spliterator(), false)
|
|
|
- .map(e -> e.value)
|
|
|
- .collect(Collectors.toSet())));
|
|
|
-
|
|
|
- // the local checkpoints known to the tracker for in-sync shards should match what is known in the primary context
|
|
|
- for (final String allocationId : Sets.union(activeAllocationIds, initializedAllocationIds)) {
|
|
|
- assertThat(
|
|
|
- tracker.inSyncLocalCheckpoints.get(allocationId), equalTo(primaryContext.inSyncLocalCheckpoints().get(allocationId)));
|
|
|
+ GlobalCheckpointTracker.PrimaryContext primaryContext = oldPrimary.startRelocationHandoff();
|
|
|
+
|
|
|
+ if (randomBoolean()) {
|
|
|
+ // cluster state update after primary context handoff
|
|
|
+ if (randomBoolean()) {
|
|
|
+ clusterState = randomUpdateClusterState(clusterState);
|
|
|
+ clusterState.apply(oldPrimary);
|
|
|
+ clusterState.apply(newPrimary);
|
|
|
+ }
|
|
|
+
|
|
|
+ // abort handoff, check that we can continue updates and retry handoff
|
|
|
+ oldPrimary.abortRelocationHandoff();
|
|
|
+
|
|
|
+ if (rarely()) {
|
|
|
+ clusterState = randomUpdateClusterState(clusterState);
|
|
|
+ clusterState.apply(oldPrimary);
|
|
|
+ clusterState.apply(newPrimary);
|
|
|
+ }
|
|
|
+ if (randomBoolean()) {
|
|
|
+ randomLocalCheckpointUpdate(oldPrimary);
|
|
|
+ }
|
|
|
+ if (randomBoolean()) {
|
|
|
+ randomMarkInSync(oldPrimary);
|
|
|
+ }
|
|
|
+
|
|
|
+ // do another handoff
|
|
|
+ primaryContext = oldPrimary.startRelocationHandoff();
|
|
|
}
|
|
|
|
|
|
- // only existing initializing allocation IDs that did not moved to initialized should be tracked
|
|
|
- assertThat(
|
|
|
- Sets.difference(initializingAllocationIds, initializedAllocationIds),
|
|
|
- equalTo(
|
|
|
- StreamSupport
|
|
|
- .stream(tracker.trackingLocalCheckpoints.keys().spliterator(), false)
|
|
|
- .map(e -> e.value)
|
|
|
- .collect(Collectors.toSet())));
|
|
|
-
|
|
|
- // the local checkpoints known to the tracker for initializing shards should match what is known in the primary context
|
|
|
- for (final String allocationId : Sets.difference(initializingAllocationIds, initializedAllocationIds)) {
|
|
|
- if (primaryContext.trackingLocalCheckpoints().containsKey(allocationId)) {
|
|
|
- assertThat(
|
|
|
- tracker.trackingLocalCheckpoints.get(allocationId),
|
|
|
- equalTo(primaryContext.trackingLocalCheckpoints().get(allocationId)));
|
|
|
- } else {
|
|
|
- assertThat(tracker.trackingLocalCheckpoints.get(allocationId), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
|
+ // send primary context through the wire
|
|
|
+ BytesStreamOutput output = new BytesStreamOutput();
|
|
|
+ primaryContext.writeTo(output);
|
|
|
+ StreamInput streamInput = output.bytes().streamInput();
|
|
|
+ primaryContext = new GlobalCheckpointTracker.PrimaryContext(streamInput);
|
|
|
+
|
|
|
+ switch (randomInt(3)) {
|
|
|
+ case 0: {
|
|
|
+ // apply cluster state update on old primary while primary context is being transferred
|
|
|
+ clusterState = randomUpdateClusterState(clusterState);
|
|
|
+ clusterState.apply(oldPrimary);
|
|
|
+ // activate new primary
|
|
|
+ newPrimary.activateWithPrimaryContext(primaryContext);
|
|
|
+ // apply cluster state update on new primary so that the states on old and new primary are comparable
|
|
|
+ clusterState.apply(newPrimary);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case 1: {
|
|
|
+ // apply cluster state update on new primary while primary context is being transferred
|
|
|
+ clusterState = randomUpdateClusterState(clusterState);
|
|
|
+ clusterState.apply(newPrimary);
|
|
|
+ // activate new primary
|
|
|
+ newPrimary.activateWithPrimaryContext(primaryContext);
|
|
|
+ // apply cluster state update on old primary so that the states on old and new primary are comparable
|
|
|
+ clusterState.apply(oldPrimary);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case 2: {
|
|
|
+ // apply cluster state update on both copies while primary context is being transferred
|
|
|
+ clusterState = randomUpdateClusterState(clusterState);
|
|
|
+ clusterState.apply(oldPrimary);
|
|
|
+ clusterState.apply(newPrimary);
|
|
|
+ newPrimary.activateWithPrimaryContext(primaryContext);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case 3: {
|
|
|
+ // no cluster state update
|
|
|
+ newPrimary.activateWithPrimaryContext(primaryContext);
|
|
|
+ break;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // the global checkpoint can only be computed from active allocation IDs and initializing allocation IDs that moved to initializing
|
|
|
- final long globalCheckpoint =
|
|
|
- StreamSupport
|
|
|
- .stream(activeAllocationIdsLocalCheckpoints.spliterator(), false)
|
|
|
- .filter(e -> tracker.inSyncLocalCheckpoints.containsKey(e.key) || initializedAllocationIds.contains(e.key))
|
|
|
- .mapToLong(e -> e.value)
|
|
|
- .min()
|
|
|
- .orElse(SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
|
|
- assertThat(tracker.getGlobalCheckpoint(), equalTo(globalCheckpoint));
|
|
|
+ assertTrue(oldPrimary.primaryMode);
|
|
|
+ assertTrue(newPrimary.primaryMode);
|
|
|
+ assertThat(newPrimary.appliedClusterStateVersion, equalTo(oldPrimary.appliedClusterStateVersion));
|
|
|
+ assertThat(newPrimary.localCheckpoints, equalTo(oldPrimary.localCheckpoints));
|
|
|
+ assertThat(newPrimary.globalCheckpoint, equalTo(oldPrimary.globalCheckpoint));
|
|
|
+
|
|
|
+ oldPrimary.completeRelocationHandoff();
|
|
|
+ assertFalse(oldPrimary.primaryMode);
|
|
|
}
|
|
|
|
|
|
- public void testPrimaryContextNewerThanAppliedClusterState() {
|
|
|
- final long initialClusterStateVersion = randomIntBetween(0, Integer.MAX_VALUE);
|
|
|
- final int numberOfActiveAllocationsIds = randomIntBetween(0, 8);
|
|
|
- final int numberOfInitializingIds = randomIntBetween(0, 8);
|
|
|
- final Tuple<Set<String>, Set<String>> activeAndInitializingAllocationIds =
|
|
|
- randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds);
|
|
|
- final Set<String> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
|
|
- final Set<String> initializingAllocationIds = activeAndInitializingAllocationIds.v2();
|
|
|
- tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, activeAllocationIds, initializingAllocationIds);
|
|
|
+ public void testIllegalStateExceptionIfUnknownAllocationId() {
|
|
|
+ final String active = randomAlphaOfLength(16);
|
|
|
+ final String initializing = randomAlphaOfLength(32);
|
|
|
+ tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active), Collections.singleton(initializing), emptySet());
|
|
|
+ tracker.activatePrimaryMode(active, NO_OPS_PERFORMED);
|
|
|
|
|
|
- /*
|
|
|
- * We are going to establish a primary context from a cluster state version older than the applied cluster state version on the
|
|
|
- * tracker. Because of recovery barriers established during relocation handoff, we know that the set of active allocation IDs in the
|
|
|
- * newer cluster state is a subset of the allocation IDs in the applied cluster state with the caveat that an existing initializing
|
|
|
- * allocation ID could have moved to an in-sync allocation ID within the tracker due to recovery finalization, and the set of
|
|
|
- * initializing allocation IDs is otherwise arbitrary.
|
|
|
- */
|
|
|
- final int numberOfNewInitializingAllocationIds = randomIntBetween(0, 8);
|
|
|
- final Set<String> initializedAllocationIds = new HashSet<>(randomSubsetOf(initializingAllocationIds));
|
|
|
- final Set<String> newInitializingAllocationIds =
|
|
|
- randomAllocationIdsExcludingExistingIds(
|
|
|
- Sets.union(activeAllocationIds, initializingAllocationIds), numberOfNewInitializingAllocationIds);
|
|
|
+ expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(randomAlphaOfLength(10)));
|
|
|
+ expectThrows(IllegalStateException.class, () -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(10), randomNonNegativeLong()));
|
|
|
+ }
|
|
|
|
|
|
- final ObjectLongMap<String> activeAllocationIdsLocalCheckpoints = new ObjectLongHashMap<>();
|
|
|
- for (final String allocationId : Sets.union(new HashSet<>(randomSubsetOf(activeAllocationIds)), initializedAllocationIds)) {
|
|
|
- activeAllocationIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong());
|
|
|
- }
|
|
|
- final ObjectLongMap<String> initializingIdsLocalCheckpoints = new ObjectLongHashMap<>();
|
|
|
- final Set<String> contextInitializingAllocationIds = Sets.union(
|
|
|
- new HashSet<>(randomSubsetOf(Sets.difference(initializingAllocationIds, initializedAllocationIds))),
|
|
|
- newInitializingAllocationIds);
|
|
|
- for (final String allocationId : contextInitializingAllocationIds) {
|
|
|
- initializingIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong());
|
|
|
- }
|
|
|
+ private static class FakeClusterState {
|
|
|
+ final long version;
|
|
|
+ final Set<String> inSyncIds;
|
|
|
+ final Set<String> initializingIds;
|
|
|
|
|
|
- final PrimaryContext primaryContext =
|
|
|
- new PrimaryContext(
|
|
|
- initialClusterStateVersion + randomIntBetween(0, Integer.MAX_VALUE) + 1,
|
|
|
- activeAllocationIdsLocalCheckpoints,
|
|
|
- initializingIdsLocalCheckpoints);
|
|
|
+ private FakeClusterState(long version, Set<String> inSyncIds, Set<String> initializingIds) {
|
|
|
+ this.version = version;
|
|
|
+ this.inSyncIds = Collections.unmodifiableSet(inSyncIds);
|
|
|
+ this.initializingIds = Collections.unmodifiableSet(initializingIds);
|
|
|
+ }
|
|
|
|
|
|
- tracker.updateAllocationIdsFromPrimaryContext(primaryContext);
|
|
|
+ public Set<String> allIds() {
|
|
|
+ return Sets.union(initializingIds, inSyncIds);
|
|
|
+ }
|
|
|
|
|
|
- final PrimaryContext trackerPrimaryContext = tracker.primaryContext();
|
|
|
- try {
|
|
|
- assertTrue(tracker.sealed());
|
|
|
- final long globalCheckpoint =
|
|
|
- StreamSupport
|
|
|
- .stream(activeAllocationIdsLocalCheckpoints.values().spliterator(), false)
|
|
|
- .mapToLong(e -> e.value)
|
|
|
- .min()
|
|
|
- .orElse(SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
|
|
-
|
|
|
- // the primary context contains knowledge of the state of the entire universe
|
|
|
- assertThat(primaryContext.clusterStateVersion(), equalTo(trackerPrimaryContext.clusterStateVersion()));
|
|
|
- assertThat(primaryContext.inSyncLocalCheckpoints(), equalTo(trackerPrimaryContext.inSyncLocalCheckpoints()));
|
|
|
- assertThat(primaryContext.trackingLocalCheckpoints(), equalTo(trackerPrimaryContext.trackingLocalCheckpoints()));
|
|
|
- assertThat(tracker.getGlobalCheckpoint(), equalTo(globalCheckpoint));
|
|
|
- } finally {
|
|
|
- tracker.releasePrimaryContext();
|
|
|
- assertFalse(tracker.sealed());
|
|
|
+ public void apply(GlobalCheckpointTracker gcp) {
|
|
|
+ gcp.updateFromMaster(version, inSyncIds, initializingIds, Collections.emptySet());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void testPrimaryContextSealing() {
|
|
|
- // the tracker should start in the state of not being sealed
|
|
|
- assertFalse(tracker.sealed());
|
|
|
+ private static FakeClusterState initialState() {
|
|
|
+ final long initialClusterStateVersion = randomIntBetween(1, Integer.MAX_VALUE);
|
|
|
+ final int numberOfActiveAllocationsIds = randomIntBetween(1, 8);
|
|
|
+ final int numberOfInitializingIds = randomIntBetween(0, 8);
|
|
|
+ final Tuple<Set<String>, Set<String>> activeAndInitializingAllocationIds =
|
|
|
+ randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds);
|
|
|
+ final Set<String> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
|
|
+ final Set<String> initializingAllocationIds = activeAndInitializingAllocationIds.v2();
|
|
|
+ return new FakeClusterState(initialClusterStateVersion, activeAllocationIds, initializingAllocationIds);
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void activatePrimary(FakeClusterState clusterState, GlobalCheckpointTracker gcp) {
|
|
|
+ gcp.activatePrimaryMode(randomFrom(clusterState.inSyncIds), randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10));
|
|
|
+ }
|
|
|
|
|
|
- // sampling the primary context should seal the tracker
|
|
|
- tracker.primaryContext();
|
|
|
- assertTrue(tracker.sealed());
|
|
|
+ private static void randomLocalCheckpointUpdate(GlobalCheckpointTracker gcp) {
|
|
|
+ String allocationId = randomFrom(gcp.localCheckpoints.keySet());
|
|
|
+ long currentLocalCheckpoint = gcp.localCheckpoints.get(allocationId).getLocalCheckpoint();
|
|
|
+ gcp.updateLocalCheckpoint(allocationId, currentLocalCheckpoint + randomInt(5));
|
|
|
+ }
|
|
|
|
|
|
- /*
|
|
|
- * Invoking methods that mutates the state of the tracker should fail (with the exception of updating allocation IDs and updating
|
|
|
- * global checkpoint on replica which can happen on the relocation source).
|
|
|
- */
|
|
|
- assertIllegalStateExceptionWhenSealed(() -> tracker.updateLocalCheckpoint(randomAlphaOfLength(16), randomNonNegativeLong()));
|
|
|
- assertIllegalStateExceptionWhenSealed(() -> tracker.updateAllocationIdsFromPrimaryContext(mock(PrimaryContext.class)));
|
|
|
- assertIllegalStateExceptionWhenSealed(() -> tracker.primaryContext());
|
|
|
- assertIllegalStateExceptionWhenSealed(() -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(16), randomNonNegativeLong()));
|
|
|
-
|
|
|
- // closing the releasable should unseal the tracker
|
|
|
- tracker.releasePrimaryContext();
|
|
|
- assertFalse(tracker.sealed());
|
|
|
+ private static void randomMarkInSync(GlobalCheckpointTracker gcp) {
|
|
|
+ String allocationId = randomFrom(gcp.localCheckpoints.keySet());
|
|
|
+ long newLocalCheckpoint = Math.max(NO_OPS_PERFORMED, gcp.getGlobalCheckpoint() + randomInt(5));
|
|
|
+ markAllocationIdAsInSyncQuietly(gcp, allocationId, newLocalCheckpoint);
|
|
|
}
|
|
|
|
|
|
- private void assertIllegalStateExceptionWhenSealed(final ThrowingRunnable runnable) {
|
|
|
- final IllegalStateException e = expectThrows(IllegalStateException.class, runnable);
|
|
|
- assertThat(e, hasToString(containsString("global checkpoint tracker is sealed")));
|
|
|
+ private static FakeClusterState randomUpdateClusterState(FakeClusterState clusterState) {
|
|
|
+ final Set<String> initializingIdsToAdd = randomAllocationIdsExcludingExistingIds(clusterState.allIds(), randomInt(2));
|
|
|
+ final Set<String> initializingIdsToRemove = new HashSet<>(
|
|
|
+ randomSubsetOf(randomInt(clusterState.initializingIds.size()), clusterState.initializingIds));
|
|
|
+ final Set<String> inSyncIdsToRemove = new HashSet<>(
|
|
|
+ randomSubsetOf(randomInt(clusterState.inSyncIds.size()), clusterState.inSyncIds));
|
|
|
+ final Set<String> remainingInSyncIds = Sets.difference(clusterState.inSyncIds, inSyncIdsToRemove);
|
|
|
+ return new FakeClusterState(clusterState.version + randomIntBetween(1, 5),
|
|
|
+ remainingInSyncIds.isEmpty() ? clusterState.inSyncIds : remainingInSyncIds,
|
|
|
+ Sets.difference(Sets.union(clusterState.initializingIds, initializingIdsToAdd), initializingIdsToRemove));
|
|
|
}
|
|
|
|
|
|
- private Tuple<Set<String>, Set<String>> randomActiveAndInitializingAllocationIds(
|
|
|
+ private static Tuple<Set<String>, Set<String>> randomActiveAndInitializingAllocationIds(
|
|
|
final int numberOfActiveAllocationsIds,
|
|
|
final int numberOfInitializingIds) {
|
|
|
final Set<String> activeAllocationIds =
|
|
|
- IntStream.range(0, numberOfActiveAllocationsIds).mapToObj(i -> randomAlphaOfLength(16) + i).collect(Collectors.toSet());
|
|
|
+ IntStream.range(0, numberOfActiveAllocationsIds).mapToObj(i -> randomAlphaOfLength(16) + i).collect(Collectors.toSet());
|
|
|
final Set<String> initializingIds = randomAllocationIdsExcludingExistingIds(activeAllocationIds, numberOfInitializingIds);
|
|
|
return Tuple.tuple(activeAllocationIds, initializingIds);
|
|
|
}
|
|
|
|
|
|
- private Set<String> randomAllocationIdsExcludingExistingIds(final Set<String> existingAllocationIds, final int numberOfAllocationIds) {
|
|
|
+ private static Set<String> randomAllocationIdsExcludingExistingIds(final Set<String> existingAllocationIds,
|
|
|
+ final int numberOfAllocationIds) {
|
|
|
return IntStream.range(0, numberOfAllocationIds).mapToObj(i -> {
|
|
|
do {
|
|
|
final String newAllocationId = randomAlphaOfLength(16);
|
|
@@ -733,7 +741,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|
|
}).collect(Collectors.toSet());
|
|
|
}
|
|
|
|
|
|
- private void markAllocationIdAsInSyncQuietly(
|
|
|
+ private static void markAllocationIdAsInSyncQuietly(
|
|
|
final GlobalCheckpointTracker tracker, final String allocationId, final long localCheckpoint) {
|
|
|
try {
|
|
|
tracker.markAllocationIdAsInSync(allocationId, localCheckpoint);
|