This is an automated email from the ASF dual-hosted git repository. rpuch 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 2c586ec04d4 IGNITE-26117 Use non-durable Raft storages destruction for table destruction (#6363) 2c586ec04d4 is described below commit 2c586ec04d43629fc14bb107bc3f0c264381fa98 Author: Roman Puchkovskiy <roman.puchkovs...@gmail.com> AuthorDate: Tue Aug 5 16:59:23 2025 +0400 IGNITE-26117 Use non-durable Raft storages destruction for table destruction (#6363) --- .../ignite/internal/replicator/ReplicaManager.java | 18 ++++++++- .../internal/table/distributed/TableManager.java | 44 ++++++++++++++++------ 2 files changed, 49 insertions(+), 13 deletions(-) diff --git a/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java index 9e9df0d4aa5..4910d788894 100644 --- a/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java +++ b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java @@ -1225,12 +1225,26 @@ public class ReplicaManager extends AbstractEventProducer<LocalReplicaEvent, Loc */ public void destroyReplicationProtocolStoragesOnStartup(ReplicationGroupId replicaGrpId) throws NodeStoppingException { - RaftNodeId raftNodeId = new RaftNodeId(replicaGrpId, new Peer(localNodeConsistentId)); // We use 'isVolatileStorage' of false because on startup it's not a problem if the value is wrong. If it actually // was volatile, the log storage is already destroyed on an earlier phase of node startup, so we will just issue an excessive // log storage destruction request, and it's not a problem as persistent log storage with same table/zone ID cannot exist // if the storage was volatile. - RaftGroupOptions groupOptions = groupOptionsForPartition(false, null); + destroyReplicationProtocolStorages(replicaGrpId, false); + } + + /** + * Destroys replication protocol storages for the given group ID. + * + * <p>No durability guarantees are provided. If a node crashes, the storage may come to life. + * + * @param replicaGrpId Replication group ID. + * @param isVolatileStorage is table storage volatile? + * @throws NodeStoppingException If the node is being stopped. + */ + public void destroyReplicationProtocolStorages(ReplicationGroupId replicaGrpId, boolean isVolatileStorage) + throws NodeStoppingException { + RaftNodeId raftNodeId = new RaftNodeId(replicaGrpId, new Peer(localNodeConsistentId)); + RaftGroupOptions groupOptions = groupOptionsForPartition(isVolatileStorage, null); ((Loza) raftManager).destroyRaftNodeStorages(raftNodeId, groupOptions); } 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 76b7429b14f..8a06ea77424 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 @@ -2942,7 +2942,7 @@ public class TableManager implements IgniteTablesInternal, IgniteComponent { var tablePartitionId = new TablePartitionId(internalTable.tableId(), partitionId); stopReplicaAndDestroyFutures[partitionId] = resourcesUnloadFuture - .thenCompose(v -> stopAndDestroyTablePartition(tablePartitionId, table)); + .thenCompose(v -> stopAndDestroyTablePartition(tablePartitionId, table, true)); } return allOf(stopReplicaAndDestroyFutures).whenComplete((res, th) -> { @@ -2972,13 +2972,17 @@ public class TableManager implements IgniteTablesInternal, IgniteComponent { TableImpl table = tables.get(tablePartitionId.tableId()); assert table != null : tablePartitionId; - return stopAndDestroyTablePartition(tablePartitionId, table); + return stopAndDestroyTablePartition(tablePartitionId, table, false); }); } - private CompletableFuture<Void> stopAndDestroyTablePartition(TablePartitionId tablePartitionId, TableImpl table) { + private CompletableFuture<Void> stopAndDestroyTablePartition( + TablePartitionId tablePartitionId, + TableImpl table, + boolean destroyingWholeTable + ) { return stopTablePartition(tablePartitionId, table) - .thenComposeAsync(v -> destroyPartitionStorages(tablePartitionId, table), ioExecutor); + .thenComposeAsync(v -> destroyPartitionStorages(tablePartitionId, table, destroyingWholeTable), ioExecutor); } /** @@ -3009,7 +3013,7 @@ public class TableManager implements IgniteTablesInternal, IgniteComponent { return replicaMgr.weakStopReplica( tablePartitionId, WeakReplicaStopReason.RESTART, - () -> stopAndDestroyTablePartition(tablePartitionId, table) + () -> stopAndDestroyTablePartition(tablePartitionId, table, false) ); } @@ -3043,7 +3047,11 @@ public class TableManager implements IgniteTablesInternal, IgniteComponent { }); } - private CompletableFuture<Void> destroyPartitionStorages(TablePartitionId tablePartitionId, TableImpl table) { + private CompletableFuture<Void> destroyPartitionStorages( + TablePartitionId tablePartitionId, + TableImpl table, + boolean destroyingWholeTable + ) { InternalTable internalTable = table.internalTable(); int partitionId = tablePartitionId.partitionId(); @@ -3065,22 +3073,36 @@ public class TableManager implements IgniteTablesInternal, IgniteComponent { destroyFutures.add(runAsync(() -> internalTable.txStateStorage().destroyPartitionStorage(partitionId), ioExecutor)); } - destroyFutures.add(runAsync(() -> destroyReplicationProtocolStorages(tablePartitionId, table), ioExecutor)); + destroyFutures.add( + runAsync(() -> { + // No need for durability guarantees if destruction reliability is guaranteed by destroying table storages + // on startup. + destroyReplicationProtocolStorages(tablePartitionId, table, !destroyingWholeTable); + }, ioExecutor) + ); } // TODO: IGNITE-24926 - reduce set in localPartsByTableId after storages destruction. return allOf(destroyFutures.toArray(new CompletableFuture[]{})); } - private void destroyReplicationProtocolStorages(TablePartitionId tablePartitionId, TableImpl table) { + private void destroyReplicationProtocolStorages(TablePartitionId tablePartitionId, TableImpl table, boolean destroyDurably) { var internalTbl = (InternalTableImpl) table.internalTable(); - destroyReplicationProtocolStorages(tablePartitionId, internalTbl.storage().isVolatile()); + destroyReplicationProtocolStorages(tablePartitionId, internalTbl.storage().isVolatile(), destroyDurably); } - private void destroyReplicationProtocolStorages(TablePartitionId tablePartitionId, boolean isVolatileStorage) { + private void destroyReplicationProtocolStorages( + TablePartitionId tablePartitionId, + boolean isVolatileStorage, + boolean destroyDurably + ) { try { - replicaMgr.destroyReplicationProtocolStoragesDurably(tablePartitionId, isVolatileStorage); + if (destroyDurably) { + replicaMgr.destroyReplicationProtocolStoragesDurably(tablePartitionId, isVolatileStorage); + } else { + replicaMgr.destroyReplicationProtocolStorages(tablePartitionId, isVolatileStorage); + } } catch (NodeStoppingException e) { throw new IgniteInternalException(NODE_STOPPING_ERR, e); }