You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ib...@apache.org on 2023/06/30 06:49:08 UTC
[ignite-3] branch main updated: IGNITE-19885 Simplify handleChangeStableAssignmentEvent in TableManager (#2270)
This is an automated email from the ASF dual-hosted git repository.
ibessonov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new e9cbb47621 IGNITE-19885 Simplify handleChangeStableAssignmentEvent in TableManager (#2270)
e9cbb47621 is described below
commit e9cbb47621199967f8a668d3d0d22c24ffcb4d17
Author: Roman Puchkovskiy <ro...@gmail.com>
AuthorDate: Fri Jun 30 10:49:03 2023 +0400
IGNITE-19885 Simplify handleChangeStableAssignmentEvent in TableManager (#2270)
---
.../internal/table/distributed/TableManager.java | 64 +++++++++++++---------
1 file changed, 38 insertions(+), 26 deletions(-)
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index 2c1e14280c..8cd61b2ca9 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -2507,40 +2507,52 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
boolean shouldStopLocalServices = Stream.concat(stableAssignments.stream(), pendingAssignments.stream())
.noneMatch(assignment -> assignment.consistentId().equals(localMemberName));
- if (!shouldStopLocalServices) {
+ if (shouldStopLocalServices) {
+ return stopAndDestroyPartition(tablePartitionId, evt.revision());
+ } else {
return completedFuture(null);
}
+ }, ioExecutor);
+ }
- try {
- raftMgr.stopRaftNodes(tablePartitionId);
- } catch (NodeStoppingException e) {
- // No-op
- }
+ private CompletableFuture<Void> stopAndDestroyPartition(TablePartitionId tablePartitionId, long revision) {
+ try {
+ raftMgr.stopRaftNodes(tablePartitionId);
+ } catch (NodeStoppingException e) {
+ // No-op
+ }
- CompletableFuture<Boolean> stopReplicaFut;
- try {
- stopReplicaFut = replicaMgr.stopReplica(tablePartitionId);
- } catch (NodeStoppingException e) {
- stopReplicaFut = completedFuture(true);
- }
+ CompletableFuture<Boolean> stopReplicaFut;
+ try {
+ stopReplicaFut = replicaMgr.stopReplica(tablePartitionId);
+ } catch (NodeStoppingException e) {
+ stopReplicaFut = completedFuture(true);
+ }
- CompletableFuture<Boolean> finalStopReplicaFut = stopReplicaFut;
+ return destroyPartitionStorages(tablePartitionId, revision, stopReplicaFut);
+ }
- return tablesById(evt.revision())
- // TODO: IGNITE-18703 Destroy raft log and meta
- .thenCombine(mvGc.removeStorage(tablePartitionId), (tables, unused) -> {
- InternalTable internalTable = tables.get(tableId).internalTable();
+ private CompletableFuture<Void> destroyPartitionStorages(
+ TablePartitionId tablePartitionId,
+ long revision,
+ CompletableFuture<Boolean> stopReplicaFut
+ ) {
+ int partitionId = tablePartitionId.partitionId();
- closePartitionTrackers(internalTable, partitionId);
+ return tablesById(revision)
+ // TODO: IGNITE-18703 Destroy raft log and meta
+ .thenCombine(mvGc.removeStorage(tablePartitionId), (tables, unused) -> {
+ InternalTable internalTable = tables.get(tablePartitionId.tableId()).internalTable();
- return allOf(
- finalStopReplicaFut,
- internalTable.storage().destroyPartition(partitionId),
- runAsync(() -> internalTable.txStateStorage().destroyTxStateStorage(partitionId), ioExecutor)
- );
- })
- .thenCompose(Function.identity());
- }, ioExecutor);
+ closePartitionTrackers(internalTable, partitionId);
+
+ return allOf(
+ stopReplicaFut,
+ internalTable.storage().destroyPartition(partitionId),
+ runAsync(() -> internalTable.txStateStorage().destroyTxStateStorage(partitionId), ioExecutor)
+ );
+ })
+ .thenCompose(Function.identity());
}
private static void handleExceptionOnCleanUpTablesResources(