kgusakov commented on code in PR #2867:
URL: https://github.com/apache/ignite-3/pull/2867#discussion_r1430703193
##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1988,51 +1981,94 @@ protected CompletableFuture<Void>
handleChangeStableAssignmentEvent(WatchEvent e
// these updates always processing only 1 partition, so, only 1 stable
partition key.
assert evt.single() : evt;
+ if (evt.entryEvent().oldEntry() == null) {
+ // This means it's an event on table creation.
+ return nullCompletedFuture();
+ }
+
Entry stableAssignmentsWatchEvent = evt.entryEvent().newEntry();
+ long revision = evt.revision();
+
+ assert stableAssignmentsWatchEvent.revision() == revision :
stableAssignmentsWatchEvent;
+
if (stableAssignmentsWatchEvent.value() == null) {
return nullCompletedFuture();
}
+ return handleChangeStableAssignmentEvent(stableAssignmentsWatchEvent,
evt.revision(), false);
+ }
+
+ protected CompletableFuture<Void> handleChangeStableAssignmentEvent(
+ Entry stableAssignmentsWatchEvent,
+ long revision,
+ boolean isRecovery
+ ) {
int partitionId =
extractPartitionNumber(stableAssignmentsWatchEvent.key());
int tableId = extractTableId(stableAssignmentsWatchEvent.key(),
STABLE_ASSIGNMENTS_PREFIX);
TablePartitionId tablePartitionId = new TablePartitionId(tableId,
partitionId);
Set<Assignment> stableAssignments =
ByteUtils.fromBytes(stableAssignmentsWatchEvent.value());
- return metaStorageMgr.get(pendingPartAssignmentsKey(tablePartitionId),
stableAssignmentsWatchEvent.revision())
+ return metaStorageMgr.get(pendingPartAssignmentsKey(tablePartitionId),
revision)
.thenComposeAsync(pendingAssignmentsEntry -> {
- // Update raft client peers and learners according to the
actual assignments.
- CompletableFuture<Void> raftClientUpdateFuture =
tablesById(evt.revision()).thenAccept(t -> {
- t.get(tableId).internalTable()
-
.partitionRaftGroupService(tablePartitionId.partitionId())
-
.updateConfiguration(configurationFromAssignments(stableAssignments));
- });
-
byte[] pendingAssignmentsFromMetaStorage =
pendingAssignmentsEntry.value();
Set<Assignment> pendingAssignments =
pendingAssignmentsFromMetaStorage == null
? Set.of()
:
ByteUtils.fromBytes(pendingAssignmentsFromMetaStorage);
- String localMemberName = localNode().name();
+ return stopAndDestroyPartitionAndUpdateClients(
+ tablePartitionId,
+ stableAssignments,
+ pendingAssignments,
+ isRecovery,
+ revision
+ );
+ }, ioExecutor);
+ }
- boolean shouldStopLocalServices =
Stream.concat(stableAssignments.stream(), pendingAssignments.stream())
- .noneMatch(assignment ->
assignment.consistentId().equals(localMemberName));
+ private CompletableFuture<Void> updateClientsOnPartitionStop(
Review Comment:
It is hard to understand, why we need to update the clients on partition
stop. Why do you want to move it from the handleChangeStableAssignmentEvent to
stopAndDestroyPartitionAndUpdateClients?
##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -1191,6 +1207,90 @@ public void updateClusterCfgWithDefaultValue() {
startNodes(3);
}
+ @Test
+ public void destroyObsoleteStoragesOnRestart() throws InterruptedException
{
+ int nodesCount = 3;
+ List<IgniteImpl> nodes = startNodes(nodesCount);
+
+ int partitions = nodesCount;
+ int replicas = nodesCount;
+ createTableWithData(nodes, TABLE_NAME, replicas, partitions);
+
+ int restartedNodeIndex = nodesCount - 1;
+
+ WatchListenerInhibitor inhibitor =
WatchListenerInhibitor.metastorageEventsInhibitor(nodes.get(restartedNodeIndex));
+
+ inhibitor.startInhibit();
+
+ // Change the zone - one replica per partition.
+ alterZone(nodes.get(0).catalogManager(), String.format("ZONE_%s",
TABLE_NAME.toUpperCase()), 1);
+
+ stopNode(restartedNodeIndex);
+
+ inhibitor.stopInhibit();
+
+ IgniteImpl restartedNode = startNode(restartedNodeIndex);
+
+ TableImpl table = (TableImpl) restartedNode.tables().table(TABLE_NAME);
+
+ assertTrue(waitForCondition(() -> {
+ // Check that only storage for 1 partition left on the restarted
node.
+ return IntStream.range(0, partitions)
+ .mapToObj(i ->
table.internalTable().storage().getMvPartition(i))
+ .filter(Objects::nonNull)
+ .count() == 1;
+ }, 10_000));
+ }
+
+ @Test
+ public void testCorrectPartitionRecoveryOnSnapshot() throws
InterruptedException {
+ int nodesCount = 3;
+ List<IgniteImpl> nodes = startNodes(nodesCount);
+
+ int partitions = nodesCount;
+ int replicas = nodesCount;
+ createTableWithData(nodes, TABLE_NAME, replicas, partitions);
+
+ int restartedNodeIndex = nodesCount - 1;
+
+ WatchListenerInhibitor inhibitor =
WatchListenerInhibitor.metastorageEventsInhibitor(nodes.get(restartedNodeIndex));
+
+ inhibitor.startInhibit();
+
+ alterZone(nodes.get(0).catalogManager(), String.format("ZONE_%s",
TABLE_NAME.toUpperCase()), 1);
+
+ stopNode(restartedNodeIndex);
+
+ inhibitor.stopInhibit();
+
+ forceSnapshotUsageOnRestart(nodes.get(0));
+
+ IgniteImpl restartedNode = startNode(restartedNodeIndex);
+
+ TableImpl table = (TableImpl) restartedNode.tables().table(TABLE_NAME);
+
+ long recoveryRevision =
restartedNode.metaStorageManager().recoveryFinishedFuture().join();
+
+ for (int p = 0; p < partitions; p++) {
+ TablePartitionId tablePartitionId = new
TablePartitionId(table.tableId(), p);
+
+ Entry e =
restartedNode.metaStorageManager().getLocally(stablePartAssignmentsKey(tablePartitionId),
recoveryRevision);
+
+ Set<Assignment> assignment = ByteUtils.fromBytes(e.value());
+
+ boolean shouldBe = assignment.stream().anyMatch(n ->
n.consistentId().equals(restartedNode.name()));
+
+ PeersAndLearners configuration =
PeersAndLearners.fromConsistentIds(assignment.stream().map(a ->
a.consistentId()).collect(
+ Collectors.toSet()), Set.of());
+
+ Peer peer = configuration.peer(restartedNode.name());
+
+ boolean isStarted = peer == null ? false :
restartedNode.raftManager().isStarted(new RaftNodeId(tablePartitionId, peer));
Review Comment:
Could you explain this check? As I understand - we received assignments from
local metastore and check double times (with the different ways, but anyway),
if this assignments has the restarted node...
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]